You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/11/17 21:20:33 UTC

[01/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Repository: hbase
Updated Branches:
  refs/heads/master ca74ec774 -> 330b0d05b


http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
new file mode 100644
index 0000000..d0b0081
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
@@ -0,0 +1,634 @@
+/*
+ *
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * Acts as the single ZooKeeper Watcher.  One instance of this is instantiated
+ * for each Master, RegionServer, and client process.
+ *
+ * <p>This is the only class that implements {@link Watcher}.  Other internal
+ * classes which need to be notified of ZooKeeper events must register with
+ * the local instance of this watcher via {@link #registerListener}.
+ *
+ * <p>This class also holds and manages the connection to ZooKeeper.  Code to
+ * deal with connection related events and exceptions are handled here.
+ */
+@InterfaceAudience.Private
+public class ZKWatcher implements Watcher, Abortable, Closeable {
+  private static final Log LOG = LogFactory.getLog(ZKWatcher.class);
+
+  // Identifier for this watcher (for logging only).  It is made of the prefix
+  // passed on construction and the zookeeper sessionid.
+  private String prefix;
+  private String identifier;
+
+  // zookeeper quorum
+  private String quorum;
+
+  // zookeeper connection
+  private final RecoverableZooKeeper recoverableZooKeeper;
+
+  // abortable in case of zk failure
+  protected Abortable abortable;
+  // Used if abortable is null
+  private boolean aborted = false;
+
+  public final ZNodePaths znodePaths;
+
+  // listeners to be notified
+  private final List<ZKListener> listeners = new CopyOnWriteArrayList<>();
+
+  // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
+  // negotiation to complete
+  public CountDownLatch saslLatch = new CountDownLatch(1);
+
+
+
+  private final Configuration conf;
+
+  /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */
+  private static final Pattern NAME_PATTERN = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
+
+  /**
+   * Instantiate a ZooKeeper connection and watcher.
+   * @param identifier string that is passed to RecoverableZookeeper to be used as
+   * identifier for this instance. Use null for default.
+   * @throws IOException
+   * @throws ZooKeeperConnectionException
+   */
+  public ZKWatcher(Configuration conf, String identifier,
+                   Abortable abortable) throws ZooKeeperConnectionException, IOException {
+    this(conf, identifier, abortable, false);
+  }
+
+  /**
+   * Instantiate a ZooKeeper connection and watcher.
+   * @param conf
+   * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
+   *          this instance. Use null for default.
+   * @param abortable Can be null if there is on error there is no host to abort: e.g. client
+   *          context.
+   * @param canCreateBaseZNode
+   * @throws IOException
+   * @throws ZooKeeperConnectionException
+   */
+  public ZKWatcher(Configuration conf, String identifier,
+                   Abortable abortable, boolean canCreateBaseZNode)
+  throws IOException, ZooKeeperConnectionException {
+    this.conf = conf;
+    this.quorum = ZKConfig.getZKQuorumServersString(conf);
+    this.prefix = identifier;
+    // Identifier will get the sessionid appended later below down when we
+    // handle the syncconnect event.
+    this.identifier = identifier + "0x0";
+    this.abortable = abortable;
+    this.znodePaths = new ZNodePaths(conf);
+    PendingWatcher pendingWatcher = new PendingWatcher();
+    this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier);
+    pendingWatcher.prepare(this);
+    if (canCreateBaseZNode) {
+      try {
+        createBaseZNodes();
+      } catch (ZooKeeperConnectionException zce) {
+        try {
+          this.recoverableZooKeeper.close();
+        } catch (InterruptedException ie) {
+          LOG.debug("Encountered InterruptedException when closing " + this.recoverableZooKeeper);
+          Thread.currentThread().interrupt();
+        }
+        throw zce;
+      }
+    }
+  }
+
+  private void createBaseZNodes() throws ZooKeeperConnectionException {
+    try {
+      // Create all the necessary "directories" of znodes
+      ZKUtil.createWithParents(this, znodePaths.baseZNode);
+      ZKUtil.createAndFailSilent(this, znodePaths.rsZNode);
+      ZKUtil.createAndFailSilent(this, znodePaths.drainingZNode);
+      ZKUtil.createAndFailSilent(this, znodePaths.tableZNode);
+      ZKUtil.createAndFailSilent(this, znodePaths.splitLogZNode);
+      ZKUtil.createAndFailSilent(this, znodePaths.backupMasterAddressesZNode);
+      ZKUtil.createAndFailSilent(this, znodePaths.tableLockZNode);
+      ZKUtil.createAndFailSilent(this, znodePaths.masterMaintZNode);
+    } catch (KeeperException e) {
+      throw new ZooKeeperConnectionException(
+          prefix("Unexpected KeeperException creating base node"), e);
+    }
+  }
+
+  /** Returns whether the znode is supposed to be readable by the client
+   * and DOES NOT contain sensitive information (world readable).*/
+  public boolean isClientReadable(String node) {
+    // Developer notice: These znodes are world readable. DO NOT add more znodes here UNLESS
+    // all clients need to access this data to work. Using zk for sharing data to clients (other
+    // than service lookup case is not a recommended design pattern.
+    return
+        node.equals(znodePaths.baseZNode) ||
+        znodePaths.isAnyMetaReplicaZNode(node) ||
+        node.equals(znodePaths.masterAddressZNode) ||
+        node.equals(znodePaths.clusterIdZNode)||
+        node.equals(znodePaths.rsZNode) ||
+        // /hbase/table and /hbase/table/foo is allowed, /hbase/table-lock is not
+        node.equals(znodePaths.tableZNode) ||
+        node.startsWith(znodePaths.tableZNode + "/");
+  }
+
+  /**
+   * On master start, we check the znode ACLs under the root directory and set the ACLs properly
+   * if needed. If the cluster goes from an unsecure setup to a secure setup, this step is needed
+   * so that the existing znodes created with open permissions are now changed with restrictive
+   * perms.
+   */
+  public void checkAndSetZNodeAcls() {
+    if (!ZKUtil.isSecureZooKeeper(getConfiguration())) {
+      LOG.info("not a secure deployment, proceeding");
+      return;
+    }
+
+    // Check the base znodes permission first. Only do the recursion if base znode's perms are not
+    // correct.
+    try {
+      List<ACL> actualAcls = recoverableZooKeeper.getAcl(znodePaths.baseZNode, new Stat());
+
+      if (!isBaseZnodeAclSetup(actualAcls)) {
+        LOG.info("setting znode ACLs");
+        setZnodeAclsRecursive(znodePaths.baseZNode);
+      }
+    } catch(KeeperException.NoNodeException nne) {
+      return;
+    } catch(InterruptedException ie) {
+      interruptedExceptionNoThrow(ie, false);
+    } catch (IOException|KeeperException e) {
+      LOG.warn("Received exception while checking and setting zookeeper ACLs", e);
+    }
+  }
+
+  /**
+   * Set the znode perms recursively. This will do post-order recursion, so that baseZnode ACLs
+   * will be set last in case the master fails in between.
+   * @param znode
+   */
+  private void setZnodeAclsRecursive(String znode) throws KeeperException, InterruptedException {
+    List<String> children = recoverableZooKeeper.getChildren(znode, false);
+
+    for (String child : children) {
+      setZnodeAclsRecursive(ZNodePaths.joinZNode(znode, child));
+    }
+    List<ACL> acls = ZKUtil.createACL(this, znode, true);
+    LOG.info("Setting ACLs for znode:" + znode + " , acl:" + acls);
+    recoverableZooKeeper.setAcl(znode, acls, -1);
+  }
+
+  /**
+   * Checks whether the ACLs returned from the base znode (/hbase) is set for secure setup.
+   * @param acls acls from zookeeper
+   * @return whether ACLs are set for the base znode
+   * @throws IOException
+   */
+  private boolean isBaseZnodeAclSetup(List<ACL> acls) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Checking znode ACLs");
+    }
+    String[] superUsers = conf.getStrings(Superusers.SUPERUSER_CONF_KEY);
+    // Check whether ACL set for all superusers
+    if (superUsers != null && !checkACLForSuperUsers(superUsers, acls)) {
+      return false;
+    }
+
+    // this assumes that current authenticated user is the same as zookeeper client user
+    // configured via JAAS
+    String hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
+
+    if (acls.isEmpty()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("ACL is empty");
+      }
+      return false;
+    }
+
+    for (ACL acl : acls) {
+      int perms = acl.getPerms();
+      Id id = acl.getId();
+      // We should only set at most 3 possible ACLs for 3 Ids. One for everyone, one for superuser
+      // and one for the hbase user
+      if (Ids.ANYONE_ID_UNSAFE.equals(id)) {
+        if (perms != Perms.READ) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
+              id, perms, Perms.READ));
+          }
+          return false;
+        }
+      } else if (superUsers != null && isSuperUserId(superUsers, id)) {
+        if (perms != Perms.ALL) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
+              id, perms, Perms.ALL));
+          }
+          return false;
+        }
+      } else if ("sasl".equals(id.getScheme())) {
+        String name = id.getId();
+        // If ZooKeeper recorded the Kerberos full name in the ACL, use only the shortname
+        Matcher match = NAME_PATTERN.matcher(name);
+        if (match.matches()) {
+          name = match.group(1);
+        }
+        if (name.equals(hbaseUser)) {
+          if (perms != Perms.ALL) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
+                id, perms, Perms.ALL));
+            }
+            return false;
+          }
+        } else {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Unexpected shortname in SASL ACL: " + id);
+          }
+          return false;
+        }
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("unexpected ACL id '" + id + "'");
+        }
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /*
+   * Validate whether ACL set for all superusers.
+   */
+  private boolean checkACLForSuperUsers(String[] superUsers, List<ACL> acls) {
+    for (String user : superUsers) {
+      boolean hasAccess = false;
+      // TODO: Validate super group members also when ZK supports setting node ACL for groups.
+      if (!AuthUtil.isGroupPrincipal(user)) {
+        for (ACL acl : acls) {
+          if (user.equals(acl.getId().getId())) {
+            if (acl.getPerms() == Perms.ALL) {
+              hasAccess = true;
+            } else {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug(String.format(
+                  "superuser '%s' does not have correct permissions: have 0x%x, want 0x%x",
+                  acl.getId().getId(), acl.getPerms(), Perms.ALL));
+              }
+            }
+            break;
+          }
+        }
+        if (!hasAccess) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /*
+   * Validate whether ACL ID is superuser.
+   */
+  public static boolean isSuperUserId(String[] superUsers, Id id) {
+    for (String user : superUsers) {
+      // TODO: Validate super group members also when ZK supports setting node ACL for groups.
+      if (!AuthUtil.isGroupPrincipal(user) && new Id("sasl", user).equals(id)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return this.identifier + ", quorum=" + quorum + ", baseZNode=" + znodePaths.baseZNode;
+  }
+
+  /**
+   * Adds this instance's identifier as a prefix to the passed <code>str</code>
+   * @param str String to amend.
+   * @return A new string with this instance's identifier as prefix: e.g.
+   * if passed 'hello world', the returned string could be
+   */
+  public String prefix(final String str) {
+    return this.toString() + " " + str;
+  }
+
+  /**
+   * Get the znodes corresponding to the meta replicas from ZK
+   * @return list of znodes
+   * @throws KeeperException
+   */
+  public List<String> getMetaReplicaNodes() throws KeeperException {
+    List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
+    List<String> metaReplicaNodes = new ArrayList<>(2);
+    if (childrenOfBaseNode != null) {
+      String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
+      for (String child : childrenOfBaseNode) {
+        if (child.startsWith(pattern)) metaReplicaNodes.add(child);
+      }
+    }
+    return metaReplicaNodes;
+  }
+
+  /**
+   * Register the specified listener to receive ZooKeeper events.
+   * @param listener
+   */
+  public void registerListener(ZKListener listener) {
+    listeners.add(listener);
+  }
+
+  /**
+   * Register the specified listener to receive ZooKeeper events and add it as
+   * the first in the list of current listeners.
+   * @param listener
+   */
+  public void registerListenerFirst(ZKListener listener) {
+    listeners.add(0, listener);
+  }
+
+  public void unregisterListener(ZKListener listener) {
+    listeners.remove(listener);
+  }
+
+  /**
+   * Clean all existing listeners
+   */
+  public void unregisterAllListeners() {
+    listeners.clear();
+  }
+
+  /**
+   * Get a copy of current registered listeners
+   */
+  public List<ZKListener> getListeners() {
+    return new ArrayList<>(listeners);
+  }
+
+  /**
+   * @return The number of currently registered listeners
+   */
+  public int getNumberOfListeners() {
+    return listeners.size();
+  }
+
+  /**
+   * Get the connection to ZooKeeper.
+   * @return connection reference to zookeeper
+   */
+  public RecoverableZooKeeper getRecoverableZooKeeper() {
+    return recoverableZooKeeper;
+  }
+
+  public void reconnectAfterExpiration() throws IOException, KeeperException, InterruptedException {
+    recoverableZooKeeper.reconnectAfterExpiration();
+  }
+
+  /**
+   * Get the quorum address of this instance.
+   * @return quorum string of this zookeeper connection instance
+   */
+  public String getQuorum() {
+    return quorum;
+  }
+
+  /**
+   * Get the znodePaths.
+   * <p>
+   * Mainly used for mocking as mockito can not mock a field access.
+   */
+  public ZNodePaths getZNodePaths() {
+    return znodePaths;
+  }
+
+  /**
+   * Method called from ZooKeeper for events and connection status.
+   * <p>
+   * Valid events are passed along to listeners.  Connection status changes
+   * are dealt with locally.
+   */
+  @Override
+  public void process(WatchedEvent event) {
+    LOG.debug(prefix("Received ZooKeeper Event, " +
+        "type=" + event.getType() + ", " +
+        "state=" + event.getState() + ", " +
+        "path=" + event.getPath()));
+
+    switch(event.getType()) {
+
+      // If event type is NONE, this is a connection status change
+      case None: {
+        connectionEvent(event);
+        break;
+      }
+
+      // Otherwise pass along to the listeners
+
+      case NodeCreated: {
+        for(ZKListener listener : listeners) {
+          listener.nodeCreated(event.getPath());
+        }
+        break;
+      }
+
+      case NodeDeleted: {
+        for(ZKListener listener : listeners) {
+          listener.nodeDeleted(event.getPath());
+        }
+        break;
+      }
+
+      case NodeDataChanged: {
+        for(ZKListener listener : listeners) {
+          listener.nodeDataChanged(event.getPath());
+        }
+        break;
+      }
+
+      case NodeChildrenChanged: {
+        for(ZKListener listener : listeners) {
+          listener.nodeChildrenChanged(event.getPath());
+        }
+        break;
+      }
+    }
+  }
+
+  // Connection management
+
+  /**
+   * Called when there is a connection-related event via the Watcher callback.
+   * <p>
+   * If Disconnected or Expired, this should shutdown the cluster. But, since
+   * we send a KeeperException.SessionExpiredException along with the abort
+   * call, it's possible for the Abortable to catch it and try to create a new
+   * session with ZooKeeper. This is what the client does in HCM.
+   * <p>
+   * @param event
+   */
+  private void connectionEvent(WatchedEvent event) {
+    switch(event.getState()) {
+      case SyncConnected:
+        this.identifier = this.prefix + "-0x" +
+          Long.toHexString(this.recoverableZooKeeper.getSessionId());
+        // Update our identifier.  Otherwise ignore.
+        LOG.debug(this.identifier + " connected");
+        break;
+
+      // Abort the server if Disconnected or Expired
+      case Disconnected:
+        LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
+        break;
+
+      case Expired:
+        String msg = prefix(this.identifier + " received expired from " +
+          "ZooKeeper, aborting");
+        // TODO: One thought is to add call to ZKListener so say,
+        // ZKNodeTracker can zero out its data values.
+        if (this.abortable != null) {
+          this.abortable.abort(msg, new KeeperException.SessionExpiredException());
+        }
+        break;
+
+      case ConnectedReadOnly:
+      case SaslAuthenticated:
+      case AuthFailed:
+        break;
+
+      default:
+        throw new IllegalStateException("Received event is not valid: " + event.getState());
+    }
+  }
+
+  /**
+   * Forces a synchronization of this ZooKeeper client connection.
+   * <p>
+   * Executing this method before running other methods will ensure that the
+   * subsequent operations are up-to-date and consistent as of the time that
+   * the sync is complete.
+   * <p>
+   * This is used for compareAndSwap type operations where we need to read the
+   * data of an existing node and delete or transition that node, utilizing the
+   * previously read version and data.  We want to ensure that the version read
+   * is up-to-date from when we begin the operation.
+   */
+  public void sync(String path) throws KeeperException {
+    this.recoverableZooKeeper.sync(path, null, null);
+  }
+
+  /**
+   * Handles KeeperExceptions in client calls.
+   * <p>
+   * This may be temporary but for now this gives one place to deal with these.
+   * <p>
+   * TODO: Currently this method rethrows the exception to let the caller handle
+   * <p>
+   * @param ke
+   * @throws KeeperException
+   */
+  public void keeperException(KeeperException ke)
+  throws KeeperException {
+    LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
+    throw ke;
+  }
+
+  /**
+   * Handles InterruptedExceptions in client calls.
+   * @param ie the InterruptedException instance thrown
+   * @throws KeeperException the exception to throw, transformed from the InterruptedException
+   */
+  public void interruptedException(InterruptedException ie) throws KeeperException {
+    interruptedExceptionNoThrow(ie, true);
+    // Throw a system error exception to let upper level handle it
+    throw new KeeperException.SystemErrorException();
+  }
+
+  /**
+   * Log the InterruptedException and interrupt current thread
+   * @param ie The IterruptedException to log
+   * @param throwLater Whether we will throw the exception latter
+   */
+  public void interruptedExceptionNoThrow(InterruptedException ie, boolean throwLater) {
+    LOG.debug(prefix("Received InterruptedException, will interrupt current thread"
+        + (throwLater ? " and rethrow a SystemErrorException" : "")),
+      ie);
+    // At least preserve interrupt.
+    Thread.currentThread().interrupt();
+  }
+
+  /**
+   * Close the connection to ZooKeeper.
+   *
+   */
+  @Override
+  public void close() {
+    try {
+      recoverableZooKeeper.close();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    if (this.abortable != null) this.abortable.abort(why, e);
+    else this.aborted = true;
+  }
+
+  @Override
+  public boolean isAborted() {
+    return this.abortable == null? this.aborted: this.abortable.isAborted();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
new file mode 100644
index 0000000..e67c9fd
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
@@ -0,0 +1,49 @@
+/*
+ * 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.atomic.AtomicReference;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestInstancePending {
+  @Test(timeout = 1000)
+  public void test() throws Exception {
+    final InstancePending<String> pending = new InstancePending<>();
+    final AtomicReference<String> getResultRef = new AtomicReference<>();
+
+    new Thread() {
+      @Override
+      public void run() {
+        getResultRef.set(pending.get());
+      }
+    }.start();
+
+    Thread.sleep(100);
+    Assert.assertNull(getResultRef.get());
+
+    pending.prepare("abc");
+    Thread.sleep(100);
+    Assert.assertEquals("abc", getResultRef.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java
new file mode 100644
index 0000000..2811cc5
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.java
@@ -0,0 +1,80 @@
+/*
+ *
+ * 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.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestZKMetrics {
+
+  @Test
+  public void testRegisterExceptions() {
+    MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class);
+    ZKMetrics metricsZK = new ZKMetrics(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);
+    ZKMetrics metricsZK = new ZKMetrics(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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
new file mode 100644
index 0000000..7006040
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
@@ -0,0 +1,113 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ *
+ */
+@Category({SmallTests.class})
+public class TestZKUtil {
+
+  @Test
+  public void testUnsecure() throws ZooKeeperConnectionException, IOException {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(Superusers.SUPERUSER_CONF_KEY, "user1");
+    String node = "/hbase/testUnsecure";
+    ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
+    List<ACL> aclList = ZKUtil.createACL(watcher, node, false);
+    Assert.assertEquals(aclList.size(), 1);
+    Assert.assertTrue(aclList.contains(Ids.OPEN_ACL_UNSAFE.iterator().next()));
+  }
+
+  @Test
+  public void testSecuritySingleSuperuser() throws ZooKeeperConnectionException, IOException {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(Superusers.SUPERUSER_CONF_KEY, "user1");
+    String node = "/hbase/testSecuritySingleSuperuser";
+    ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
+    List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
+    Assert.assertEquals(aclList.size(), 2); // 1+1, since ACL will be set for the creator by default
+    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
+    Assert.assertTrue(aclList.contains(Ids.CREATOR_ALL_ACL.iterator().next()));
+  }
+
+  @Test
+  public void testCreateACL() throws ZooKeeperConnectionException, IOException {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(Superusers.SUPERUSER_CONF_KEY, "user1,@group1,user2,@group2,user3");
+    String node = "/hbase/testCreateACL";
+    ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
+    List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
+    Assert.assertEquals(aclList.size(), 4); // 3+1, since ACL will be set for the creator by default
+    Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
+    Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group2"))));
+    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
+    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user2"))));
+    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user3"))));
+  }
+
+  @Test
+  public void testCreateACLWithSameUser() throws ZooKeeperConnectionException, IOException {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(Superusers.SUPERUSER_CONF_KEY, "user4,@group1,user5,user6");
+    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("user4"));
+    String node = "/hbase/testCreateACL";
+    ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
+    List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
+    Assert.assertEquals(aclList.size(), 3); // 3, since service user the same as one of superuser
+    Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
+    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("auth", ""))));
+    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user5"))));
+    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user6"))));
+  }
+
+  @Test(expected = KeeperException.SystemErrorException.class)
+  public void testInterruptedDuringAction()
+      throws ZooKeeperConnectionException, IOException, KeeperException, InterruptedException {
+    final RecoverableZooKeeper recoverableZk = Mockito.mock(RecoverableZooKeeper.class);
+    ZKWatcher zkw = new ZKWatcher(HBaseConfiguration.create(), "unittest", null) {
+      @Override
+      public RecoverableZooKeeper getRecoverableZooKeeper() {
+        return recoverableZk;
+      }
+    };
+    Mockito.doThrow(new InterruptedException()).when(recoverableZk)
+        .getChildren(zkw.znodePaths.baseZNode, null);
+    ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.baseZNode);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java
new file mode 100644
index 0000000..bd4575d
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java
@@ -0,0 +1,57 @@
+/**
+ * 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.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ SmallTests.class })
+public class TestZKWatcher {
+
+  @Test
+  public void testIsClientReadable() throws IOException {
+    ZKWatcher watcher =
+      new ZKWatcher(HBaseConfiguration.create(), "testIsClientReadable", null, false);
+
+    assertTrue(watcher.isClientReadable(watcher.znodePaths.baseZNode));
+    assertTrue(watcher.isClientReadable(watcher.znodePaths.getZNodeForReplica(0)));
+    assertTrue(watcher.isClientReadable(watcher.znodePaths.masterAddressZNode));
+    assertTrue(watcher.isClientReadable(watcher.znodePaths.clusterIdZNode));
+    assertTrue(watcher.isClientReadable(watcher.znodePaths.tableZNode));
+    assertTrue(
+      watcher.isClientReadable(ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, "foo")));
+    assertTrue(watcher.isClientReadable(watcher.znodePaths.rsZNode));
+
+    assertFalse(watcher.isClientReadable(watcher.znodePaths.tableLockZNode));
+    assertFalse(watcher.isClientReadable(watcher.znodePaths.balancerZNode));
+    assertFalse(watcher.isClientReadable(watcher.znodePaths.regionNormalizerZNode));
+    assertFalse(watcher.isClientReadable(watcher.znodePaths.clusterStateZNode));
+    assertFalse(watcher.isClientReadable(watcher.znodePaths.drainingZNode));
+    assertFalse(watcher.isClientReadable(watcher.znodePaths.splitLogZNode));
+    assertFalse(watcher.isClientReadable(watcher.znodePaths.backupMasterAddressesZNode));
+
+    watcher.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b423085..ee35212 100755
--- a/pom.xml
+++ b/pom.xml
@@ -92,6 +92,7 @@
     <module>hbase-metrics</module>
     <module>hbase-spark-it</module>
     <module>hbase-backup</module>
+    <module>hbase-zookeeper</module>
   </modules>
   <!--Add apache snapshots in case we want to use unreleased versions of plugins:
       e.g. surefire 2.18-SNAPSHOT-->
@@ -1789,6 +1790,11 @@
         <type>test-jar</type>
         <scope>test</scope>
       </dependency>
+      <dependency>
+        <artifactId>hbase-zookeeper</artifactId>
+        <groupId>org.apache.hbase</groupId>
+        <version>${project.version}</version>
+      </dependency>
       <!-- General dependencies -->
       <dependency>
         <groupId>com.github.stephenc.findbugs</groupId>


[11/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
deleted file mode 100644
index 61a5a42..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ /dev/null
@@ -1,2072 +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.IOException;
-import java.io.InputStreamReader;
-import java.io.PrintWriter;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.security.Superusers;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.util.KerberosUtil;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooDefs.Perms;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.client.ZooKeeperSaslClient;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.CreateRequest;
-import org.apache.zookeeper.proto.DeleteRequest;
-import org.apache.zookeeper.proto.SetDataRequest;
-import org.apache.zookeeper.server.ZooKeeperSaslServer;
-
-/**
- * Internal HBase utility class for ZooKeeper.
- *
- * <p>Contains only static methods and constants.
- *
- * <p>Methods all throw {@link KeeperException} if there is an unexpected
- * zookeeper exception, so callers of these methods must handle appropriately.
- * If ZK is required for the operation, the server will need to be aborted.
- */
-@InterfaceAudience.Private
-public class ZKUtil {
-  private static final Log LOG = LogFactory.getLog(ZKUtil.class);
-
-  private static int zkDumpConnectionTimeOut;
-
-  /**
-   * Creates a new connection to ZooKeeper, pulling settings and ensemble config
-   * from the specified configuration object using methods from {@link ZKConfig}.
-   *
-   * Sets the connection status monitoring watcher to the specified watcher.
-   *
-   * @param conf configuration to pull ensemble and other settings from
-   * @param watcher watcher to monitor connection changes
-   * @return connection to zookeeper
-   * @throws IOException if unable to connect to zk or config problem
-   */
-  public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
-  throws IOException {
-    String ensemble = ZKConfig.getZKQuorumServersString(conf);
-    return connect(conf, ensemble, watcher);
-  }
-
-  public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
-      Watcher watcher)
-  throws IOException {
-    return connect(conf, ensemble, watcher, null);
-  }
-
-  public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
-      Watcher watcher, final String identifier)
-  throws IOException {
-    if(ensemble == null) {
-      throw new IOException("Unable to determine ZooKeeper ensemble");
-    }
-    int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
-        HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble);
-    }
-    int retry = conf.getInt("zookeeper.recovery.retry", 3);
-    int retryIntervalMillis =
-      conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
-    int maxSleepTime = conf.getInt("zookeeper.recovery.retry.maxsleeptime", 60000);
-    zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
-        1000);
-    return new RecoverableZooKeeper(ensemble, timeout, watcher,
-        retry, retryIntervalMillis, maxSleepTime, identifier);
-  }
-
-  /**
-   * Log in the current zookeeper server process using the given configuration
-   * keys for the credential file and login principal.
-   *
-   * <p><strong>This is only applicable when running on secure hbase</strong>
-   * On regular HBase (without security features), this will safely be ignored.
-   * </p>
-   *
-   * @param conf The configuration data to use
-   * @param keytabFileKey Property key used to configure the path to the credential file
-   * @param userNameKey Property key used to configure the login principal
-   * @param hostname Current hostname to use in any credentials
-   * @throws IOException underlying exception from SecurityUtil.login() call
-   */
-  public static void loginServer(Configuration conf, String keytabFileKey,
-      String userNameKey, String hostname) throws IOException {
-    login(conf, keytabFileKey, userNameKey, hostname,
-          ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY,
-          JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME);
-  }
-
-  /**
-   * Log in the current zookeeper client using the given configuration
-   * keys for the credential file and login principal.
-   *
-   * <p><strong>This is only applicable when running on secure hbase</strong>
-   * On regular HBase (without security features), this will safely be ignored.
-   * </p>
-   *
-   * @param conf The configuration data to use
-   * @param keytabFileKey Property key used to configure the path to the credential file
-   * @param userNameKey Property key used to configure the login principal
-   * @param hostname Current hostname to use in any credentials
-   * @throws IOException underlying exception from SecurityUtil.login() call
-   */
-  public static void loginClient(Configuration conf, String keytabFileKey,
-      String userNameKey, String hostname) throws IOException {
-    login(conf, keytabFileKey, userNameKey, hostname,
-          ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
-          JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME);
-  }
-
-  /**
-   * Log in the current process using the given configuration keys for the
-   * credential file and login principal.
-   *
-   * <p><strong>This is only applicable when running on secure hbase</strong>
-   * On regular HBase (without security features), this will safely be ignored.
-   * </p>
-   *
-   * @param conf The configuration data to use
-   * @param keytabFileKey Property key used to configure the path to the credential file
-   * @param userNameKey Property key used to configure the login principal
-   * @param hostname Current hostname to use in any credentials
-   * @param loginContextProperty property name to expose the entry name
-   * @param loginContextName jaas entry name
-   * @throws IOException underlying exception from SecurityUtil.login() call
-   */
-  private static void login(Configuration conf, String keytabFileKey,
-      String userNameKey, String hostname,
-      String loginContextProperty, String loginContextName)
-      throws IOException {
-    if (!isSecureZooKeeper(conf))
-      return;
-
-    // User has specified a jaas.conf, keep this one as the good one.
-    // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf"
-    if (System.getProperty("java.security.auth.login.config") != null)
-      return;
-
-    // No keytab specified, no auth
-    String keytabFilename = conf.get(keytabFileKey);
-    if (keytabFilename == null) {
-      LOG.warn("no keytab specified for: " + keytabFileKey);
-      return;
-    }
-
-    String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
-    String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
-
-    // Initialize the "jaas.conf" for keyTab/principal,
-    // If keyTab is not specified use the Ticket Cache.
-    // and set the zookeeper login context name.
-    JaasConfiguration jaasConf = new JaasConfiguration(loginContextName,
-        principalName, keytabFilename);
-    javax.security.auth.login.Configuration.setConfiguration(jaasConf);
-    System.setProperty(loginContextProperty, loginContextName);
-  }
-
-  /**
-   * A JAAS configuration that defines the login modules that we want to use for login.
-   */
-  private static class JaasConfiguration extends javax.security.auth.login.Configuration {
-    private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME =
-      "zookeeper-server-keytab-kerberos";
-    private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
-      "zookeeper-client-keytab-kerberos";
-
-    private static final Map<String, String> BASIC_JAAS_OPTIONS = new HashMap<>();
-    static {
-      String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
-      if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
-        BASIC_JAAS_OPTIONS.put("debug", "true");
-      }
-    }
-
-    private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS = new HashMap<>();
-    static {
-      KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
-      KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
-      KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
-      KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
-    }
-
-    private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
-      new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
-                                LoginModuleControlFlag.REQUIRED,
-                                KEYTAB_KERBEROS_OPTIONS);
-
-    private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
-      new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN};
-
-    private javax.security.auth.login.Configuration baseConfig;
-    private final String loginContextName;
-    private final boolean useTicketCache;
-    private final String keytabFile;
-    private final String principal;
-
-    public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
-      this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
-    }
-
-    private JaasConfiguration(String loginContextName, String principal,
-                             String keytabFile, boolean useTicketCache) {
-      try {
-        this.baseConfig = javax.security.auth.login.Configuration.getConfiguration();
-      } catch (SecurityException e) {
-        this.baseConfig = null;
-      }
-      this.loginContextName = loginContextName;
-      this.useTicketCache = useTicketCache;
-      this.keytabFile = keytabFile;
-      this.principal = principal;
-      LOG.info("JaasConfiguration loginContextName=" + loginContextName +
-               " principal=" + principal + " useTicketCache=" + useTicketCache +
-               " keytabFile=" + keytabFile);
-    }
-
-    @Override
-    public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
-      if (loginContextName.equals(appName)) {
-        if (!useTicketCache) {
-          KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
-          KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
-        }
-        KEYTAB_KERBEROS_OPTIONS.put("principal", principal);
-        KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
-        return KEYTAB_KERBEROS_CONF;
-      }
-      if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
-      return(null);
-    }
-  }
-
-  //
-  // Helper methods
-  //
-  /**
-   * Returns the full path of the immediate parent of the specified node.
-   * @param node path to get parent of
-   * @return parent of path, null if passed the root node or an invalid node
-   */
-  public static String getParent(String node) {
-    int idx = node.lastIndexOf(ZNodePaths.ZNODE_PATH_SEPARATOR);
-    return idx <= 0 ? null : node.substring(0, idx);
-  }
-
-  /**
-   * Get the name of the current node from the specified fully-qualified path.
-   * @param path fully-qualified path
-   * @return name of the current node
-   */
-  public static String getNodeName(String path) {
-    return path.substring(path.lastIndexOf("/")+1);
-  }
-
-  //
-  // Existence checks and watches
-  //
-
-  /**
-   * Watch the specified znode for delete/create/change events.  The watcher is
-   * set whether or not the node exists.  If the node already exists, the method
-   * returns true.  If the node does not exist, the method returns false.
-   *
-   * @param zkw zk reference
-   * @param znode path of node to watch
-   * @return true if znode exists, false if does not exist or error
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static boolean watchAndCheckExists(ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
-    try {
-      Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw);
-      boolean exists = s != null ? true : false;
-      if (exists) {
-        LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode));
-      } else {
-        LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode));
-      }
-      return exists;
-    } catch (KeeperException e) {
-      LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
-      zkw.keeperException(e);
-      return false;
-    } catch (InterruptedException e) {
-      LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
-      zkw.interruptedException(e);
-      return false;
-    }
-  }
-
-  /**
-   * Watch the specified znode, but only if exists. Useful when watching
-   * for deletions. Uses .getData() (and handles NoNodeException) instead
-   * of .exists() to accomplish this, as .getData() will only set a watch if
-   * the znode exists.
-   * @param zkw zk reference
-   * @param znode path of node to watch
-   * @return true if the watch is set, false if node does not exists
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static boolean setWatchIfNodeExists(ZooKeeperWatcher zkw, String znode)
-      throws KeeperException {
-    try {
-      zkw.getRecoverableZooKeeper().getData(znode, true, null);
-      return true;
-    } catch (NoNodeException e) {
-      return false;
-    } catch (InterruptedException e) {
-      LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
-      zkw.interruptedException(e);
-      return false;
-    }
-  }
-
-  /**
-   * Check if the specified node exists.  Sets no watches.
-   *
-   * @param zkw zk reference
-   * @param znode path of node to watch
-   * @return version of the node if it exists, -1 if does not exist
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static int checkExists(ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
-    try {
-      Stat s = zkw.getRecoverableZooKeeper().exists(znode, null);
-      return s != null ? s.getVersion() : -1;
-    } catch (KeeperException e) {
-      LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
-      zkw.keeperException(e);
-      return -1;
-    } catch (InterruptedException e) {
-      LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
-      zkw.interruptedException(e);
-      return -1;
-    }
-  }
-
-  //
-  // Znode listings
-  //
-
-  /**
-   * Lists the children znodes of the specified znode.  Also sets a watch on
-   * the specified znode which will capture a NodeDeleted event on the specified
-   * znode as well as NodeChildrenChanged if any children of the specified znode
-   * are created or deleted.
-   *
-   * Returns null if the specified node does not exist.  Otherwise returns a
-   * list of children of the specified node.  If the node exists but it has no
-   * children, an empty list will be returned.
-   *
-   * @param zkw zk reference
-   * @param znode path of node to list and watch children of
-   * @return list of children of the specified node, an empty list if the node
-   *          exists but has no children, and null if the node does not exist
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static List<String> listChildrenAndWatchForNewChildren(
-      ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
-    try {
-      List<String> children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw);
-      return children;
-    } catch(KeeperException.NoNodeException ke) {
-      LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
-          "because node does not exist (not an error)"));
-      return null;
-    } catch (KeeperException e) {
-      LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
-      zkw.keeperException(e);
-      return null;
-    } catch (InterruptedException e) {
-      LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
-      zkw.interruptedException(e);
-      return null;
-    }
-  }
-
-  /**
-   * List all the children of the specified znode, setting a watch for children
-   * changes and also setting a watch on every individual child in order to get
-   * the NodeCreated and NodeDeleted events.
-   * @param zkw zookeeper reference
-   * @param znode node to get children of and watch
-   * @return list of znode names, null if the node doesn't exist
-   * @throws KeeperException
-   */
-  public static List<String> listChildrenAndWatchThem(ZooKeeperWatcher zkw,
-      String znode) throws KeeperException {
-    List<String> children = listChildrenAndWatchForNewChildren(zkw, znode);
-    if (children == null) {
-      return null;
-    }
-    for (String child : children) {
-      watchAndCheckExists(zkw, ZNodePaths.joinZNode(znode, child));
-    }
-    return children;
-  }
-
-  /**
-   * Lists the children of the specified znode without setting any watches.
-   *
-   * Sets no watches at all, this method is best effort.
-   *
-   * Returns an empty list if the node has no children.  Returns null if the
-   * parent node itself does not exist.
-   *
-   * @param zkw zookeeper reference
-   * @param znode node to get children
-   * @return list of data of children of specified znode, empty if no children,
-   *         null if parent does not exist
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static List<String> listChildrenNoWatch(ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
-    List<String> children = null;
-    try {
-      // List the children without watching
-      children = zkw.getRecoverableZooKeeper().getChildren(znode, null);
-    } catch(KeeperException.NoNodeException nne) {
-      return null;
-    } catch(InterruptedException ie) {
-      zkw.interruptedException(ie);
-    }
-    return children;
-  }
-
-  /**
-   * Simple class to hold a node path and node data.
-   * @deprecated Unused
-   */
-  @Deprecated
-  public static class NodeAndData {
-    private String node;
-    private byte [] data;
-    public NodeAndData(String node, byte [] data) {
-      this.node = node;
-      this.data = data;
-    }
-    public String getNode() {
-      return node;
-    }
-    public byte [] getData() {
-      return data;
-    }
-    @Override
-    public String toString() {
-      return node;
-    }
-    public boolean isEmpty() {
-      return (data == null || data.length == 0);
-    }
-  }
-
-  /**
-   * Checks if the specified znode has any children.  Sets no watches.
-   *
-   * Returns true if the node exists and has children.  Returns false if the
-   * node does not exist or if the node does not have any children.
-   *
-   * Used during master initialization to determine if the master is a
-   * failed-over-to master or the first master during initial cluster startup.
-   * If the directory for regionserver ephemeral nodes is empty then this is
-   * a cluster startup, if not then it is not cluster startup.
-   *
-   * @param zkw zk reference
-   * @param znode path of node to check for children of
-   * @return true if node has children, false if not or node does not exist
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static boolean nodeHasChildren(ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
-    try {
-      return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty();
-    } catch(KeeperException.NoNodeException ke) {
-      LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
-      "because node does not exist (not an error)"));
-      return false;
-    } catch (KeeperException e) {
-      LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
-      zkw.keeperException(e);
-      return false;
-    } catch (InterruptedException e) {
-      LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
-      zkw.interruptedException(e);
-      return false;
-    }
-  }
-
-  /**
-   * Get the number of children of the specified node.
-   *
-   * If the node does not exist or has no children, returns 0.
-   *
-   * Sets no watches at all.
-   *
-   * @param zkw zk reference
-   * @param znode path of node to count children of
-   * @return number of children of specified node, 0 if none or parent does not
-   *         exist
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static int getNumberOfChildren(ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
-    try {
-      Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null);
-      return stat == null ? 0 : stat.getNumChildren();
-    } catch(KeeperException e) {
-      LOG.warn(zkw.prefix("Unable to get children of node " + znode));
-      zkw.keeperException(e);
-    } catch(InterruptedException e) {
-      zkw.interruptedException(e);
-    }
-    return 0;
-  }
-
-  //
-  // Data retrieval
-  //
-
-  /**
-   * Get znode data. Does not set a watcher.
-   * @return ZNode data, null if the node does not exist or if there is an
-   *  error.
-   */
-  public static byte [] getData(ZooKeeperWatcher zkw, String znode)
-      throws KeeperException, InterruptedException {
-    try {
-      byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null);
-      logRetrievedMsg(zkw, znode, data, false);
-      return data;
-    } catch (KeeperException.NoNodeException e) {
-      LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
-          "because node does not exist (not an error)"));
-      return null;
-    } catch (KeeperException e) {
-      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
-      zkw.keeperException(e);
-      return null;
-    }
-  }
-
-  /**
-   * Get the data at the specified znode and set a watch.
-   *
-   * Returns the data and sets a watch if the node exists.  Returns null and no
-   * watch is set if the node does not exist or there is an exception.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @return data of the specified znode, or null
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
-    return getDataInternal(zkw, znode, null, true);
-  }
-
-  /**
-   * Get the data at the specified znode and set a watch.
-   *
-   * Returns the data and sets a watch if the node exists.  Returns null and no
-   * watch is set if the node does not exist or there is an exception.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param stat object to populate the version of the znode
-   * @return data of the specified znode, or null
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode,
-      Stat stat) throws KeeperException {
-    return getDataInternal(zkw, znode, stat, true);
-  }
-
-  private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat,
-      boolean watcherSet)
-      throws KeeperException {
-    try {
-      byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
-      logRetrievedMsg(zkw, znode, data, watcherSet);
-      return data;
-    } catch (KeeperException.NoNodeException e) {
-      // This log can get pretty annoying when we cycle on 100ms waits.
-      // Enable trace if you really want to see it.
-      LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " +
-        "because node does not exist (not an error)"));
-      return null;
-    } catch (KeeperException e) {
-      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
-      zkw.keeperException(e);
-      return null;
-    } catch (InterruptedException e) {
-      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
-      zkw.interruptedException(e);
-      return null;
-    }
-  }
-
-  /**
-   * Get the data at the specified znode without setting a watch.
-   *
-   * Returns the data if the node exists.  Returns null if the node does not
-   * exist.
-   *
-   * Sets the stats of the node in the passed Stat object.  Pass a null stat if
-   * not interested.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param stat node status to get if node exists
-   * @return data of the specified znode, or null if node does not exist
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static byte [] getDataNoWatch(ZooKeeperWatcher zkw, String znode,
-      Stat stat)
-  throws KeeperException {
-    try {
-      byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat);
-      logRetrievedMsg(zkw, znode, data, false);
-      return data;
-    } catch (KeeperException.NoNodeException e) {
-      LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
-          "because node does not exist (not necessarily an error)"));
-      return null;
-    } catch (KeeperException e) {
-      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
-      zkw.keeperException(e);
-      return null;
-    } catch (InterruptedException e) {
-      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
-      zkw.interruptedException(e);
-      return null;
-    }
-  }
-
-  /**
-   * Returns the date of child znodes of the specified znode.  Also sets a watch on
-   * the specified znode which will capture a NodeDeleted event on the specified
-   * znode as well as NodeChildrenChanged if any children of the specified znode
-   * are created or deleted.
-   *
-   * Returns null if the specified node does not exist.  Otherwise returns a
-   * list of children of the specified node.  If the node exists but it has no
-   * children, an empty list will be returned.
-   *
-   * @param zkw zk reference
-   * @param baseNode path of node to list and watch children of
-   * @return list of data of children of the specified node, an empty list if the node
-   *          exists but has no children, and null if the node does not exist
-   * @throws KeeperException if unexpected zookeeper exception
-   * @deprecated Unused
-   */
-  @Deprecated
-  public static List<NodeAndData> getChildDataAndWatchForNewChildren(
-      ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
-    List<String> nodes =
-      ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
-    if (nodes != null) {
-      List<NodeAndData> newNodes = new ArrayList<>();
-      for (String node : nodes) {
-        String nodePath = ZNodePaths.joinZNode(baseNode, node);
-        byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
-        newNodes.add(new NodeAndData(nodePath, data));
-      }
-      return newNodes;
-    }
-    return null;
-  }
-
-  /**
-   * Update the data of an existing node with the expected version to have the
-   * specified data.
-   *
-   * Throws an exception if there is a version mismatch or some other problem.
-   *
-   * Sets no watches under any conditions.
-   *
-   * @param zkw zk reference
-   * @param znode
-   * @param data
-   * @param expectedVersion
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.BadVersionException if version mismatch
-   * @deprecated Unused
-   */
-  @Deprecated
-  public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode,
-      byte [] data, int expectedVersion)
-  throws KeeperException {
-    try {
-      zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion);
-    } catch(InterruptedException ie) {
-      zkw.interruptedException(ie);
-    }
-  }
-
-  //
-  // Data setting
-  //
-
-  /**
-   * Sets the data of the existing znode to be the specified data.  Ensures that
-   * the current data has the specified expected version.
-   *
-   * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
-   *
-   * <p>If their is a version mismatch, method returns null.
-   *
-   * <p>No watches are set but setting data will trigger other watchers of this
-   * node.
-   *
-   * <p>If there is another problem, a KeeperException will be thrown.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param data data to set for node
-   * @param expectedVersion version expected when setting data
-   * @return true if data set, false if version mismatch
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static boolean setData(ZooKeeperWatcher zkw, String znode,
-      byte [] data, int expectedVersion)
-  throws KeeperException, KeeperException.NoNodeException {
-    try {
-      return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null;
-    } catch (InterruptedException e) {
-      zkw.interruptedException(e);
-      return false;
-    }
-  }
-
-  /**
-   * Set data into node creating node if it doesn't yet exist.
-   * Does not set watch.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param data data to set for node
-   * @throws KeeperException
-   */
-  public static void createSetData(final ZooKeeperWatcher zkw, final String znode,
-      final byte [] data)
-  throws KeeperException {
-    if (checkExists(zkw, znode) == -1) {
-      ZKUtil.createWithParents(zkw, znode, data);
-    } else {
-      ZKUtil.setData(zkw, znode, data);
-    }
-  }
-
-  /**
-   * Sets the data of the existing znode to be the specified data.  The node
-   * must exist but no checks are done on the existing data or version.
-   *
-   * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
-   *
-   * <p>No watches are set but setting data will trigger other watchers of this
-   * node.
-   *
-   * <p>If there is another problem, a KeeperException will be thrown.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param data data to set for node
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data)
-  throws KeeperException, KeeperException.NoNodeException {
-    setData(zkw, (SetData)ZKUtilOp.setData(znode, data));
-  }
-
-  private static void setData(ZooKeeperWatcher zkw, SetData setData)
-  throws KeeperException, KeeperException.NoNodeException {
-    SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord();
-    setData(zkw, sd.getPath(), sd.getData(), sd.getVersion());
-  }
-
-  /**
-   * Returns whether or not secure authentication is enabled
-   * (whether <code>hbase.security.authentication</code> is set to
-   * <code>kerberos</code>.
-   */
-  public static boolean isSecureZooKeeper(Configuration conf) {
-    // Detection for embedded HBase client with jaas configuration
-    // defined for third party programs.
-    try {
-      javax.security.auth.login.Configuration testConfig =
-          javax.security.auth.login.Configuration.getConfiguration();
-      if (testConfig.getAppConfigurationEntry("Client") == null
-          && testConfig.getAppConfigurationEntry(
-            JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME) == null
-          && testConfig.getAppConfigurationEntry(
-              JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME) == null
-          && conf.get(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL) == null
-          && conf.get(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL) == null) {
-
-        return false;
-      }
-    } catch(Exception e) {
-      // No Jaas configuration defined.
-      return false;
-    }
-
-    // Master & RSs uses hbase.zookeeper.client.*
-    return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"));
-  }
-
-  private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
-    return createACL(zkw, node, isSecureZooKeeper(zkw.getConfiguration()));
-  }
-
-  public static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node,
-    boolean isSecureZooKeeper) {
-    if (!node.startsWith(zkw.znodePaths.baseZNode)) {
-      return Ids.OPEN_ACL_UNSAFE;
-    }
-    if (isSecureZooKeeper) {
-      ArrayList<ACL> acls = new ArrayList<>();
-      // add permission to hbase supper user
-      String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY);
-      String hbaseUser = null;
-      try {
-        hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
-      } catch (IOException e) {
-        LOG.debug("Could not acquire current User.", e);
-      }
-      if (superUsers != null) {
-        List<String> groups = new ArrayList<>();
-        for (String user : superUsers) {
-          if (AuthUtil.isGroupPrincipal(user)) {
-            // TODO: Set node ACL for groups when ZK supports this feature
-            groups.add(user);
-          } else {
-            if(!user.equals(hbaseUser)) {
-              acls.add(new ACL(Perms.ALL, new Id("sasl", user)));
-            }
-          }
-        }
-        if (!groups.isEmpty()) {
-          LOG.warn("Znode ACL setting for group " + groups
-              + " is skipped, ZooKeeper doesn't support this feature presently.");
-        }
-      }
-      // Certain znodes are accessed directly by the client,
-      // so they must be readable by non-authenticated clients
-      if (zkw.isClientReadable(node)) {
-        acls.addAll(Ids.CREATOR_ALL_ACL);
-        acls.addAll(Ids.READ_ACL_UNSAFE);
-      } else {
-        acls.addAll(Ids.CREATOR_ALL_ACL);
-      }
-      return acls;
-    } else {
-      return Ids.OPEN_ACL_UNSAFE;
-    }
-  }
-
-  //
-  // Node creation
-  //
-
-  /**
-   *
-   * Set the specified znode to be an ephemeral node carrying the specified
-   * data.
-   *
-   * If the node is created successfully, a watcher is also set on the node.
-   *
-   * If the node is not created successfully because it already exists, this
-   * method will also set a watcher on the node.
-   *
-   * If there is another problem, a KeeperException will be thrown.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param data data of node
-   * @return true if node created, false if not, watch set in both cases
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw,
-      String znode, byte [] data)
-  throws KeeperException {
-    boolean ret = true;
-    try {
-      zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
-          CreateMode.EPHEMERAL);
-    } catch (KeeperException.NodeExistsException nee) {
-      ret = false;
-    } catch (InterruptedException e) {
-      LOG.info("Interrupted", e);
-      Thread.currentThread().interrupt();
-    }
-    if(!watchAndCheckExists(zkw, znode)) {
-      // It did exist but now it doesn't, try again
-      return createEphemeralNodeAndWatch(zkw, znode, data);
-    }
-    return ret;
-  }
-
-  /**
-   * Creates the specified znode to be a persistent node carrying the specified
-   * data.
-   *
-   * Returns true if the node was successfully created, false if the node
-   * already existed.
-   *
-   * If the node is created successfully, a watcher is also set on the node.
-   *
-   * If the node is not created successfully because it already exists, this
-   * method will also set a watcher on the node but return false.
-   *
-   * If there is another problem, a KeeperException will be thrown.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param data data of node
-   * @return true if node created, false if not, watch set in both cases
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static boolean createNodeIfNotExistsAndWatch(
-      ZooKeeperWatcher zkw, String znode, byte [] data)
-  throws KeeperException {
-    boolean ret = true;
-    try {
-      zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
-          CreateMode.PERSISTENT);
-    } catch (KeeperException.NodeExistsException nee) {
-      ret = false;
-    } catch (InterruptedException e) {
-      zkw.interruptedException(e);
-      return false;
-    }
-    try {
-      zkw.getRecoverableZooKeeper().exists(znode, zkw);
-    } catch (InterruptedException e) {
-      zkw.interruptedException(e);
-      return false;
-    }
-    return ret;
-  }
-
-  /**
-   * Creates the specified znode with the specified data but does not watch it.
-   *
-   * Returns the znode of the newly created node
-   *
-   * If there is another problem, a KeeperException will be thrown.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param data data of node
-   * @param createMode specifying whether the node to be created is ephemeral and/or sequential
-   * @return true name of the newly created znode or null
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static String createNodeIfNotExistsNoWatch(ZooKeeperWatcher zkw, String znode,
-      byte[] data, CreateMode createMode) throws KeeperException {
-
-    String createdZNode = null;
-    try {
-      createdZNode = zkw.getRecoverableZooKeeper().create(znode, data,
-          createACL(zkw, znode), createMode);
-    } catch (KeeperException.NodeExistsException nee) {
-      return znode;
-    } catch (InterruptedException e) {
-      zkw.interruptedException(e);
-      return null;
-    }
-    return createdZNode;
-  }
-
-  /**
-   * Creates the specified node with the specified data and watches it.
-   *
-   * <p>Throws an exception if the node already exists.
-   *
-   * <p>The node created is persistent and open access.
-   *
-   * <p>Returns the version number of the created node if successful.
-   *
-   * @param zkw zk reference
-   * @param znode path of node to create
-   * @param data data of node to create
-   * @return version of node created
-   * @throws KeeperException if unexpected zookeeper exception
-   * @throws KeeperException.NodeExistsException if node already exists
-   */
-  public static int createAndWatch(ZooKeeperWatcher zkw,
-      String znode, byte [] data)
-  throws KeeperException, KeeperException.NodeExistsException {
-    try {
-      zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
-          CreateMode.PERSISTENT);
-      Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw);
-      if (stat == null){
-        // Likely a race condition. Someone deleted the znode.
-        throw KeeperException.create(KeeperException.Code.SYSTEMERROR,
-            "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode);
-      }
-     return stat.getVersion();
-    } catch (InterruptedException e) {
-      zkw.interruptedException(e);
-      return -1;
-    }
-  }
-
-  /**
-   * Async creates the specified node with the specified data.
-   *
-   * <p>Throws an exception if the node already exists.
-   *
-   * <p>The node created is persistent and open access.
-   *
-   * @param zkw zk reference
-   * @param znode path of node to create
-   * @param data data of node to create
-   * @param cb
-   * @param ctx
-   */
-  public static void asyncCreate(ZooKeeperWatcher zkw,
-      String znode, byte [] data, final AsyncCallback.StringCallback cb,
-      final Object ctx) {
-    zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data,
-        createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx);
-  }
-
-  /**
-   * Creates the specified node, iff the node does not exist.  Does not set a
-   * watch and fails silently if the node already exists.
-   *
-   * The node created is persistent and open access.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static void createAndFailSilent(ZooKeeperWatcher zkw,
-      String znode) throws KeeperException {
-    createAndFailSilent(zkw, znode, new byte[0]);
-  }
-
-  /**
-   * Creates the specified node containing specified data, iff the node does not exist.  Does
-   * not set a watch and fails silently if the node already exists.
-   *
-   * The node created is persistent and open access.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @param data a byte array data to store in the znode
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static void createAndFailSilent(ZooKeeperWatcher zkw,
-      String znode, byte[] data)
-  throws KeeperException {
-    createAndFailSilent(zkw,
-        (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data));
-  }
-
-  private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs)
-  throws KeeperException {
-    CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord();
-    String znode = create.getPath();
-    try {
-      RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
-      if (zk.exists(znode, false) == null) {
-        zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags()));
-      }
-    } catch(KeeperException.NodeExistsException nee) {
-    } catch(KeeperException.NoAuthException nee){
-      try {
-        if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) {
-          // If we failed to create the file and it does not already exist.
-          throw(nee);
-        }
-      } catch (InterruptedException ie) {
-        zkw.interruptedException(ie);
-      }
-    } catch(InterruptedException ie) {
-      zkw.interruptedException(ie);
-    }
-  }
-
-  /**
-   * Creates the specified node and all parent nodes required for it to exist.
-   *
-   * No watches are set and no errors are thrown if the node already exists.
-   *
-   * The nodes created are persistent and open access.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static void createWithParents(ZooKeeperWatcher zkw, String znode)
-  throws KeeperException {
-    createWithParents(zkw, znode, new byte[0]);
-  }
-
-  /**
-   * Creates the specified node and all parent nodes required for it to exist.  The creation of
-   * parent znodes is not atomic with the leafe znode creation but the data is written atomically
-   * when the leaf node is created.
-   *
-   * No watches are set and no errors are thrown if the node already exists.
-   *
-   * The nodes created are persistent and open access.
-   *
-   * @param zkw zk reference
-   * @param znode path of node
-   * @throws KeeperException if unexpected zookeeper exception
-   */
-  public static void createWithParents(ZooKeeperWatcher zkw, String znode, byte[] data)
-  throws KeeperException {
-    try {
-      if(znode == null) {
-        return;
-      }
-      zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
-          CreateMode.PERSISTENT);
-    } catch(KeeperException.NodeExistsException nee) {
-      return;
-    } catch(KeeperException.NoNodeException nne) {
-      createWithParents(zkw, getParent(znode));
-      createWithParents(zkw, znode, data);
-    } catch(InterruptedException ie) {
-      zkw.interruptedException(ie);
-    }
-  }
-
-  //
-  // Deletes
-  //
-
-  /**
-   * Delete the specified node.  Sets no watches.  Throws all exceptions.
-   */
-  public static void deleteNode(ZooKeeperWatcher zkw, String node)
-  throws KeeperException {
-    deleteNode(zkw, node, -1);
-  }
-
-  /**
-   * Delete the specified node with the specified version.  Sets no watches.
-   * Throws all exceptions.
-   */
-  public static boolean deleteNode(ZooKeeperWatcher zkw, String node,
-      int version)
-  throws KeeperException {
-    try {
-      zkw.getRecoverableZooKeeper().delete(node, version);
-      return true;
-    } catch(KeeperException.BadVersionException bve) {
-      return false;
-    } catch(InterruptedException ie) {
-      zkw.interruptedException(ie);
-      return false;
-    }
-  }
-
-  /**
-   * Deletes the specified node.  Fails silent if the node does not exist.
-   * @param zkw
-   * @param node
-   * @throws KeeperException
-   */
-  public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node)
-  throws KeeperException {
-    deleteNodeFailSilent(zkw,
-      (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node));
-  }
-
-  private static void deleteNodeFailSilent(ZooKeeperWatcher zkw,
-      DeleteNodeFailSilent dnfs) throws KeeperException {
-    DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord();
-    try {
-      zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion());
-    } catch(KeeperException.NoNodeException nne) {
-    } catch(InterruptedException ie) {
-      zkw.interruptedException(ie);
-    }
-  }
-
-
-  /**
-   * Delete the specified node and all of it's children.
-   * <p>
-   * If the node does not exist, just returns.
-   * <p>
-   * Sets no watches. Throws all exceptions besides dealing with deletion of
-   * children.
-   */
-  public static void deleteNodeRecursively(ZooKeeperWatcher zkw, String node)
-  throws KeeperException {
-    deleteNodeRecursivelyMultiOrSequential(zkw, true, node);
-  }
-
-  /**
-   * Delete all the children of the specified node but not the node itself.
-   *
-   * Sets no watches.  Throws all exceptions besides dealing with deletion of
-   * children.
-   *
-   * @throws KeeperException
-   */
-  public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node)
-      throws KeeperException {
-    deleteChildrenRecursivelyMultiOrSequential(zkw, true, node);
-  }
-
-  /**
-   * Delete all the children of the specified node but not the node itself. This
-   * will first traverse the znode tree for listing the children and then delete
-   * these znodes using multi-update api or sequential based on the specified
-   * configurations.
-   * <p>
-   * Sets no watches. Throws all exceptions besides dealing with deletion of
-   * children.
-   * <p>
-   * If the following is true:
-   * <ul>
-   * <li>runSequentialOnMultiFailure is true
-   * </ul>
-   * on calling multi, we get a ZooKeeper exception that can be handled by a
-   * sequential call(*), we retry the operations one-by-one (sequentially).
-   *
-   * @param zkw
-   *          - zk reference
-   * @param runSequentialOnMultiFailure
-   *          - if true when we get a ZooKeeper exception that could retry the
-   *          operations one-by-one (sequentially)
-   * @param pathRoots
-   *          - path of the parent node(s)
-   * @throws KeeperException.NotEmptyException
-   *           if node has children while deleting
-   * @throws KeeperException
-   *           if unexpected ZooKeeper exception
-   * @throws IllegalArgumentException
-   *           if an invalid path is specified
-   */
-  public static void deleteChildrenRecursivelyMultiOrSequential(
-      ZooKeeperWatcher zkw, boolean runSequentialOnMultiFailure,
-      String... pathRoots) throws KeeperException {
-    if (pathRoots == null || pathRoots.length <= 0) {
-      LOG.warn("Given path is not valid!");
-      return;
-    }
-    List<ZKUtilOp> ops = new ArrayList<>();
-    for (String eachRoot : pathRoots) {
-      List<String> children = listChildrenBFSNoWatch(zkw, eachRoot);
-      // Delete the leaves first and eventually get rid of the root
-      for (int i = children.size() - 1; i >= 0; --i) {
-        ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
-      }
-    }
-    // atleast one element should exist
-    if (ops.size() > 0) {
-      multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
-    }
-  }
-
-  /**
-   * Delete the specified node and its children. This traverse the
-   * znode tree for listing the children and then delete
-   * these znodes including the parent using multi-update api or
-   * sequential based on the specified configurations.
-   * <p>
-   * Sets no watches. Throws all exceptions besides dealing with deletion of
-   * children.
-   * <p>
-   * If the following is true:
-   * <ul>
-   * <li>runSequentialOnMultiFailure is true
-   * </ul>
-   * on calling multi, we get a ZooKeeper exception that can be handled by a
-   * sequential call(*), we retry the operations one-by-one (sequentially).
-   *
-   * @param zkw
-   *          - zk reference
-   * @param runSequentialOnMultiFailure
-   *          - if true when we get a ZooKeeper exception that could retry the
-   *          operations one-by-one (sequentially)
-   * @param pathRoots
-   *          - path of the parent node(s)
-   * @throws KeeperException.NotEmptyException
-   *           if node has children while deleting
-   * @throws KeeperException
-   *           if unexpected ZooKeeper exception
-   * @throws IllegalArgumentException
-   *           if an invalid path is specified
-   */
-  public static void deleteNodeRecursivelyMultiOrSequential(ZooKeeperWatcher zkw,
-      boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException {
-    if (pathRoots == null || pathRoots.length <= 0) {
-      LOG.warn("Given path is not valid!");
-      return;
-    }
-    List<ZKUtilOp> ops = new ArrayList<>();
-    for (String eachRoot : pathRoots) {
-      // ZooKeeper Watches are one time triggers; When children of parent nodes are deleted
-      // recursively, must set another watch, get notified of delete node
-      List<String> children = listChildrenBFSAndWatchThem(zkw, eachRoot);
-      // Delete the leaves first and eventually get rid of the root
-      for (int i = children.size() - 1; i >= 0; --i) {
-        ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
-      }
-      try {
-        if (zkw.getRecoverableZooKeeper().exists(eachRoot, zkw) != null) {
-          ops.add(ZKUtilOp.deleteNodeFailSilent(eachRoot));
-        }
-      } catch (InterruptedException e) {
-        zkw.interruptedException(e);
-      }
-    }
-    // atleast one element should exist
-    if (ops.size() > 0) {
-      multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
-    }
-  }
-
-  /**
-   * BFS Traversal of all the children under path, with the entries in the list,
-   * in the same order as that of the traversal. Lists all the children without
-   * setting any watches.
-   *
-   * @param zkw
-   *          - zk reference
-   * @param znode
-   *          - path of node
-   * @return list of children znodes under the path
-   * @throws KeeperException
-   *           if unexpected ZooKeeper exception
-   */
-  private static List<String> listChildrenBFSNoWatch(ZooKeeperWatcher zkw,
-      final String znode) throws KeeperException {
-    Deque<String> queue = new LinkedList<>();
-    List<String> tree = new ArrayList<>();
-    queue.add(znode);
-    while (true) {
-      String node = queue.pollFirst();
-      if (node == null) {
-        break;
-      }
-      List<String> children = listChildrenNoWatch(zkw, node);
-      if (children == null) {
-        continue;
-      }
-      for (final String child : children) {
-        final String childPath = node + "/" + child;
-        queue.add(childPath);
-        tree.add(childPath);
-      }
-    }
-    return tree;
-  }
-
-  /**
-   * BFS Traversal of all the children under path, with the entries in the list,
-   * in the same order as that of the traversal.
-   * Lists all the children and set watches on to them.
-   *
-   * @param zkw
-   *          - zk reference
-   * @param znode
-   *          - path of node
-   * @return list of children znodes under the path
-   * @throws KeeperException
-   *           if unexpected ZooKeeper exception
-   */
-  private static List<String> listChildrenBFSAndWatchThem(ZooKeeperWatcher zkw, final String znode)
-      throws KeeperException {
-    Deque<String> queue = new LinkedList<>();
-    List<String> tree = new ArrayList<>();
-    queue.add(znode);
-    while (true) {
-      String node = queue.pollFirst();
-      if (node == null) {
-        break;
-      }
-      List<String> children = listChildrenAndWatchThem(zkw, node);
-      if (children == null) {
-        continue;
-      }
-      for (final String child : children) {
-        final String childPath = node + "/" + child;
-        queue.add(childPath);
-        tree.add(childPath);
-      }
-    }
-    return tree;
-  }
-
-  /**
-   * Represents an action taken by ZKUtil, e.g. createAndFailSilent.
-   * These actions are higher-level than ZKOp actions, which represent
-   * individual actions in the ZooKeeper API, like create.
-   */
-  public abstract static class ZKUtilOp {
-    private String path;
-
-    private ZKUtilOp(String path) {
-      this.path = path;
-    }
-
-    /**
-     * @return a createAndFailSilent ZKUtilOp
-     */
-    public static ZKUtilOp createAndFailSilent(String path, byte[] data) {
-      return new CreateAndFailSilent(path, data);
-    }
-
-    /**
-     * @return a deleteNodeFailSilent ZKUtilOP
-     */
-    public static ZKUtilOp deleteNodeFailSilent(String path) {
-      return new DeleteNodeFailSilent(path);
-    }
-
-    /**
-     * @return a setData ZKUtilOp
-     */
-    public static ZKUtilOp setData(String path, byte [] data) {
-      return new SetData(path, data);
-    }
-
-    /**
-     * @return path to znode where the ZKOp will occur
-     */
-    public String getPath() {
-      return path;
-    }
-
-    /**
-     * ZKUtilOp representing createAndFailSilent in ZooKeeper
-     * (attempt to create node, ignore error if already exists)
-     */
-    public static class CreateAndFailSilent extends ZKUtilOp {
-      private byte [] data;
-
-      private CreateAndFailSilent(String path, byte [] data) {
-        super(path);
-        this.data = data;
-      }
-
-      public byte[] getData() {
-        return data;
-      }
-
-      @Override
-      public boolean equals(Object o) {
-        if (this == o) return true;
-        if (!(o instanceof CreateAndFailSilent)) return false;
-
-        CreateAndFailSilent op = (CreateAndFailSilent) o;
-        return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
-      }
-
-      @Override
-      public int hashCode() {
-        int ret = 17 + getPath().hashCode() * 31;
-        return ret * 31 + Bytes.hashCode(data);
-      }
-    }
-
-    /**
-     * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper
-     * (attempt to delete node, ignore error if node doesn't exist)
-     */
-    public static class DeleteNodeFailSilent extends ZKUtilOp {
-      private DeleteNodeFailSilent(String path) {
-        super(path);
-      }
-
-      @Override
-      public boolean equals(Object o) {
-        if (this == o) return true;
-        if (!(o instanceof DeleteNodeFailSilent)) return false;
-
-        return super.equals(o);
-      }
-
-      @Override
-      public int hashCode() {
-        return getPath().hashCode();
-      }
-    }
-
-    /**
-     * ZKUtilOp representing setData in ZooKeeper
-     */
-    public static class SetData extends ZKUtilOp {
-      private byte [] data;
-
-      private SetData(String path, byte [] data) {
-        super(path);
-        this.data = data;
-      }
-
-      public byte[] getData() {
-        return data;
-      }
-
-      @Override
-      public boolean equals(Object o) {
-        if (this == o) return true;
-        if (!(o instanceof SetData)) return false;
-
-        SetData op = (SetData) o;
-        return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
-      }
-
-      @Override
-      public int hashCode() {
-        int ret = getPath().hashCode();
-        return ret * 31 + Bytes.hashCode(data);
-      }
-    }
-  }
-
-  /**
-   * Convert from ZKUtilOp to ZKOp
-   */
-  private static Op toZooKeeperOp(ZooKeeperWatcher zkw, ZKUtilOp op)
-  throws UnsupportedOperationException {
-    if(op == null) return null;
-
-    if (op instanceof CreateAndFailSilent) {
-      CreateAndFailSilent cafs = (CreateAndFailSilent)op;
-      return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()),
-        CreateMode.PERSISTENT);
-    } else if (op instanceof DeleteNodeFailSilent) {
-      DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op;
-      return Op.delete(dnfs.getPath(), -1);
-    } else if (op instanceof SetData) {
-      SetData sd = (SetData)op;
-      return Op.setData(sd.getPath(), sd.getData(), -1);
-    } else {
-      throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
-        + op.getClass().getName());
-    }
-  }
-
-  /**
-   * Use ZooKeeper's multi-update functionality.
-   *
-   * If all of the following are true:
-   * - runSequentialOnMultiFailure is true
-   * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*)
-   * Then:
-   * - we retry the operations one-by-one (sequentially)
-   *
-   * Note *: an example is receiving a NodeExistsException from a "create" call.  Without multi,
-   * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who
-   * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught).
-   * This will cause all operations in the multi to fail, however, because
-   * the NodeExistsException that zk.create throws will fail the multi transaction.
-   * In this case, if the previous conditions hold, the commands are run sequentially, which should
-   * result in the correct final state, but means that the operations will not run atomically.
-   *
-   * @throws KeeperException
-   */
-  public static void multiOrSequential(ZooKeeperWatcher zkw, List<ZKUtilOp> ops,
-      boolean runSequentialOnMultiFailure) throws KeeperException {
-    if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) {
-      LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always.");
-    }
-    if (ops == null) return;
-
-    List<Op> zkOps = new LinkedList<>();
-    for (ZKUtilOp op : ops) {
-      zkOps.add(toZooKeeperOp(zkw, op));
-    }
-    try {
-      zkw.getRecoverableZooKeeper().multi(zkOps);
-    } catch (KeeperException ke) {
-      switch (ke.code()) {
-      case NODEEXISTS:
-      case NONODE:
-      case BADVERSION:
-      case NOAUTH:
-        // if we get an exception that could be solved by running sequentially
-        // (and the client asked us to), then break out and run sequentially
-        if (runSequentialOnMultiFailure) {
-          LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
-              + "  Attempting to run operations sequentially because"
-              + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
-          processSequentially(zkw, ops);
-          break;
-        }
-      default:
-        throw ke;
-      }
-    } catch (InterruptedException ie) {
-      zkw.interruptedException(ie);
-    }
-  }
-
-  private static void processSequentially(ZooKeeperWatcher zkw, List<ZKUtilOp> ops)
-      throws KeeperException, NoNodeException {
-    for (ZKUtilOp op : ops) {
-      if (op instanceof CreateAndFailSilent) {
-        createAndFailSilent(zkw, (CreateAndFailSilent) op);
-      } else if (op instanceof DeleteNodeFailSilent) {
-        deleteNodeFailSilent(zkw, (DeleteNodeFailSilent) op);
-      } else if (op instanceof SetData) {
-        setData(zkw, (SetData) op);
-      } else {
-        throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
-            + op.getClass().getName());
-      }
-    }
-  }
-
-  //
-  // ZooKeeper cluster information
-  //
-
-  /** @return String dump of everything in ZooKeeper. */
-  public static String dump(ZooKeeperWatcher zkw) {
-    StringBuilder sb = new StringBuilder();
-    try {
-      sb.append("HBase is rooted at ").append(zkw.znodePaths.baseZNode);
-      sb.append("\nActive master address: ");
-      try {
-        sb.append(MasterAddressTracker.getMasterAddress(zkw));
-      } catch (IOException e) {
-        sb.append("<<FAILED LOOKUP: " + e.getMessage() + ">>");
-      }
-      sb.append("\nBackup master addresses:");
-      for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.backupMasterAddressesZNode)) {
-        sb.append("\n ").append(child);
-      }
-      sb.append("\nRegion server holding hbase:meta: "
-        + new MetaTableLocator().getMetaRegionLocation(zkw));
-      Configuration conf = HBaseConfiguration.create();
-      int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
-               HConstants.DEFAULT_META_REPLICA_NUM);
-      for (int i = 1; i < numMetaReplicas; i++) {
-        sb.append("\nRegion server holding hbase:meta, replicaId " + i + " "
-                    + new MetaTableLocator().getMetaRegionLocation(zkw, i));
-      }
-      sb.append("\nRegion servers:");
-      for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode)) {
-        sb.append("\n ").append(child);
-      }
-      try {
-        getReplicationZnodesDump(zkw, sb);
-      } catch (KeeperException ke) {
-        LOG.warn("Couldn't get the replication znode dump", ke);
-      }
-      sb.append("\nQuorum Server Statistics:");
-      String[] servers = zkw.getQuorum().split(",");
-      for (String server : servers) {
-        sb.append("\n ").append(server);
-        try {
-          String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut);
-
-          if (stat == null) {
-            sb.append("[Error] invalid quorum server: " + server);
-            break;
-          }
-
-          for (String s : stat) {
-            sb.append("\n  ").append(s);
-          }
-        } catch (Exception e) {
-          sb.append("\n  ERROR: ").append(e.getMessage());
-        }
-      }
-    } catch (KeeperException ke) {
-      sb.append("\nFATAL ZooKeeper Exception!\n");
-      sb.append("\n" + ke.getMessage());
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Appends replication znodes to the passed StringBuilder.
-   * @param zkw
-   * @param sb
-   * @throws KeeperException
-   */
-  private static void getReplicationZnodesDump(ZooKeeperWatcher zkw, StringBuilder sb)
-      throws KeeperException {
-    String replicationZnode = zkw.znodePaths.replicationZNode;
-    if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return;
-    // do a ls -r on this znode
-    sb.append("\n").append(replicationZnode).append(": ");
-    List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
-    for (String child : children) {
-      String znode = ZNodePaths.joinZNode(replicationZnode, child);
-      if (znode.equals(zkw.znodePaths.peersZNode)) {
-        appendPeersZnodes(zkw, znode, sb);
-      } else if (znode.equals(zkw.znodePaths.queuesZNode)) {
-        appendRSZnodes(zkw, znode, sb);
-      } else if (znode.equals(zkw.znodePaths.hfileRefsZNode)) {
-        appendHFileRefsZnodes(zkw, znode, sb);
-      }
-    }
-  }
-
-  private static void appendHFileRefsZnodes(ZooKeeperWatcher zkw, String hfileRefsZnode,
-      StringBuilder sb) throws KeeperException {
-    sb.append("\n").append(hfileRefsZnode).append(": ");
-    for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, hfileRefsZnode)) {
-      String znodeToProcess = ZNodePaths.joinZNode(hfileRefsZnode, peerIdZnode);
-      sb.append("\n").append(znodeToProcess).append(": ");
-      List<String> peerHFileRefsZnodes = ZKUtil.listChildrenNoWatch(zkw, znodeToProcess);
-      int size = peerHFileRefsZnodes.size();
-      for (int i = 0; i < size; i++) {
-        sb.append(peerHFileRefsZnodes.get(i));
-        if (i != size - 1) {
-          sb.append(", ");
-        }
-      }
-    }
-  }
-
-  /**
-   * Returns a string with replication znodes and position of the replication log
-   * @param zkw
-   * @return aq string of replication znodes and log positions
-   */
-  public static String getReplicationZnodesDump(ZooKeeperWatcher zkw) throws KeeperException {
-    StringBuilder sb = new StringBuilder();
-    getReplicationZnodesDump(zkw, sb);
-    return sb.toString();
-  }
-
-  private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb)
-      throws KeeperException {
-    List<String> stack = new LinkedList<>();
-    stack.add(znode);
-    do {
-      String znodeToProcess = stack.remove(stack.size() - 1);
-      sb.append("\n").append(znodeToProcess).append(": ");
-      byte[] data;
-      try {
-        data = ZKUtil.getData(zkw, znodeToProcess);
-      } catch (InterruptedException e) {
-        zkw.interruptedException(e);
-        return;
-      }
-      if (data != null && data.length > 0) { // log position
-        long position = 0;
-        try {
-          position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
-          sb.append(position);
-        } catch (DeserializationException ignored) {
-        } catch (InterruptedException e) {
-          zkw.interruptedException(e);
-          return;
-        }
-      }
-      for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
-        stack.add(ZNodePaths.joinZNode(znodeToProcess, zNodeChild));
-      }
-    } while (stack.size() > 0);
-  }
-
-  private static void appendPeersZnodes(ZooKeeperWatcher zkw, String peersZnode,
-    StringBuilder sb) throws KeeperException {
-    int pblen = ProtobufUtil.lengthOfPBMagic();
-    sb.append("\n").append(peersZnode).append(": ");
-    for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
-      String znodeToProcess = ZNodePaths.joinZNode(peersZnode, peerIdZnode);
-      byte[] data;
-      try {
-        data = ZKUtil.getData(zkw, znodeToProcess);
-      } catch (InterruptedException e) {
-        zkw.interruptedException(e);
-        return;
-      }
-      // parse the data of the above peer znode.
-      try {
-        ReplicationProtos.ReplicationPeer.Builder builder =
-          ReplicationProtos.ReplicationPeer.newBuilder();
-        ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
-        String clusterKey = builder.getClusterkey();
-        sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
-        // add the peer-state.
-        appendPeerState(zkw, znodeToProcess, sb);
-      } catch (IOException ipbe) {
-        LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
-      }
-    }
-  }
-
-  private static void appendPeerState(ZooKeeperWatcher zkw, String znodeToProcess,
-      StringBuilder sb) throws KeeperException, InvalidProtocolBufferException {
-    String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state",
-      "peer-state");
-    int pblen = ProtobufUtil.lengthOfPBMagic();
-    for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
-      if (!child.equals(peerState)) continue;
-      String peerStateZnode = ZNodePaths.joinZNode(znodeToProcess, child);
-      sb.append("\n").append(peerStateZnode).append(": ");
-      byte[] peerStateData;
-      try {
-        peerStateData = ZKUtil.getData(zkw, peerStateZnode);
-        ReplicationProtos.ReplicationState.Builder builder =
-            ReplicationProtos.ReplicationState.newBuilder();
-        ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen);
-        sb.append(builder.getState().name());
-      } catch (IOException ipbe) {
-        LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
-      } catch (InterruptedException e) {
-        zkw.interruptedException(e);
-        return;
-      }
-    }
-  }
-
-  /**
-   * Gets the statistics from the given server.
-   *
-   * @param server  The server to get the statistics from.
-   * @param timeout  The socket timeout to use.
-   * @return The array of response strings.
-   * @throws IOException When the socket communication fails.
-   */
-  public static String[] getServerStats(String server, int timeout)
-  throws IOException {
-    String[] sp = server.split(":");
-    if (sp == null || sp.length == 0) {
-      return null;
-    }
-
-    String host = sp[0];
-    int port = sp.length > 1 ? Integer.parseInt(sp[1])
-        : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
-
-    InetSocketAddress sockAddr = new InetSocketAddress(host, port);
-    try (Socket socket = new Socket()) {
-      socket.connect(sockAddr, timeout);
-
-      socket.setSoTimeout(timeout);
-      try (PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
-          BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
-        out.println("stat");
-        out.flush();
-        ArrayList<String> res = new ArrayList<>();
-        while (true) {
-          String line = in.readLine();
-          if (line != null) {
-            res.add(line);
-          } else {
-            break;
-          }
-        }
-        return res.toArray(new String[res.size()]);
-      }
-    }
-  }
-
-  private static void logRetrievedMsg(final ZooKeeperWatcher zkw,
-      final String znode, final byte [] data, final boolean watcherSet) {
-    if (!LOG.isTraceEnabled()) return;
-    LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) +
-      " byte(s) of data from znode " + znode +
-      (watcherSet? " and set watcher; ": "; data=") +
-      (data == null? "null": data.length == 0? "empty": (
-          znode.startsWith(zkw.znodePaths.metaZNodePrefix)?
-            getServerNameOrEmptyString(data):
-          znode.startsWith(zkw.znodePaths.backupMasterAddressesZNode)?
-            getServerNameOrEmptyString(data):
-          StringUtils.abbreviate(Bytes.toStringBinary(data), 32)))));
-  }
-
-  private static String getServerNameOrEmptyString(final byte [] data) {
-    try {
-      return ProtobufUtil.parseServerNameFrom(data).toString();
-    } catch (DeserializationException e) {
-      return "";
-    }
-  }
-
-  /**
-   * Waits for HBase installation's base (parent) znode to become available.
-   * @throws IOException on ZK errors
-   */
-  public static void waitForBaseZNode(Configuration conf) throws IOException {
-    LOG.info("Waiting until the base znode is available");
-    String parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
-        HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
-    ZooKeeper zk = new ZooKeeper(ZKConfig.getZKQuorumServersString(conf),
-        conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
-        HConstants.DEFAULT_ZK_SESSION_TIMEOUT), EmptyWatcher.instance);
-
-    final int maxTimeMs = 10000;
-    final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
-
-    KeeperException keeperEx = null;
-    try {
-      try {
-        for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
-          try {
-            if (zk.exists(parentZNode, false) != null) {
-              LOG.info("Parent znode exists: " + parentZNode);
-              keeperEx = null;
-              break;
-            }
-          } catch (KeeperException e) {
-            keeperEx = e;
-          }
-          Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
-        }
-      } finally {
-        zk.close();
-      }
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-    }
-
-    if (keeperEx != null) {
-      throw new IOException(keeperEx);
-    }
-  }
-
-  /**
-   * Convert a {@link DeserializationException} to a more palatable {@link KeeperException}.
-   * Used when can't let a {@link DeserializationException} out w/o changing public API.
-   * @param e Exception to convert
-   * @return Converted exception
-   */
-  public static KeeperException convert(final DeserializationException e) {
-    KeeperException ke = new KeeperException.DataInconsistencyException();
-    ke.initCause(e);
-    return ke;
-  }
-
-  /**
-   * Recursively print the current state of ZK (non-transactional)
-   * @param root name of the root directory in zk to print
-   */
-  public static void logZKTree(ZooKeeperWatcher zkw, String root) {
-    if (!LOG.isDebugEnabled()) return;
-    LOG.debug("Current zk system:");
-    String prefix = "|-";
-    LOG.debug(prefix + root);
-    try {
-      logZKTree(zkw, root, prefix);
-    } catch (KeeperException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Helper method to print the current state of the ZK tree.
-   * @see #logZKTree(ZooKeeperWatcher, String)
-   * @throws KeeperException if an unexpected exception occurs
-   */
-  protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix)
-      throws KeeperException {
-    List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
-    if (children == null) return;
-    for (String child : children) {
-      LOG.debug(prefix + child);
-      String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child);
-      logZKTree(zkw, node, prefix + "---");
-    }
-  }
-
-  /**
-   * @param position
-   * @return Serialized protobuf of <code>position</code> with pb magic prefix prepended suitable
-   *         for use as content of an wal position in a replication queue.
-   */
-  public static byte[] positionToByteArray(final long position) {
-    byte[] bytes = ReplicationProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
-        .build().toByteArray();
-    return ProtobufUtil.prependPBMagic(bytes);
-  }
-
-  /**
-   * @param bytes - Content of a WAL position znode.
-   * @return long - The current WAL position.
-   * @throws DeserializationException
-   */
-  public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException {
-    if (bytes == null) {
-      throw new DeserializationException("Unable to parse null WAL position.");
-    }
-    if (ProtobufUtil.isPBMagicPrefix(bytes)) {
-      int pblen = ProtobufUtil.lengthOfPBMagic();
-      ReplicationProtos.ReplicationHLogPosition.Builder builder =
-          ReplicationProtos.ReplicationHLogPosition.newBuilder();
-      ReplicationProtos.ReplicationHLogPosition position;
-      try {
-        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
-        position = builder.build();
-      } catch (IOException e) {
-        throw new DeserializationException(e);
-      }
-      return position.getPosition();
-    } else {
-      if (bytes.length > 0) {
-        return Bytes.toLong(bytes);
-      }
-      return 0;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java
deleted file mode 100644
index 045fd97..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java
+++ /dev/null
@@ -1,116 +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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-
-/**
- * You may add the jaas.conf option
- *    -Djava.security.auth.login.config=/PATH/jaas.conf
- *
- * You may also specify -D to set options
- *    "hbase.zookeeper.quorum"    (it should be in hbase-site.xml)
- *    "zookeeper.znode.parent"    (it should be in hbase-site.xml)
- *
- * Use -set-acls to set the ACLs, no option to erase ACLs
- */
-@InterfaceAudience.Private
-public class ZkAclReset extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(ZkAclReset.class);
-
-  private static void resetAcls(final ZooKeeperWatcher zkw, final String znode,
-      final boolean eraseAcls) throws Exception {
-    List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
-    if (children != null) {
-      for (String child: children) {
-        resetAcls(zkw, ZNodePaths.joinZNode(znode, child), eraseAcls);
-      }
-    }
-
-    ZooKeeper zk = zkw.getRecoverableZooKeeper().getZooKeeper();
-    if (eraseAcls) {
-      LOG.info(" - erase ACLs for " + znode);
-      zk.setACL(znode, ZooDefs.Ids.OPEN_ACL_UNSAFE, -1);
-    } else {
-      LOG.info(" - set ACLs for " + znode);
-      zk.setACL(znode, ZKUtil.createACL(zkw, znode, true), -1);
-    }
-  }
-
-  private static void resetAcls(final Configuration conf, boolean eraseAcls)
-      throws Exception {
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "ZkAclReset", null);
-    try {
-      LOG.info((eraseAcls ? "Erase" : "Set") + " HBase ACLs for " +
-                zkw.getQuorum() + " " + zkw.znodePaths.baseZNode);
-      resetAcls(zkw, zkw.znodePaths.baseZNode, eraseAcls);
-    } finally {
-      zkw.close();
-    }
-  }
-
-  private void printUsageAndExit() {
-    System.err.printf("Usage: hbase %s [options]%n", getClass().getName());
-    System.err.println(" where [options] are:");
-    System.err.println("  -h|-help                Show this help and exit.");
-    System.err.println("  -set-acls               Setup the hbase znode ACLs for a secure cluster");
-    System.err.println();
-    System.err.println("Examples:");
-    System.err.println("  To reset the ACLs to the unsecure cluster behavior:");
-    System.err.println("  hbase " + getClass().getName());
-    System.err.println();
-    System.err.println("  To reset the ACLs to the secure cluster behavior:");
-    System.err.println("  hbase " + getClass().getName() + " -set-acls");
-    System.exit(1);
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    boolean eraseAcls = true;
-
-    for (int i = 0; i < args.length; ++i) {
-      if (args[i].equals("-help")) {
-        printUsageAndExit();
-      } else if (args[i].equals("-set-acls")) {
-        eraseAcls = false;
-      } else {
-        printUsageAndExit();
-      }
-    }
-
-    resetAcls(getConf(), eraseAcls);
-    return(0);
-  }
-
-  public static void main(String[] args) throws Exception {
-    System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZkAclReset(), args));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java
deleted file mode 100644
index f78f1d8..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java
+++ /dev/null
@@ -1,86 +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.yetus.audience.InterfaceAudience;
-
-/**
- * Base class for internal listeners of ZooKeeper events.
- *
- * The {@link ZooKeeperWatcher} for a process will execute the appropriate
- * methods of implementations of this class.  In order to receive events from
- * the watcher, every listener must register itself via {@link ZooKeeperWatcher#registerListener}.
- *
- * Subclasses need only override those methods in which they are interested.
- *
- * Note that the watcher will be blocked when invoking methods in listeners so
- * they must not be long-running.
- */
-@InterfaceAudience.Private
-public abstract class ZooKeeperListener {
-
-  // Reference to the zk watcher which also contains configuration and constants
-  protected ZooKeeperWatcher watcher;
-
-  /**
-   * Construct a ZooKeeper event listener.
-   */
-  public ZooKeeperListener(ZooKeeperWatcher watcher) {
-    this.watcher = watcher;
-  }
-
-  /**
-   * Called when a new node has been created.
-   * @param path full path of the new node
-   */
-  public void nodeCreated(String path) {
-    // no-op
-  }
-
-  /**
-   * Called when a node has been deleted
-   * @param path full path of the deleted node
-   */
-  public void nodeDeleted(String path) {
-    // no-op
-  }
-
-  /**
-   * Called when an existing node has changed data.
-   * @param path full path of the updated node
-   */
-  public void nodeDataChanged(String path) {
-    // no-op
-  }
-
-  /**
-   * Called when an existing node has a child node added or removed.
-   * @param path full path of the node whose children have changed
-   */
-  public void nodeChildrenChanged(String path) {
-    // no-op
-  }
-
-  /**
-   * @return The watcher associated with this listener
-   */
-  public ZooKeeperWatcher getWatcher() {
-    return this.watcher;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/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 12ce2cf..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.yetus.audience.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);
-}


[02/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
new file mode 100644
index 0000000..d8472a9
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -0,0 +1,2072 @@
+/*
+ *
+ * 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.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.CreateRequest;
+import org.apache.zookeeper.proto.DeleteRequest;
+import org.apache.zookeeper.proto.SetDataRequest;
+import org.apache.zookeeper.server.ZooKeeperSaslServer;
+
+/**
+ * Internal HBase utility class for ZooKeeper.
+ *
+ * <p>Contains only static methods and constants.
+ *
+ * <p>Methods all throw {@link KeeperException} if there is an unexpected
+ * zookeeper exception, so callers of these methods must handle appropriately.
+ * If ZK is required for the operation, the server will need to be aborted.
+ */
+@InterfaceAudience.Private
+public class ZKUtil {
+  private static final Log LOG = LogFactory.getLog(ZKUtil.class);
+
+  private static int zkDumpConnectionTimeOut;
+
+  /**
+   * Creates a new connection to ZooKeeper, pulling settings and ensemble config
+   * from the specified configuration object using methods from {@link ZKConfig}.
+   *
+   * Sets the connection status monitoring watcher to the specified watcher.
+   *
+   * @param conf configuration to pull ensemble and other settings from
+   * @param watcher watcher to monitor connection changes
+   * @return connection to zookeeper
+   * @throws IOException if unable to connect to zk or config problem
+   */
+  public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
+  throws IOException {
+    String ensemble = ZKConfig.getZKQuorumServersString(conf);
+    return connect(conf, ensemble, watcher);
+  }
+
+  public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
+      Watcher watcher)
+  throws IOException {
+    return connect(conf, ensemble, watcher, null);
+  }
+
+  public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
+      Watcher watcher, final String identifier)
+  throws IOException {
+    if(ensemble == null) {
+      throw new IOException("Unable to determine ZooKeeper ensemble");
+    }
+    int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
+        HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble);
+    }
+    int retry = conf.getInt("zookeeper.recovery.retry", 3);
+    int retryIntervalMillis =
+      conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
+    int maxSleepTime = conf.getInt("zookeeper.recovery.retry.maxsleeptime", 60000);
+    zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
+        1000);
+    return new RecoverableZooKeeper(ensemble, timeout, watcher,
+        retry, retryIntervalMillis, maxSleepTime, identifier);
+  }
+
+  /**
+   * Log in the current zookeeper server process using the given configuration
+   * keys for the credential file and login principal.
+   *
+   * <p><strong>This is only applicable when running on secure hbase</strong>
+   * On regular HBase (without security features), this will safely be ignored.
+   * </p>
+   *
+   * @param conf The configuration data to use
+   * @param keytabFileKey Property key used to configure the path to the credential file
+   * @param userNameKey Property key used to configure the login principal
+   * @param hostname Current hostname to use in any credentials
+   * @throws IOException underlying exception from SecurityUtil.login() call
+   */
+  public static void loginServer(Configuration conf, String keytabFileKey,
+      String userNameKey, String hostname) throws IOException {
+    login(conf, keytabFileKey, userNameKey, hostname,
+          ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY,
+          JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME);
+  }
+
+  /**
+   * Log in the current zookeeper client using the given configuration
+   * keys for the credential file and login principal.
+   *
+   * <p><strong>This is only applicable when running on secure hbase</strong>
+   * On regular HBase (without security features), this will safely be ignored.
+   * </p>
+   *
+   * @param conf The configuration data to use
+   * @param keytabFileKey Property key used to configure the path to the credential file
+   * @param userNameKey Property key used to configure the login principal
+   * @param hostname Current hostname to use in any credentials
+   * @throws IOException underlying exception from SecurityUtil.login() call
+   */
+  public static void loginClient(Configuration conf, String keytabFileKey,
+      String userNameKey, String hostname) throws IOException {
+    login(conf, keytabFileKey, userNameKey, hostname,
+          ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
+          JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME);
+  }
+
+  /**
+   * Log in the current process using the given configuration keys for the
+   * credential file and login principal.
+   *
+   * <p><strong>This is only applicable when running on secure hbase</strong>
+   * On regular HBase (without security features), this will safely be ignored.
+   * </p>
+   *
+   * @param conf The configuration data to use
+   * @param keytabFileKey Property key used to configure the path to the credential file
+   * @param userNameKey Property key used to configure the login principal
+   * @param hostname Current hostname to use in any credentials
+   * @param loginContextProperty property name to expose the entry name
+   * @param loginContextName jaas entry name
+   * @throws IOException underlying exception from SecurityUtil.login() call
+   */
+  private static void login(Configuration conf, String keytabFileKey,
+      String userNameKey, String hostname,
+      String loginContextProperty, String loginContextName)
+      throws IOException {
+    if (!isSecureZooKeeper(conf))
+      return;
+
+    // User has specified a jaas.conf, keep this one as the good one.
+    // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf"
+    if (System.getProperty("java.security.auth.login.config") != null)
+      return;
+
+    // No keytab specified, no auth
+    String keytabFilename = conf.get(keytabFileKey);
+    if (keytabFilename == null) {
+      LOG.warn("no keytab specified for: " + keytabFileKey);
+      return;
+    }
+
+    String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
+    String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
+
+    // Initialize the "jaas.conf" for keyTab/principal,
+    // If keyTab is not specified use the Ticket Cache.
+    // and set the zookeeper login context name.
+    JaasConfiguration jaasConf = new JaasConfiguration(loginContextName,
+        principalName, keytabFilename);
+    javax.security.auth.login.Configuration.setConfiguration(jaasConf);
+    System.setProperty(loginContextProperty, loginContextName);
+  }
+
+  /**
+   * A JAAS configuration that defines the login modules that we want to use for login.
+   */
+  private static class JaasConfiguration extends javax.security.auth.login.Configuration {
+    private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME =
+      "zookeeper-server-keytab-kerberos";
+    private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
+      "zookeeper-client-keytab-kerberos";
+
+    private static final Map<String, String> BASIC_JAAS_OPTIONS = new HashMap<>();
+    static {
+      String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
+      if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
+        BASIC_JAAS_OPTIONS.put("debug", "true");
+      }
+    }
+
+    private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS = new HashMap<>();
+    static {
+      KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
+      KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
+      KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
+      KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
+    }
+
+    private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
+      new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
+                                LoginModuleControlFlag.REQUIRED,
+                                KEYTAB_KERBEROS_OPTIONS);
+
+    private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
+      new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN};
+
+    private javax.security.auth.login.Configuration baseConfig;
+    private final String loginContextName;
+    private final boolean useTicketCache;
+    private final String keytabFile;
+    private final String principal;
+
+    public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
+      this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
+    }
+
+    private JaasConfiguration(String loginContextName, String principal,
+                             String keytabFile, boolean useTicketCache) {
+      try {
+        this.baseConfig = javax.security.auth.login.Configuration.getConfiguration();
+      } catch (SecurityException e) {
+        this.baseConfig = null;
+      }
+      this.loginContextName = loginContextName;
+      this.useTicketCache = useTicketCache;
+      this.keytabFile = keytabFile;
+      this.principal = principal;
+      LOG.info("JaasConfiguration loginContextName=" + loginContextName +
+               " principal=" + principal + " useTicketCache=" + useTicketCache +
+               " keytabFile=" + keytabFile);
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
+      if (loginContextName.equals(appName)) {
+        if (!useTicketCache) {
+          KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
+          KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
+        }
+        KEYTAB_KERBEROS_OPTIONS.put("principal", principal);
+        KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
+        return KEYTAB_KERBEROS_CONF;
+      }
+      if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
+      return(null);
+    }
+  }
+
+  //
+  // Helper methods
+  //
+  /**
+   * Returns the full path of the immediate parent of the specified node.
+   * @param node path to get parent of
+   * @return parent of path, null if passed the root node or an invalid node
+   */
+  public static String getParent(String node) {
+    int idx = node.lastIndexOf(ZNodePaths.ZNODE_PATH_SEPARATOR);
+    return idx <= 0 ? null : node.substring(0, idx);
+  }
+
+  /**
+   * Get the name of the current node from the specified fully-qualified path.
+   * @param path fully-qualified path
+   * @return name of the current node
+   */
+  public static String getNodeName(String path) {
+    return path.substring(path.lastIndexOf("/")+1);
+  }
+
+  //
+  // Existence checks and watches
+  //
+
+  /**
+   * Watch the specified znode for delete/create/change events.  The watcher is
+   * set whether or not the node exists.  If the node already exists, the method
+   * returns true.  If the node does not exist, the method returns false.
+   *
+   * @param zkw zk reference
+   * @param znode path of node to watch
+   * @return true if znode exists, false if does not exist or error
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static boolean watchAndCheckExists(ZKWatcher zkw, String znode)
+  throws KeeperException {
+    try {
+      Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw);
+      boolean exists = s != null ? true : false;
+      if (exists) {
+        LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode));
+      } else {
+        LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode));
+      }
+      return exists;
+    } catch (KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
+      zkw.keeperException(e);
+      return false;
+    } catch (InterruptedException e) {
+      LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
+      zkw.interruptedException(e);
+      return false;
+    }
+  }
+
+  /**
+   * Watch the specified znode, but only if exists. Useful when watching
+   * for deletions. Uses .getData() (and handles NoNodeException) instead
+   * of .exists() to accomplish this, as .getData() will only set a watch if
+   * the znode exists.
+   * @param zkw zk reference
+   * @param znode path of node to watch
+   * @return true if the watch is set, false if node does not exists
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static boolean setWatchIfNodeExists(ZKWatcher zkw, String znode)
+      throws KeeperException {
+    try {
+      zkw.getRecoverableZooKeeper().getData(znode, true, null);
+      return true;
+    } catch (NoNodeException e) {
+      return false;
+    } catch (InterruptedException e) {
+      LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
+      zkw.interruptedException(e);
+      return false;
+    }
+  }
+
+  /**
+   * Check if the specified node exists.  Sets no watches.
+   *
+   * @param zkw zk reference
+   * @param znode path of node to watch
+   * @return version of the node if it exists, -1 if does not exist
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static int checkExists(ZKWatcher zkw, String znode)
+  throws KeeperException {
+    try {
+      Stat s = zkw.getRecoverableZooKeeper().exists(znode, null);
+      return s != null ? s.getVersion() : -1;
+    } catch (KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
+      zkw.keeperException(e);
+      return -1;
+    } catch (InterruptedException e) {
+      LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
+      zkw.interruptedException(e);
+      return -1;
+    }
+  }
+
+  //
+  // Znode listings
+  //
+
+  /**
+   * Lists the children znodes of the specified znode.  Also sets a watch on
+   * the specified znode which will capture a NodeDeleted event on the specified
+   * znode as well as NodeChildrenChanged if any children of the specified znode
+   * are created or deleted.
+   *
+   * Returns null if the specified node does not exist.  Otherwise returns a
+   * list of children of the specified node.  If the node exists but it has no
+   * children, an empty list will be returned.
+   *
+   * @param zkw zk reference
+   * @param znode path of node to list and watch children of
+   * @return list of children of the specified node, an empty list if the node
+   *          exists but has no children, and null if the node does not exist
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static List<String> listChildrenAndWatchForNewChildren(
+          ZKWatcher zkw, String znode)
+  throws KeeperException {
+    try {
+      List<String> children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw);
+      return children;
+    } catch(KeeperException.NoNodeException ke) {
+      LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
+          "because node does not exist (not an error)"));
+      return null;
+    } catch (KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
+      zkw.keeperException(e);
+      return null;
+    } catch (InterruptedException e) {
+      LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
+      zkw.interruptedException(e);
+      return null;
+    }
+  }
+
+  /**
+   * List all the children of the specified znode, setting a watch for children
+   * changes and also setting a watch on every individual child in order to get
+   * the NodeCreated and NodeDeleted events.
+   * @param zkw zookeeper reference
+   * @param znode node to get children of and watch
+   * @return list of znode names, null if the node doesn't exist
+   * @throws KeeperException
+   */
+  public static List<String> listChildrenAndWatchThem(ZKWatcher zkw,
+      String znode) throws KeeperException {
+    List<String> children = listChildrenAndWatchForNewChildren(zkw, znode);
+    if (children == null) {
+      return null;
+    }
+    for (String child : children) {
+      watchAndCheckExists(zkw, ZNodePaths.joinZNode(znode, child));
+    }
+    return children;
+  }
+
+  /**
+   * Lists the children of the specified znode without setting any watches.
+   *
+   * Sets no watches at all, this method is best effort.
+   *
+   * Returns an empty list if the node has no children.  Returns null if the
+   * parent node itself does not exist.
+   *
+   * @param zkw zookeeper reference
+   * @param znode node to get children
+   * @return list of data of children of specified znode, empty if no children,
+   *         null if parent does not exist
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static List<String> listChildrenNoWatch(ZKWatcher zkw, String znode)
+  throws KeeperException {
+    List<String> children = null;
+    try {
+      // List the children without watching
+      children = zkw.getRecoverableZooKeeper().getChildren(znode, null);
+    } catch(KeeperException.NoNodeException nne) {
+      return null;
+    } catch(InterruptedException ie) {
+      zkw.interruptedException(ie);
+    }
+    return children;
+  }
+
+  /**
+   * Simple class to hold a node path and node data.
+   * @deprecated Unused
+   */
+  @Deprecated
+  public static class NodeAndData {
+    private String node;
+    private byte [] data;
+    public NodeAndData(String node, byte [] data) {
+      this.node = node;
+      this.data = data;
+    }
+    public String getNode() {
+      return node;
+    }
+    public byte [] getData() {
+      return data;
+    }
+    @Override
+    public String toString() {
+      return node;
+    }
+    public boolean isEmpty() {
+      return (data == null || data.length == 0);
+    }
+  }
+
+  /**
+   * Checks if the specified znode has any children.  Sets no watches.
+   *
+   * Returns true if the node exists and has children.  Returns false if the
+   * node does not exist or if the node does not have any children.
+   *
+   * Used during master initialization to determine if the master is a
+   * failed-over-to master or the first master during initial cluster startup.
+   * If the directory for regionserver ephemeral nodes is empty then this is
+   * a cluster startup, if not then it is not cluster startup.
+   *
+   * @param zkw zk reference
+   * @param znode path of node to check for children of
+   * @return true if node has children, false if not or node does not exist
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static boolean nodeHasChildren(ZKWatcher zkw, String znode)
+  throws KeeperException {
+    try {
+      return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty();
+    } catch(KeeperException.NoNodeException ke) {
+      LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
+      "because node does not exist (not an error)"));
+      return false;
+    } catch (KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
+      zkw.keeperException(e);
+      return false;
+    } catch (InterruptedException e) {
+      LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
+      zkw.interruptedException(e);
+      return false;
+    }
+  }
+
+  /**
+   * Get the number of children of the specified node.
+   *
+   * If the node does not exist or has no children, returns 0.
+   *
+   * Sets no watches at all.
+   *
+   * @param zkw zk reference
+   * @param znode path of node to count children of
+   * @return number of children of specified node, 0 if none or parent does not
+   *         exist
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static int getNumberOfChildren(ZKWatcher zkw, String znode)
+  throws KeeperException {
+    try {
+      Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null);
+      return stat == null ? 0 : stat.getNumChildren();
+    } catch(KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to get children of node " + znode));
+      zkw.keeperException(e);
+    } catch(InterruptedException e) {
+      zkw.interruptedException(e);
+    }
+    return 0;
+  }
+
+  //
+  // Data retrieval
+  //
+
+  /**
+   * Get znode data. Does not set a watcher.
+   * @return ZNode data, null if the node does not exist or if there is an
+   *  error.
+   */
+  public static byte [] getData(ZKWatcher zkw, String znode)
+      throws KeeperException, InterruptedException {
+    try {
+      byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null);
+      logRetrievedMsg(zkw, znode, data, false);
+      return data;
+    } catch (KeeperException.NoNodeException e) {
+      LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
+          "because node does not exist (not an error)"));
+      return null;
+    } catch (KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+      zkw.keeperException(e);
+      return null;
+    }
+  }
+
+  /**
+   * Get the data at the specified znode and set a watch.
+   *
+   * Returns the data and sets a watch if the node exists.  Returns null and no
+   * watch is set if the node does not exist or there is an exception.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @return data of the specified znode, or null
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static byte [] getDataAndWatch(ZKWatcher zkw, String znode)
+  throws KeeperException {
+    return getDataInternal(zkw, znode, null, true);
+  }
+
+  /**
+   * Get the data at the specified znode and set a watch.
+   *
+   * Returns the data and sets a watch if the node exists.  Returns null and no
+   * watch is set if the node does not exist or there is an exception.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param stat object to populate the version of the znode
+   * @return data of the specified znode, or null
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static byte[] getDataAndWatch(ZKWatcher zkw, String znode,
+                                       Stat stat) throws KeeperException {
+    return getDataInternal(zkw, znode, stat, true);
+  }
+
+  private static byte[] getDataInternal(ZKWatcher zkw, String znode, Stat stat,
+                                        boolean watcherSet)
+      throws KeeperException {
+    try {
+      byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
+      logRetrievedMsg(zkw, znode, data, watcherSet);
+      return data;
+    } catch (KeeperException.NoNodeException e) {
+      // This log can get pretty annoying when we cycle on 100ms waits.
+      // Enable trace if you really want to see it.
+      LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " +
+        "because node does not exist (not an error)"));
+      return null;
+    } catch (KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+      zkw.keeperException(e);
+      return null;
+    } catch (InterruptedException e) {
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+      zkw.interruptedException(e);
+      return null;
+    }
+  }
+
+  /**
+   * Get the data at the specified znode without setting a watch.
+   *
+   * Returns the data if the node exists.  Returns null if the node does not
+   * exist.
+   *
+   * Sets the stats of the node in the passed Stat object.  Pass a null stat if
+   * not interested.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param stat node status to get if node exists
+   * @return data of the specified znode, or null if node does not exist
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static byte [] getDataNoWatch(ZKWatcher zkw, String znode,
+                                       Stat stat)
+  throws KeeperException {
+    try {
+      byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat);
+      logRetrievedMsg(zkw, znode, data, false);
+      return data;
+    } catch (KeeperException.NoNodeException e) {
+      LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
+          "because node does not exist (not necessarily an error)"));
+      return null;
+    } catch (KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+      zkw.keeperException(e);
+      return null;
+    } catch (InterruptedException e) {
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
+      zkw.interruptedException(e);
+      return null;
+    }
+  }
+
+  /**
+   * Returns the date of child znodes of the specified znode.  Also sets a watch on
+   * the specified znode which will capture a NodeDeleted event on the specified
+   * znode as well as NodeChildrenChanged if any children of the specified znode
+   * are created or deleted.
+   *
+   * Returns null if the specified node does not exist.  Otherwise returns a
+   * list of children of the specified node.  If the node exists but it has no
+   * children, an empty list will be returned.
+   *
+   * @param zkw zk reference
+   * @param baseNode path of node to list and watch children of
+   * @return list of data of children of the specified node, an empty list if the node
+   *          exists but has no children, and null if the node does not exist
+   * @throws KeeperException if unexpected zookeeper exception
+   * @deprecated Unused
+   */
+  @Deprecated
+  public static List<NodeAndData> getChildDataAndWatchForNewChildren(
+          ZKWatcher zkw, String baseNode) throws KeeperException {
+    List<String> nodes =
+      ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
+    if (nodes != null) {
+      List<NodeAndData> newNodes = new ArrayList<>();
+      for (String node : nodes) {
+        String nodePath = ZNodePaths.joinZNode(baseNode, node);
+        byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
+        newNodes.add(new NodeAndData(nodePath, data));
+      }
+      return newNodes;
+    }
+    return null;
+  }
+
+  /**
+   * Update the data of an existing node with the expected version to have the
+   * specified data.
+   *
+   * Throws an exception if there is a version mismatch or some other problem.
+   *
+   * Sets no watches under any conditions.
+   *
+   * @param zkw zk reference
+   * @param znode
+   * @param data
+   * @param expectedVersion
+   * @throws KeeperException if unexpected zookeeper exception
+   * @throws KeeperException.BadVersionException if version mismatch
+   * @deprecated Unused
+   */
+  @Deprecated
+  public static void updateExistingNodeData(ZKWatcher zkw, String znode,
+                                            byte [] data, int expectedVersion)
+  throws KeeperException {
+    try {
+      zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion);
+    } catch(InterruptedException ie) {
+      zkw.interruptedException(ie);
+    }
+  }
+
+  //
+  // Data setting
+  //
+
+  /**
+   * Sets the data of the existing znode to be the specified data.  Ensures that
+   * the current data has the specified expected version.
+   *
+   * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
+   *
+   * <p>If their is a version mismatch, method returns null.
+   *
+   * <p>No watches are set but setting data will trigger other watchers of this
+   * node.
+   *
+   * <p>If there is another problem, a KeeperException will be thrown.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param data data to set for node
+   * @param expectedVersion version expected when setting data
+   * @return true if data set, false if version mismatch
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static boolean setData(ZKWatcher zkw, String znode,
+                                byte [] data, int expectedVersion)
+  throws KeeperException, KeeperException.NoNodeException {
+    try {
+      return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null;
+    } catch (InterruptedException e) {
+      zkw.interruptedException(e);
+      return false;
+    }
+  }
+
+  /**
+   * Set data into node creating node if it doesn't yet exist.
+   * Does not set watch.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param data data to set for node
+   * @throws KeeperException
+   */
+  public static void createSetData(final ZKWatcher zkw, final String znode,
+                                   final byte [] data)
+  throws KeeperException {
+    if (checkExists(zkw, znode) == -1) {
+      ZKUtil.createWithParents(zkw, znode, data);
+    } else {
+      ZKUtil.setData(zkw, znode, data);
+    }
+  }
+
+  /**
+   * Sets the data of the existing znode to be the specified data.  The node
+   * must exist but no checks are done on the existing data or version.
+   *
+   * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
+   *
+   * <p>No watches are set but setting data will trigger other watchers of this
+   * node.
+   *
+   * <p>If there is another problem, a KeeperException will be thrown.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param data data to set for node
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static void setData(ZKWatcher zkw, String znode, byte [] data)
+  throws KeeperException, KeeperException.NoNodeException {
+    setData(zkw, (SetData)ZKUtilOp.setData(znode, data));
+  }
+
+  private static void setData(ZKWatcher zkw, SetData setData)
+  throws KeeperException, KeeperException.NoNodeException {
+    SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord();
+    setData(zkw, sd.getPath(), sd.getData(), sd.getVersion());
+  }
+
+  /**
+   * Returns whether or not secure authentication is enabled
+   * (whether <code>hbase.security.authentication</code> is set to
+   * <code>kerberos</code>.
+   */
+  public static boolean isSecureZooKeeper(Configuration conf) {
+    // Detection for embedded HBase client with jaas configuration
+    // defined for third party programs.
+    try {
+      javax.security.auth.login.Configuration testConfig =
+          javax.security.auth.login.Configuration.getConfiguration();
+      if (testConfig.getAppConfigurationEntry("Client") == null
+          && testConfig.getAppConfigurationEntry(
+            JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME) == null
+          && testConfig.getAppConfigurationEntry(
+              JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME) == null
+          && conf.get(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL) == null
+          && conf.get(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL) == null) {
+
+        return false;
+      }
+    } catch(Exception e) {
+      // No Jaas configuration defined.
+      return false;
+    }
+
+    // Master & RSs uses hbase.zookeeper.client.*
+    return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"));
+  }
+
+  private static ArrayList<ACL> createACL(ZKWatcher zkw, String node) {
+    return createACL(zkw, node, isSecureZooKeeper(zkw.getConfiguration()));
+  }
+
+  public static ArrayList<ACL> createACL(ZKWatcher zkw, String node,
+                                         boolean isSecureZooKeeper) {
+    if (!node.startsWith(zkw.znodePaths.baseZNode)) {
+      return Ids.OPEN_ACL_UNSAFE;
+    }
+    if (isSecureZooKeeper) {
+      ArrayList<ACL> acls = new ArrayList<>();
+      // add permission to hbase supper user
+      String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY);
+      String hbaseUser = null;
+      try {
+        hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
+      } catch (IOException e) {
+        LOG.debug("Could not acquire current User.", e);
+      }
+      if (superUsers != null) {
+        List<String> groups = new ArrayList<>();
+        for (String user : superUsers) {
+          if (AuthUtil.isGroupPrincipal(user)) {
+            // TODO: Set node ACL for groups when ZK supports this feature
+            groups.add(user);
+          } else {
+            if(!user.equals(hbaseUser)) {
+              acls.add(new ACL(Perms.ALL, new Id("sasl", user)));
+            }
+          }
+        }
+        if (!groups.isEmpty()) {
+          LOG.warn("Znode ACL setting for group " + groups
+              + " is skipped, ZooKeeper doesn't support this feature presently.");
+        }
+      }
+      // Certain znodes are accessed directly by the client,
+      // so they must be readable by non-authenticated clients
+      if (zkw.isClientReadable(node)) {
+        acls.addAll(Ids.CREATOR_ALL_ACL);
+        acls.addAll(Ids.READ_ACL_UNSAFE);
+      } else {
+        acls.addAll(Ids.CREATOR_ALL_ACL);
+      }
+      return acls;
+    } else {
+      return Ids.OPEN_ACL_UNSAFE;
+    }
+  }
+
+  //
+  // Node creation
+  //
+
+  /**
+   *
+   * Set the specified znode to be an ephemeral node carrying the specified
+   * data.
+   *
+   * If the node is created successfully, a watcher is also set on the node.
+   *
+   * If the node is not created successfully because it already exists, this
+   * method will also set a watcher on the node.
+   *
+   * If there is another problem, a KeeperException will be thrown.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param data data of node
+   * @return true if node created, false if not, watch set in both cases
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static boolean createEphemeralNodeAndWatch(ZKWatcher zkw,
+      String znode, byte [] data)
+  throws KeeperException {
+    boolean ret = true;
+    try {
+      zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
+          CreateMode.EPHEMERAL);
+    } catch (KeeperException.NodeExistsException nee) {
+      ret = false;
+    } catch (InterruptedException e) {
+      LOG.info("Interrupted", e);
+      Thread.currentThread().interrupt();
+    }
+    if(!watchAndCheckExists(zkw, znode)) {
+      // It did exist but now it doesn't, try again
+      return createEphemeralNodeAndWatch(zkw, znode, data);
+    }
+    return ret;
+  }
+
+  /**
+   * Creates the specified znode to be a persistent node carrying the specified
+   * data.
+   *
+   * Returns true if the node was successfully created, false if the node
+   * already existed.
+   *
+   * If the node is created successfully, a watcher is also set on the node.
+   *
+   * If the node is not created successfully because it already exists, this
+   * method will also set a watcher on the node but return false.
+   *
+   * If there is another problem, a KeeperException will be thrown.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param data data of node
+   * @return true if node created, false if not, watch set in both cases
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static boolean createNodeIfNotExistsAndWatch(
+          ZKWatcher zkw, String znode, byte [] data)
+  throws KeeperException {
+    boolean ret = true;
+    try {
+      zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
+          CreateMode.PERSISTENT);
+    } catch (KeeperException.NodeExistsException nee) {
+      ret = false;
+    } catch (InterruptedException e) {
+      zkw.interruptedException(e);
+      return false;
+    }
+    try {
+      zkw.getRecoverableZooKeeper().exists(znode, zkw);
+    } catch (InterruptedException e) {
+      zkw.interruptedException(e);
+      return false;
+    }
+    return ret;
+  }
+
+  /**
+   * Creates the specified znode with the specified data but does not watch it.
+   *
+   * Returns the znode of the newly created node
+   *
+   * If there is another problem, a KeeperException will be thrown.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param data data of node
+   * @param createMode specifying whether the node to be created is ephemeral and/or sequential
+   * @return true name of the newly created znode or null
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static String createNodeIfNotExistsNoWatch(ZKWatcher zkw, String znode,
+                                                    byte[] data, CreateMode createMode) throws KeeperException {
+
+    String createdZNode = null;
+    try {
+      createdZNode = zkw.getRecoverableZooKeeper().create(znode, data,
+          createACL(zkw, znode), createMode);
+    } catch (KeeperException.NodeExistsException nee) {
+      return znode;
+    } catch (InterruptedException e) {
+      zkw.interruptedException(e);
+      return null;
+    }
+    return createdZNode;
+  }
+
+  /**
+   * Creates the specified node with the specified data and watches it.
+   *
+   * <p>Throws an exception if the node already exists.
+   *
+   * <p>The node created is persistent and open access.
+   *
+   * <p>Returns the version number of the created node if successful.
+   *
+   * @param zkw zk reference
+   * @param znode path of node to create
+   * @param data data of node to create
+   * @return version of node created
+   * @throws KeeperException if unexpected zookeeper exception
+   * @throws KeeperException.NodeExistsException if node already exists
+   */
+  public static int createAndWatch(ZKWatcher zkw,
+      String znode, byte [] data)
+  throws KeeperException, KeeperException.NodeExistsException {
+    try {
+      zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
+          CreateMode.PERSISTENT);
+      Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw);
+      if (stat == null){
+        // Likely a race condition. Someone deleted the znode.
+        throw KeeperException.create(KeeperException.Code.SYSTEMERROR,
+            "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode);
+      }
+     return stat.getVersion();
+    } catch (InterruptedException e) {
+      zkw.interruptedException(e);
+      return -1;
+    }
+  }
+
+  /**
+   * Async creates the specified node with the specified data.
+   *
+   * <p>Throws an exception if the node already exists.
+   *
+   * <p>The node created is persistent and open access.
+   *
+   * @param zkw zk reference
+   * @param znode path of node to create
+   * @param data data of node to create
+   * @param cb
+   * @param ctx
+   */
+  public static void asyncCreate(ZKWatcher zkw,
+      String znode, byte [] data, final AsyncCallback.StringCallback cb,
+      final Object ctx) {
+    zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data,
+        createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx);
+  }
+
+  /**
+   * Creates the specified node, iff the node does not exist.  Does not set a
+   * watch and fails silently if the node already exists.
+   *
+   * The node created is persistent and open access.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static void createAndFailSilent(ZKWatcher zkw,
+      String znode) throws KeeperException {
+    createAndFailSilent(zkw, znode, new byte[0]);
+  }
+
+  /**
+   * Creates the specified node containing specified data, iff the node does not exist.  Does
+   * not set a watch and fails silently if the node already exists.
+   *
+   * The node created is persistent and open access.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @param data a byte array data to store in the znode
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static void createAndFailSilent(ZKWatcher zkw,
+      String znode, byte[] data)
+  throws KeeperException {
+    createAndFailSilent(zkw,
+        (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data));
+  }
+
+  private static void createAndFailSilent(ZKWatcher zkw, CreateAndFailSilent cafs)
+  throws KeeperException {
+    CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord();
+    String znode = create.getPath();
+    try {
+      RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
+      if (zk.exists(znode, false) == null) {
+        zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags()));
+      }
+    } catch(KeeperException.NodeExistsException nee) {
+    } catch(KeeperException.NoAuthException nee){
+      try {
+        if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) {
+          // If we failed to create the file and it does not already exist.
+          throw(nee);
+        }
+      } catch (InterruptedException ie) {
+        zkw.interruptedException(ie);
+      }
+    } catch(InterruptedException ie) {
+      zkw.interruptedException(ie);
+    }
+  }
+
+  /**
+   * Creates the specified node and all parent nodes required for it to exist.
+   *
+   * No watches are set and no errors are thrown if the node already exists.
+   *
+   * The nodes created are persistent and open access.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static void createWithParents(ZKWatcher zkw, String znode)
+  throws KeeperException {
+    createWithParents(zkw, znode, new byte[0]);
+  }
+
+  /**
+   * Creates the specified node and all parent nodes required for it to exist.  The creation of
+   * parent znodes is not atomic with the leafe znode creation but the data is written atomically
+   * when the leaf node is created.
+   *
+   * No watches are set and no errors are thrown if the node already exists.
+   *
+   * The nodes created are persistent and open access.
+   *
+   * @param zkw zk reference
+   * @param znode path of node
+   * @throws KeeperException if unexpected zookeeper exception
+   */
+  public static void createWithParents(ZKWatcher zkw, String znode, byte[] data)
+  throws KeeperException {
+    try {
+      if(znode == null) {
+        return;
+      }
+      zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
+          CreateMode.PERSISTENT);
+    } catch(KeeperException.NodeExistsException nee) {
+      return;
+    } catch(KeeperException.NoNodeException nne) {
+      createWithParents(zkw, getParent(znode));
+      createWithParents(zkw, znode, data);
+    } catch(InterruptedException ie) {
+      zkw.interruptedException(ie);
+    }
+  }
+
+  //
+  // Deletes
+  //
+
+  /**
+   * Delete the specified node.  Sets no watches.  Throws all exceptions.
+   */
+  public static void deleteNode(ZKWatcher zkw, String node)
+  throws KeeperException {
+    deleteNode(zkw, node, -1);
+  }
+
+  /**
+   * Delete the specified node with the specified version.  Sets no watches.
+   * Throws all exceptions.
+   */
+  public static boolean deleteNode(ZKWatcher zkw, String node,
+                                   int version)
+  throws KeeperException {
+    try {
+      zkw.getRecoverableZooKeeper().delete(node, version);
+      return true;
+    } catch(KeeperException.BadVersionException bve) {
+      return false;
+    } catch(InterruptedException ie) {
+      zkw.interruptedException(ie);
+      return false;
+    }
+  }
+
+  /**
+   * Deletes the specified node.  Fails silent if the node does not exist.
+   * @param zkw
+   * @param node
+   * @throws KeeperException
+   */
+  public static void deleteNodeFailSilent(ZKWatcher zkw, String node)
+  throws KeeperException {
+    deleteNodeFailSilent(zkw,
+      (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node));
+  }
+
+  private static void deleteNodeFailSilent(ZKWatcher zkw,
+      DeleteNodeFailSilent dnfs) throws KeeperException {
+    DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord();
+    try {
+      zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion());
+    } catch(KeeperException.NoNodeException nne) {
+    } catch(InterruptedException ie) {
+      zkw.interruptedException(ie);
+    }
+  }
+
+
+  /**
+   * Delete the specified node and all of it's children.
+   * <p>
+   * If the node does not exist, just returns.
+   * <p>
+   * Sets no watches. Throws all exceptions besides dealing with deletion of
+   * children.
+   */
+  public static void deleteNodeRecursively(ZKWatcher zkw, String node)
+  throws KeeperException {
+    deleteNodeRecursivelyMultiOrSequential(zkw, true, node);
+  }
+
+  /**
+   * Delete all the children of the specified node but not the node itself.
+   *
+   * Sets no watches.  Throws all exceptions besides dealing with deletion of
+   * children.
+   *
+   * @throws KeeperException
+   */
+  public static void deleteChildrenRecursively(ZKWatcher zkw, String node)
+      throws KeeperException {
+    deleteChildrenRecursivelyMultiOrSequential(zkw, true, node);
+  }
+
+  /**
+   * Delete all the children of the specified node but not the node itself. This
+   * will first traverse the znode tree for listing the children and then delete
+   * these znodes using multi-update api or sequential based on the specified
+   * configurations.
+   * <p>
+   * Sets no watches. Throws all exceptions besides dealing with deletion of
+   * children.
+   * <p>
+   * If the following is true:
+   * <ul>
+   * <li>runSequentialOnMultiFailure is true
+   * </ul>
+   * on calling multi, we get a ZooKeeper exception that can be handled by a
+   * sequential call(*), we retry the operations one-by-one (sequentially).
+   *
+   * @param zkw
+   *          - zk reference
+   * @param runSequentialOnMultiFailure
+   *          - if true when we get a ZooKeeper exception that could retry the
+   *          operations one-by-one (sequentially)
+   * @param pathRoots
+   *          - path of the parent node(s)
+   * @throws KeeperException.NotEmptyException
+   *           if node has children while deleting
+   * @throws KeeperException
+   *           if unexpected ZooKeeper exception
+   * @throws IllegalArgumentException
+   *           if an invalid path is specified
+   */
+  public static void deleteChildrenRecursivelyMultiOrSequential(
+          ZKWatcher zkw, boolean runSequentialOnMultiFailure,
+          String... pathRoots) throws KeeperException {
+    if (pathRoots == null || pathRoots.length <= 0) {
+      LOG.warn("Given path is not valid!");
+      return;
+    }
+    List<ZKUtilOp> ops = new ArrayList<>();
+    for (String eachRoot : pathRoots) {
+      List<String> children = listChildrenBFSNoWatch(zkw, eachRoot);
+      // Delete the leaves first and eventually get rid of the root
+      for (int i = children.size() - 1; i >= 0; --i) {
+        ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
+      }
+    }
+    // atleast one element should exist
+    if (ops.size() > 0) {
+      multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
+    }
+  }
+
+  /**
+   * Delete the specified node and its children. This traverse the
+   * znode tree for listing the children and then delete
+   * these znodes including the parent using multi-update api or
+   * sequential based on the specified configurations.
+   * <p>
+   * Sets no watches. Throws all exceptions besides dealing with deletion of
+   * children.
+   * <p>
+   * If the following is true:
+   * <ul>
+   * <li>runSequentialOnMultiFailure is true
+   * </ul>
+   * on calling multi, we get a ZooKeeper exception that can be handled by a
+   * sequential call(*), we retry the operations one-by-one (sequentially).
+   *
+   * @param zkw
+   *          - zk reference
+   * @param runSequentialOnMultiFailure
+   *          - if true when we get a ZooKeeper exception that could retry the
+   *          operations one-by-one (sequentially)
+   * @param pathRoots
+   *          - path of the parent node(s)
+   * @throws KeeperException.NotEmptyException
+   *           if node has children while deleting
+   * @throws KeeperException
+   *           if unexpected ZooKeeper exception
+   * @throws IllegalArgumentException
+   *           if an invalid path is specified
+   */
+  public static void deleteNodeRecursivelyMultiOrSequential(ZKWatcher zkw,
+      boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException {
+    if (pathRoots == null || pathRoots.length <= 0) {
+      LOG.warn("Given path is not valid!");
+      return;
+    }
+    List<ZKUtilOp> ops = new ArrayList<>();
+    for (String eachRoot : pathRoots) {
+      // ZooKeeper Watches are one time triggers; When children of parent nodes are deleted
+      // recursively, must set another watch, get notified of delete node
+      List<String> children = listChildrenBFSAndWatchThem(zkw, eachRoot);
+      // Delete the leaves first and eventually get rid of the root
+      for (int i = children.size() - 1; i >= 0; --i) {
+        ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
+      }
+      try {
+        if (zkw.getRecoverableZooKeeper().exists(eachRoot, zkw) != null) {
+          ops.add(ZKUtilOp.deleteNodeFailSilent(eachRoot));
+        }
+      } catch (InterruptedException e) {
+        zkw.interruptedException(e);
+      }
+    }
+    // atleast one element should exist
+    if (ops.size() > 0) {
+      multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
+    }
+  }
+
+  /**
+   * BFS Traversal of all the children under path, with the entries in the list,
+   * in the same order as that of the traversal. Lists all the children without
+   * setting any watches.
+   *
+   * @param zkw
+   *          - zk reference
+   * @param znode
+   *          - path of node
+   * @return list of children znodes under the path
+   * @throws KeeperException
+   *           if unexpected ZooKeeper exception
+   */
+  private static List<String> listChildrenBFSNoWatch(ZKWatcher zkw,
+      final String znode) throws KeeperException {
+    Deque<String> queue = new LinkedList<>();
+    List<String> tree = new ArrayList<>();
+    queue.add(znode);
+    while (true) {
+      String node = queue.pollFirst();
+      if (node == null) {
+        break;
+      }
+      List<String> children = listChildrenNoWatch(zkw, node);
+      if (children == null) {
+        continue;
+      }
+      for (final String child : children) {
+        final String childPath = node + "/" + child;
+        queue.add(childPath);
+        tree.add(childPath);
+      }
+    }
+    return tree;
+  }
+
+  /**
+   * BFS Traversal of all the children under path, with the entries in the list,
+   * in the same order as that of the traversal.
+   * Lists all the children and set watches on to them.
+   *
+   * @param zkw
+   *          - zk reference
+   * @param znode
+   *          - path of node
+   * @return list of children znodes under the path
+   * @throws KeeperException
+   *           if unexpected ZooKeeper exception
+   */
+  private static List<String> listChildrenBFSAndWatchThem(ZKWatcher zkw, final String znode)
+      throws KeeperException {
+    Deque<String> queue = new LinkedList<>();
+    List<String> tree = new ArrayList<>();
+    queue.add(znode);
+    while (true) {
+      String node = queue.pollFirst();
+      if (node == null) {
+        break;
+      }
+      List<String> children = listChildrenAndWatchThem(zkw, node);
+      if (children == null) {
+        continue;
+      }
+      for (final String child : children) {
+        final String childPath = node + "/" + child;
+        queue.add(childPath);
+        tree.add(childPath);
+      }
+    }
+    return tree;
+  }
+
+  /**
+   * Represents an action taken by ZKUtil, e.g. createAndFailSilent.
+   * These actions are higher-level than ZKOp actions, which represent
+   * individual actions in the ZooKeeper API, like create.
+   */
+  public abstract static class ZKUtilOp {
+    private String path;
+
+    private ZKUtilOp(String path) {
+      this.path = path;
+    }
+
+    /**
+     * @return a createAndFailSilent ZKUtilOp
+     */
+    public static ZKUtilOp createAndFailSilent(String path, byte[] data) {
+      return new CreateAndFailSilent(path, data);
+    }
+
+    /**
+     * @return a deleteNodeFailSilent ZKUtilOP
+     */
+    public static ZKUtilOp deleteNodeFailSilent(String path) {
+      return new DeleteNodeFailSilent(path);
+    }
+
+    /**
+     * @return a setData ZKUtilOp
+     */
+    public static ZKUtilOp setData(String path, byte [] data) {
+      return new SetData(path, data);
+    }
+
+    /**
+     * @return path to znode where the ZKOp will occur
+     */
+    public String getPath() {
+      return path;
+    }
+
+    /**
+     * ZKUtilOp representing createAndFailSilent in ZooKeeper
+     * (attempt to create node, ignore error if already exists)
+     */
+    public static class CreateAndFailSilent extends ZKUtilOp {
+      private byte [] data;
+
+      private CreateAndFailSilent(String path, byte [] data) {
+        super(path);
+        this.data = data;
+      }
+
+      public byte[] getData() {
+        return data;
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof CreateAndFailSilent)) return false;
+
+        CreateAndFailSilent op = (CreateAndFailSilent) o;
+        return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
+      }
+
+      @Override
+      public int hashCode() {
+        int ret = 17 + getPath().hashCode() * 31;
+        return ret * 31 + Bytes.hashCode(data);
+      }
+    }
+
+    /**
+     * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper
+     * (attempt to delete node, ignore error if node doesn't exist)
+     */
+    public static class DeleteNodeFailSilent extends ZKUtilOp {
+      private DeleteNodeFailSilent(String path) {
+        super(path);
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof DeleteNodeFailSilent)) return false;
+
+        return super.equals(o);
+      }
+
+      @Override
+      public int hashCode() {
+        return getPath().hashCode();
+      }
+    }
+
+    /**
+     * ZKUtilOp representing setData in ZooKeeper
+     */
+    public static class SetData extends ZKUtilOp {
+      private byte [] data;
+
+      private SetData(String path, byte [] data) {
+        super(path);
+        this.data = data;
+      }
+
+      public byte[] getData() {
+        return data;
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof SetData)) return false;
+
+        SetData op = (SetData) o;
+        return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
+      }
+
+      @Override
+      public int hashCode() {
+        int ret = getPath().hashCode();
+        return ret * 31 + Bytes.hashCode(data);
+      }
+    }
+  }
+
+  /**
+   * Convert from ZKUtilOp to ZKOp
+   */
+  private static Op toZooKeeperOp(ZKWatcher zkw, ZKUtilOp op)
+  throws UnsupportedOperationException {
+    if(op == null) return null;
+
+    if (op instanceof CreateAndFailSilent) {
+      CreateAndFailSilent cafs = (CreateAndFailSilent)op;
+      return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()),
+        CreateMode.PERSISTENT);
+    } else if (op instanceof DeleteNodeFailSilent) {
+      DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op;
+      return Op.delete(dnfs.getPath(), -1);
+    } else if (op instanceof SetData) {
+      SetData sd = (SetData)op;
+      return Op.setData(sd.getPath(), sd.getData(), -1);
+    } else {
+      throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
+        + op.getClass().getName());
+    }
+  }
+
+  /**
+   * Use ZooKeeper's multi-update functionality.
+   *
+   * If all of the following are true:
+   * - runSequentialOnMultiFailure is true
+   * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*)
+   * Then:
+   * - we retry the operations one-by-one (sequentially)
+   *
+   * Note *: an example is receiving a NodeExistsException from a "create" call.  Without multi,
+   * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who
+   * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught).
+   * This will cause all operations in the multi to fail, however, because
+   * the NodeExistsException that zk.create throws will fail the multi transaction.
+   * In this case, if the previous conditions hold, the commands are run sequentially, which should
+   * result in the correct final state, but means that the operations will not run atomically.
+   *
+   * @throws KeeperException
+   */
+  public static void multiOrSequential(ZKWatcher zkw, List<ZKUtilOp> ops,
+                                       boolean runSequentialOnMultiFailure) throws KeeperException {
+    if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) {
+      LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always.");
+    }
+    if (ops == null) return;
+
+    List<Op> zkOps = new LinkedList<>();
+    for (ZKUtilOp op : ops) {
+      zkOps.add(toZooKeeperOp(zkw, op));
+    }
+    try {
+      zkw.getRecoverableZooKeeper().multi(zkOps);
+    } catch (KeeperException ke) {
+      switch (ke.code()) {
+      case NODEEXISTS:
+      case NONODE:
+      case BADVERSION:
+      case NOAUTH:
+        // if we get an exception that could be solved by running sequentially
+        // (and the client asked us to), then break out and run sequentially
+        if (runSequentialOnMultiFailure) {
+          LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
+              + "  Attempting to run operations sequentially because"
+              + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
+          processSequentially(zkw, ops);
+          break;
+        }
+      default:
+        throw ke;
+      }
+    } catch (InterruptedException ie) {
+      zkw.interruptedException(ie);
+    }
+  }
+
+  private static void processSequentially(ZKWatcher zkw, List<ZKUtilOp> ops)
+      throws KeeperException, NoNodeException {
+    for (ZKUtilOp op : ops) {
+      if (op instanceof CreateAndFailSilent) {
+        createAndFailSilent(zkw, (CreateAndFailSilent) op);
+      } else if (op instanceof DeleteNodeFailSilent) {
+        deleteNodeFailSilent(zkw, (DeleteNodeFailSilent) op);
+      } else if (op instanceof SetData) {
+        setData(zkw, (SetData) op);
+      } else {
+        throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
+            + op.getClass().getName());
+      }
+    }
+  }
+
+  //
+  // ZooKeeper cluster information
+  //
+
+  /** @return String dump of everything in ZooKeeper. */
+  public static String dump(ZKWatcher zkw) {
+    StringBuilder sb = new StringBuilder();
+    try {
+      sb.append("HBase is rooted at ").append(zkw.znodePaths.baseZNode);
+      sb.append("\nActive master address: ");
+      try {
+        sb.append(MasterAddressTracker.getMasterAddress(zkw));
+      } catch (IOException e) {
+        sb.append("<<FAILED LOOKUP: " + e.getMessage() + ">>");
+      }
+      sb.append("\nBackup master addresses:");
+      for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.backupMasterAddressesZNode)) {
+        sb.append("\n ").append(child);
+      }
+      sb.append("\nRegion server holding hbase:meta: "
+        + new MetaTableLocator().getMetaRegionLocation(zkw));
+      Configuration conf = HBaseConfiguration.create();
+      int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
+               HConstants.DEFAULT_META_REPLICA_NUM);
+      for (int i = 1; i < numMetaReplicas; i++) {
+        sb.append("\nRegion server holding hbase:meta, replicaId " + i + " "
+                    + new MetaTableLocator().getMetaRegionLocation(zkw, i));
+      }
+      sb.append("\nRegion servers:");
+      for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode)) {
+        sb.append("\n ").append(child);
+      }
+      try {
+        getReplicationZnodesDump(zkw, sb);
+      } catch (KeeperException ke) {
+        LOG.warn("Couldn't get the replication znode dump", ke);
+      }
+      sb.append("\nQuorum Server Statistics:");
+      String[] servers = zkw.getQuorum().split(",");
+      for (String server : servers) {
+        sb.append("\n ").append(server);
+        try {
+          String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut);
+
+          if (stat == null) {
+            sb.append("[Error] invalid quorum server: " + server);
+            break;
+          }
+
+          for (String s : stat) {
+            sb.append("\n  ").append(s);
+          }
+        } catch (Exception e) {
+          sb.append("\n  ERROR: ").append(e.getMessage());
+        }
+      }
+    } catch (KeeperException ke) {
+      sb.append("\nFATAL ZooKeeper Exception!\n");
+      sb.append("\n" + ke.getMessage());
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Appends replication znodes to the passed StringBuilder.
+   * @param zkw
+   * @param sb
+   * @throws KeeperException
+   */
+  private static void getReplicationZnodesDump(ZKWatcher zkw, StringBuilder sb)
+      throws KeeperException {
+    String replicationZnode = zkw.znodePaths.replicationZNode;
+    if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return;
+    // do a ls -r on this znode
+    sb.append("\n").append(replicationZnode).append(": ");
+    List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
+    for (String child : children) {
+      String znode = ZNodePaths.joinZNode(replicationZnode, child);
+      if (znode.equals(zkw.znodePaths.peersZNode)) {
+        appendPeersZnodes(zkw, znode, sb);
+      } else if (znode.equals(zkw.znodePaths.queuesZNode)) {
+        appendRSZnodes(zkw, znode, sb);
+      } else if (znode.equals(zkw.znodePaths.hfileRefsZNode)) {
+        appendHFileRefsZnodes(zkw, znode, sb);
+      }
+    }
+  }
+
+  private static void appendHFileRefsZnodes(ZKWatcher zkw, String hfileRefsZnode,
+                                            StringBuilder sb) throws KeeperException {
+    sb.append("\n").append(hfileRefsZnode).append(": ");
+    for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, hfileRefsZnode)) {
+      String znodeToProcess = ZNodePaths.joinZNode(hfileRefsZnode, peerIdZnode);
+      sb.append("\n").append(znodeToProcess).append(": ");
+      List<String> peerHFileRefsZnodes = ZKUtil.listChildrenNoWatch(zkw, znodeToProcess);
+      int size = peerHFileRefsZnodes.size();
+      for (int i = 0; i < size; i++) {
+        sb.append(peerHFileRefsZnodes.get(i));
+        if (i != size - 1) {
+          sb.append(", ");
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns a string with replication znodes and position of the replication log
+   * @param zkw
+   * @return aq string of replication znodes and log positions
+   */
+  public static String getReplicationZnodesDump(ZKWatcher zkw) throws KeeperException {
+    StringBuilder sb = new StringBuilder();
+    getReplicationZnodesDump(zkw, sb);
+    return sb.toString();
+  }
+
+  private static void appendRSZnodes(ZKWatcher zkw, String znode, StringBuilder sb)
+      throws KeeperException {
+    List<String> stack = new LinkedList<>();
+    stack.add(znode);
+    do {
+      String znodeToProcess = stack.remove(stack.size() - 1);
+      sb.append("\n").append(znodeToProcess).append(": ");
+      byte[] data;
+      try {
+        data = ZKUtil.getData(zkw, znodeToProcess);
+      } catch (InterruptedException e) {
+        zkw.interruptedException(e);
+        return;
+      }
+      if (data != null && data.length > 0) { // log position
+        long position = 0;
+        try {
+          position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
+          sb.append(position);
+        } catch (DeserializationException ignored) {
+        } catch (InterruptedException e) {
+          zkw.interruptedException(e);
+          return;
+        }
+      }
+      for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
+        stack.add(ZNodePaths.joinZNode(znodeToProcess, zNodeChild));
+      }
+    } while (stack.size() > 0);
+  }
+
+  private static void appendPeersZnodes(ZKWatcher zkw, String peersZnode,
+                                        StringBuilder sb) throws KeeperException {
+    int pblen = ProtobufUtil.lengthOfPBMagic();
+    sb.append("\n").append(peersZnode).append(": ");
+    for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
+      String znodeToProcess = ZNodePaths.joinZNode(peersZnode, peerIdZnode);
+      byte[] data;
+      try {
+        data = ZKUtil.getData(zkw, znodeToProcess);
+      } catch (InterruptedException e) {
+        zkw.interruptedException(e);
+        return;
+      }
+      // parse the data of the above peer znode.
+      try {
+        ReplicationProtos.ReplicationPeer.Builder builder =
+          ReplicationProtos.ReplicationPeer.newBuilder();
+        ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
+        String clusterKey = builder.getClusterkey();
+        sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
+        // add the peer-state.
+        appendPeerState(zkw, znodeToProcess, sb);
+      } catch (IOException ipbe) {
+        LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
+      }
+    }
+  }
+
+  private static void appendPeerState(ZKWatcher zkw, String znodeToProcess,
+                                      StringBuilder sb) throws KeeperException, InvalidProtocolBufferException {
+    String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state",
+      "peer-state");
+    int pblen = ProtobufUtil.lengthOfPBMagic();
+    for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
+      if (!child.equals(peerState)) continue;
+      String peerStateZnode = ZNodePaths.joinZNode(znodeToProcess, child);
+      sb.append("\n").append(peerStateZnode).append(": ");
+      byte[] peerStateData;
+      try {
+        peerStateData = ZKUtil.getData(zkw, peerStateZnode);
+        ReplicationProtos.ReplicationState.Builder builder =
+            ReplicationProtos.ReplicationState.newBuilder();
+        ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen);
+        sb.append(builder.getState().name());
+      } catch (IOException ipbe) {
+        LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
+      } catch (InterruptedException e) {
+        zkw.interruptedException(e);
+        return;
+      }
+    }
+  }
+
+  /**
+   * Gets the statistics from the given server.
+   *
+   * @param server  The server to get the statistics from.
+   * @param timeout  The socket timeout to use.
+   * @return The array of response strings.
+   * @throws IOException When the socket communication fails.
+   */
+  public static String[] getServerStats(String server, int timeout)
+  throws IOException {
+    String[] sp = server.split(":");
+    if (sp == null || sp.length == 0) {
+      return null;
+    }
+
+    String host = sp[0];
+    int port = sp.length > 1 ? Integer.parseInt(sp[1])
+        : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
+
+    InetSocketAddress sockAddr = new InetSocketAddress(host, port);
+    try (Socket socket = new Socket()) {
+      socket.connect(sockAddr, timeout);
+
+      socket.setSoTimeout(timeout);
+      try (PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
+          BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
+        out.println("stat");
+        out.flush();
+        ArrayList<String> res = new ArrayList<>();
+        while (true) {
+          String line = in.readLine();
+          if (line != null) {
+            res.add(line);
+          } else {
+            break;
+          }
+        }
+        return res.toArray(new String[res.size()]);
+      }
+    }
+  }
+
+  private static void logRetrievedMsg(final ZKWatcher zkw,
+      final String znode, final byte [] data, final boolean watcherSet) {
+    if (!LOG.isTraceEnabled()) return;
+    LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) +
+      " byte(s) of data from znode " + znode +
+      (watcherSet? " and set watcher; ": "; data=") +
+      (data == null? "null": data.length == 0? "empty": (
+          znode.startsWith(zkw.znodePaths.metaZNodePrefix)?
+            getServerNameOrEmptyString(data):
+          znode.startsWith(zkw.znodePaths.backupMasterAddressesZNode)?
+            getServerNameOrEmptyString(data):
+          StringUtils.abbreviate(Bytes.toStringBinary(data), 32)))));
+  }
+
+  private static String getServerNameOrEmptyString(final byte [] data) {
+    try {
+      return ProtobufUtil.parseServerNameFrom(data).toString();
+    } catch (DeserializationException e) {
+      return "";
+    }
+  }
+
+  /**
+   * Waits for HBase installation's base (parent) znode to become available.
+   * @throws IOException on ZK errors
+   */
+  public static void waitForBaseZNode(Configuration conf) throws IOException {
+    LOG.info("Waiting until the base znode is available");
+    String parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
+        HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
+    ZooKeeper zk = new ZooKeeper(ZKConfig.getZKQuorumServersString(conf),
+        conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
+        HConstants.DEFAULT_ZK_SESSION_TIMEOUT), EmptyWatcher.instance);
+
+    final int maxTimeMs = 10000;
+    final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
+
+    KeeperException keeperEx = null;
+    try {
+      try {
+        for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
+          try {
+            if (zk.exists(parentZNode, false) != null) {
+              LOG.info("Parent znode exists: " + parentZNode);
+              keeperEx = null;
+              break;
+            }
+          } catch (KeeperException e) {
+            keeperEx = e;
+          }
+          Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
+        }
+      } finally {
+        zk.close();
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+    }
+
+    if (keeperEx != null) {
+      throw new IOException(keeperEx);
+    }
+  }
+
+  /**
+   * Convert a {@link DeserializationException} to a more palatable {@link KeeperException}.
+   * Used when can't let a {@link DeserializationException} out w/o changing public API.
+   * @param e Exception to convert
+   * @return Converted exception
+   */
+  public static KeeperException convert(final DeserializationException e) {
+    KeeperException ke = new KeeperException.DataInconsistencyException();
+    ke.initCause(e);
+    return ke;
+  }
+
+  /**
+   * Recursively print the current state of ZK (non-transactional)
+   * @param root name of the root directory in zk to print
+   */
+  public static void logZKTree(ZKWatcher zkw, String root) {
+    if (!LOG.isDebugEnabled()) return;
+    LOG.debug("Current zk system:");
+    String prefix = "|-";
+    LOG.debug(prefix + root);
+    try {
+      logZKTree(zkw, root, prefix);
+    } catch (KeeperException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Helper method to print the current state of the ZK tree.
+   * @see #logZKTree(ZKWatcher, String)
+   * @throws KeeperException if an unexpected exception occurs
+   */
+  protected static void logZKTree(ZKWatcher zkw, String root, String prefix)
+      throws KeeperException {
+    List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
+    if (children == null) return;
+    for (String child : children) {
+      LOG.debug(prefix + child);
+      String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child);
+      logZKTree(zkw, node, prefix + "---");
+    }
+  }
+
+  /**
+   * @param position
+   * @return Serialized protobuf of <code>position</code> with pb magic prefix prepended suitable
+   *         for use as content of an wal position in a replication queue.
+   */
+  public static byte[] positionToByteArray(final long position) {
+    byte[] bytes = ReplicationProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
+        .build().toByteArray();
+    return ProtobufUtil.prependPBMagic(bytes);
+  }
+
+  /**
+   * @param bytes - Content of a WAL position znode.
+   * @return long - The current WAL position.
+   * @throws DeserializationException
+   */
+  public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException {
+    if (bytes == null) {
+      throw new DeserializationException("Unable to parse null WAL position.");
+    }
+    if (ProtobufUtil.isPBMagicPrefix(bytes)) {
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      ReplicationProtos.ReplicationHLogPosition.Builder builder =
+          ReplicationProtos.ReplicationHLogPosition.newBuilder();
+      ReplicationProtos.ReplicationHLogPosition position;
+      try {
+        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+        position = builder.build();
+      } catch (IOException e) {
+        throw new DeserializationException(e);
+      }
+      return position.getPosition();
+    } else {
+      if (bytes.length > 0) {
+        return Bytes.toLong(bytes);
+      }
+      return 0;
+    }
+  }
+}


[05/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java
deleted file mode 100644
index 951c6ba..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java
+++ /dev/null
@@ -1,351 +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.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.Random;
-import java.util.concurrent.Semaphore;
-
-import junit.framework.Assert;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.master.TestActiveMasterManager.NodeDeletionListener;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, MediumTests.class})
-public class TestZooKeeperNodeTracker {
-  private static final Log LOG = LogFactory.getLog(TestZooKeeperNodeTracker.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  private final static Random rand = new Random();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.startMiniZKCluster();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniZKCluster();
-  }
-
-  /**
-   * Test that we can interrupt a node that is blocked on a wait.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  @Test public void testInterruptible() throws IOException, InterruptedException {
-    Abortable abortable = new StubAbortable();
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-      "testInterruptible", abortable);
-    final TestTracker tracker = new TestTracker(zk, "/xyz", abortable);
-    tracker.start();
-    Thread t = new Thread() {
-      @Override
-      public void run() {
-        try {
-          tracker.blockUntilAvailable();
-        } catch (InterruptedException e) {
-          throw new RuntimeException("Interrupted", e);
-        }
-      }
-    };
-    t.start();
-    while (!t.isAlive()) Threads.sleep(1);
-    tracker.stop();
-    t.join();
-    // If it wasn't interruptible, we'd never get to here.
-  }
-
-  @Test
-  public void testNodeTracker() throws Exception {
-    Abortable abortable = new StubAbortable();
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-        "testNodeTracker", abortable);
-    ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode);
-
-    final String node =
-      ZNodePaths.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString());
-
-    final byte [] dataOne = Bytes.toBytes("dataOne");
-    final byte [] dataTwo = Bytes.toBytes("dataTwo");
-
-    // Start a ZKNT with no node currently available
-    TestTracker localTracker = new TestTracker(zk, node, abortable);
-    localTracker.start();
-    zk.registerListener(localTracker);
-
-    // Make sure we don't have a node
-    assertNull(localTracker.getData(false));
-
-    // Spin up a thread with another ZKNT and have it block
-    WaitToGetDataThread thread = new WaitToGetDataThread(zk, node);
-    thread.start();
-
-    // Verify the thread doesn't have a node
-    assertFalse(thread.hasData);
-
-    // Now, start a new ZKNT with the node already available
-    TestTracker secondTracker = new TestTracker(zk, node, null);
-    secondTracker.start();
-    zk.registerListener(secondTracker);
-
-    // Put up an additional zk listener so we know when zk event is done
-    TestingZKListener zkListener = new TestingZKListener(zk, node);
-    zk.registerListener(zkListener);
-    assertEquals(0, zkListener.createdLock.availablePermits());
-
-    // Create a completely separate zk connection for test triggers and avoid
-    // any weird watcher interactions from the test
-    final ZooKeeper zkconn = new ZooKeeper(
-        ZKConfig.getZKQuorumServersString(TEST_UTIL.getConfiguration()), 60000,
-        new StubWatcher());
-
-    // Add the node with data one
-    zkconn.create(node, dataOne, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
-    // Wait for the zk event to be processed
-    zkListener.waitForCreation();
-    thread.join();
-
-    // Both trackers should have the node available with data one
-    assertNotNull(localTracker.getData(false));
-    assertNotNull(localTracker.blockUntilAvailable());
-    assertTrue(Bytes.equals(localTracker.getData(false), dataOne));
-    assertTrue(thread.hasData);
-    assertTrue(Bytes.equals(thread.tracker.getData(false), dataOne));
-    LOG.info("Successfully got data one");
-
-    // Make sure it's available and with the expected data
-    assertNotNull(secondTracker.getData(false));
-    assertNotNull(secondTracker.blockUntilAvailable());
-    assertTrue(Bytes.equals(secondTracker.getData(false), dataOne));
-    LOG.info("Successfully got data one with the second tracker");
-
-    // Drop the node
-    zkconn.delete(node, -1);
-    zkListener.waitForDeletion();
-
-    // Create a new thread but with the existing thread's tracker to wait
-    TestTracker threadTracker = thread.tracker;
-    thread = new WaitToGetDataThread(zk, node, threadTracker);
-    thread.start();
-
-    // Verify other guys don't have data
-    assertFalse(thread.hasData);
-    assertNull(secondTracker.getData(false));
-    assertNull(localTracker.getData(false));
-    LOG.info("Successfully made unavailable");
-
-    // Create with second data
-    zkconn.create(node, dataTwo, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
-    // Wait for the zk event to be processed
-    zkListener.waitForCreation();
-    thread.join();
-
-    // All trackers should have the node available with data two
-    assertNotNull(localTracker.getData(false));
-    assertNotNull(localTracker.blockUntilAvailable());
-    assertTrue(Bytes.equals(localTracker.getData(false), dataTwo));
-    assertNotNull(secondTracker.getData(false));
-    assertNotNull(secondTracker.blockUntilAvailable());
-    assertTrue(Bytes.equals(secondTracker.getData(false), dataTwo));
-    assertTrue(thread.hasData);
-    assertTrue(Bytes.equals(thread.tracker.getData(false), dataTwo));
-    LOG.info("Successfully got data two on all trackers and threads");
-
-    // Change the data back to data one
-    zkconn.setData(node, dataOne, -1);
-
-    // Wait for zk event to be processed
-    zkListener.waitForDataChange();
-
-    // All trackers should have the node available with data one
-    assertNotNull(localTracker.getData(false));
-    assertNotNull(localTracker.blockUntilAvailable());
-    assertTrue(Bytes.equals(localTracker.getData(false), dataOne));
-    assertNotNull(secondTracker.getData(false));
-    assertNotNull(secondTracker.blockUntilAvailable());
-    assertTrue(Bytes.equals(secondTracker.getData(false), dataOne));
-    assertTrue(thread.hasData);
-    assertTrue(Bytes.equals(thread.tracker.getData(false), dataOne));
-    LOG.info("Successfully got data one following a data change on all trackers and threads");
-  }
-
-  public static class WaitToGetDataThread extends Thread {
-
-    TestTracker tracker;
-    boolean hasData;
-
-    public WaitToGetDataThread(ZooKeeperWatcher zk, String node) {
-      tracker = new TestTracker(zk, node, null);
-      tracker.start();
-      zk.registerListener(tracker);
-      hasData = false;
-    }
-
-    public WaitToGetDataThread(ZooKeeperWatcher zk, String node,
-        TestTracker tracker) {
-      this.tracker = tracker;
-      hasData = false;
-    }
-
-    @Override
-    public void run() {
-      LOG.info("Waiting for data to be available in WaitToGetDataThread");
-      try {
-        tracker.blockUntilAvailable();
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      }
-      LOG.info("Data now available in tracker from WaitToGetDataThread");
-      hasData = true;
-    }
-  }
-
-  public static class TestTracker extends ZooKeeperNodeTracker {
-    public TestTracker(ZooKeeperWatcher watcher, String node,
-        Abortable abortable) {
-      super(watcher, node, abortable);
-    }
-  }
-
-  public static class TestingZKListener extends ZooKeeperListener {
-    private static final Log LOG = LogFactory.getLog(NodeDeletionListener.class);
-
-    private Semaphore deletedLock;
-    private Semaphore createdLock;
-    private Semaphore changedLock;
-    private String node;
-
-    public TestingZKListener(ZooKeeperWatcher watcher, String node) {
-      super(watcher);
-      deletedLock = new Semaphore(0);
-      createdLock = new Semaphore(0);
-      changedLock = new Semaphore(0);
-      this.node = node;
-    }
-
-    @Override
-    public void nodeDeleted(String path) {
-      if(path.equals(node)) {
-        LOG.debug("nodeDeleted(" + path + ")");
-        deletedLock.release();
-      }
-    }
-
-    @Override
-    public void nodeCreated(String path) {
-      if(path.equals(node)) {
-        LOG.debug("nodeCreated(" + path + ")");
-        createdLock.release();
-      }
-    }
-
-    @Override
-    public void nodeDataChanged(String path) {
-      if(path.equals(node)) {
-        LOG.debug("nodeDataChanged(" + path + ")");
-        changedLock.release();
-      }
-    }
-
-    public void waitForDeletion() throws InterruptedException {
-      deletedLock.acquire();
-    }
-
-    public void waitForCreation() throws InterruptedException {
-      createdLock.acquire();
-    }
-
-    public void waitForDataChange() throws InterruptedException {
-      changedLock.acquire();
-    }
-  }
-
-  public static class StubAbortable implements Abortable {
-    @Override
-    public void abort(final String msg, final Throwable t) {}
-    
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-    
-  }
-
-  public static class StubWatcher implements Watcher {
-    @Override
-    public void process(WatchedEvent event) {}
-  }
-
-  @Test
-  public void testCleanZNode() throws Exception {
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-        "testNodeTracker", new TestZooKeeperNodeTracker.StubAbortable());
-
-    final ServerName sn = ServerName.valueOf("127.0.0.1:52", 45L);
-
-    ZKUtil.createAndFailSilent(zkw,
-        TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT,
-            HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT));
-
-    final String nodeName =  zkw.znodePaths.masterAddressZNode;
-
-    // Check that we manage the case when there is no data
-    ZKUtil.createAndFailSilent(zkw, nodeName);
-    MasterAddressTracker.deleteIfEquals(zkw, sn.toString());
-    Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null);
-
-    // Check that we don't delete if we're not supposed to
-    ZKUtil.setData(zkw, nodeName, MasterAddressTracker.toByteArray(sn, 0));
-    MasterAddressTracker.deleteIfEquals(zkw, ServerName.valueOf("127.0.0.2:52", 45L).toString());
-    Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null);
-
-    // Check that we delete when we're supposed to
-    ZKUtil.setData(zkw, nodeName,MasterAddressTracker.toByteArray(sn, 0));
-    MasterAddressTracker.deleteIfEquals(zkw, sn.toString());
-    Assert.assertTrue( ZKUtil.getData(zkw, nodeName)== null );
-
-    // Check that we support the case when the znode does not exist
-    MasterAddressTracker.deleteIfEquals(zkw, sn.toString()); // must not throw an exception
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 0ce38cc..9f8551c 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -328,7 +328,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Returns ZooKeeper status dump
     def zk_dump
-      @zk_wrapper = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(
+      @zk_wrapper = org.apache.hadoop.hbase.zookeeper.ZKWatcher.new(
         @admin.getConfiguration,
         'admin',
         nil

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/pom.xml b/hbase-zookeeper/pom.xml
new file mode 100644
index 0000000..06b7dff
--- /dev/null
+++ b/hbase-zookeeper/pom.xml
@@ -0,0 +1,412 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!--
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+-->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase-build-configuration</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>3.0.0-SNAPSHOT</version>
+    <relativePath>../hbase-build-configuration</relativePath>
+  </parent>
+  <artifactId>hbase-zookeeper</artifactId>
+  <name>Apache HBase - Zookeeper</name>
+  <description>Zookeeper Helpers for HBase</description>
+
+  <build>
+    <!-- Makes sure the resources get added before they are processed
+      by placing this first -->
+    <testResources>
+      <!-- Our test artifact has different license info than our source/bin ones -->
+      <testResource>
+        <directory>src/test/resources/META-INF/</directory>
+        <targetPath>META-INF/</targetPath>
+        <includes>
+          <include>NOTICE</include>
+        </includes>
+        <filtering>true</filtering>
+      </testResource>
+    </testResources>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <!-- Run with -Dmaven.test.skip.exec=true to build -tests.jar without running
+        tests (this is needed for upstream projects whose tests need this jar simply for
+        compilation) -->
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>jar</goal>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <!-- General plugins -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-eclipse-plugin</artifactId>
+        <configuration>
+          <additionalProjectnatures>
+            <projectnature>org.jamon.project.jamonnature</projectnature>
+          </additionalProjectnatures>
+          <buildcommands>
+            <buildcommand>org.jamon.project.templateBuilder</buildcommand>
+            <buildcommand>org.eclipse.jdt.core.javabuilder</buildcommand>
+            <buildcommand>org.jamon.project.markerUpdater</buildcommand>
+          </buildcommands>
+          <additionalConfig>
+            <file>
+              <name>.settings/org.jamon.prefs</name>
+              <content># now
+                eclipse.preferences.version=1
+                templateSourceDir=src/main/jamon
+                templateOutputDir=target/generated-jamon
+              </content>
+            </file>
+          </additionalConfig>
+        </configuration>
+      </plugin>
+      <!-- Run findbugs -->
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+      </plugin>
+      <!-- Testing plugins -->
+      <plugin>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>org.apache.hadoop.hbase.ResourceCheckerJUnitListener</value>
+            </property>
+          </properties>
+        </configuration>
+      </plugin>
+    </plugins>
+    <!-- General Resources -->
+    <pluginManagement>
+       <plugins>
+         <!--This plugin's configuration is used to store Eclipse m2e settings
+             only. It has no influence on the Maven build itself and needs to
+             be kept in plugin management, not in the actual plugins. -->
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-antrun-plugin</artifactId>
+                    <versionRange>[1.6,)</versionRange>
+                    <goals>
+                      <goal>run</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <execute>
+                      <runOnIncremental>false</runOnIncremental>
+                      <runOnConfiguration>true</runOnConfiguration>
+                    </execute>
+                  </action>
+                </pluginExecution>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-dependency-plugin</artifactId>
+                    <versionRange>[2.8,)</versionRange>
+                    <goals>
+                      <goal>build-classpath</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore></ignore>
+                  </action>
+                </pluginExecution>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-compiler-plugin</artifactId>
+                    <versionRange>[3.2,)</versionRange>
+                    <goals>
+                      <goal>compile</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore></ignore>
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-protobuf</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-miscellaneous</artifactId>
+    </dependency>
+    <!-- Intra-project dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol-shaded</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop-compat</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop2-compat</artifactId>
+      <exclusions>
+        <!-- We don't need MR support classes here. -->
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- General dependencies -->
+    <dependency>
+       <groupId>com.github.stephenc.findbugs</groupId>
+       <artifactId>findbugs-annotations</artifactId>
+       <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <profiles>
+    <!-- Needs to make the profile in apache parent pom -->
+    <profile>
+      <id>apache-release</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-resources-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>license-javadocs</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <outputDirectory>${project.build.directory}/apidocs</outputDirectory>
+                  <resources>
+                    <resource>
+                      <directory>src/main/javadoc/META-INF/</directory>
+                      <targetPath>META-INF/</targetPath>
+                      <includes>
+                        <include>LICENSE</include>
+                        <include>NOTICE</include>
+                      </includes>
+                      <filtering>true</filtering>
+                    </resource>
+                  </resources>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <!-- Skip the tests in this module -->
+    <profile>
+      <id>skipZooKeeperTests</id>
+      <activation>
+        <property>
+          <name>skipZooKeeperTests</name>
+        </property>
+      </activation>
+      <properties>
+        <surefire.skipFirstPart>true</surefire.skipFirstPart>
+        <surefire.skipSecondPart>true</surefire.skipSecondPart>
+      </properties>
+    </profile>
+    <!-- Profiles for building against different hadoop versions -->
+    <!-- There are a lot of common dependencies used here, should investigate
+    if we can combine these profiles somehow -->
+
+    <!-- profile for building against Hadoop 2.x.  This is the default.  -->
+    <profile>
+      <id>hadoop-2.0</id>
+      <activation>
+        <property>
+            <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+            <!--h2--><name>!hadoop.profile</name>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-mrapp-generated-classpath</id>
+                <phase>generate-test-resources</phase>
+                <goals>
+                  <goal>build-classpath</goal>
+                </goals>
+                <configuration>
+                  <!-- needed to run the unit test for DS to generate
+                  the required classpath that is required in the env
+                  of the launch container in the mini mr/yarn cluster
+                  -->
+                  <outputFile>${project.build.directory}/test-classes/mrapp-generated-classpath</outputFile>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <!--
+      profile for building against Hadoop 3.0.x. Activate using:
+       mvn -Dhadoop.profile=3.0
+    -->
+    <profile>
+      <id>hadoop-3.0</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>3.0</value>
+        </property>
+      </activation>
+      <properties>
+        <hadoop.version>${hadoop-three.version}</hadoop.version>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-mrapp-generated-classpath</id>
+                <phase>generate-test-resources</phase>
+                <goals>
+                  <goal>build-classpath</goal>
+                </goals>
+                <configuration>
+                  <!-- needed to run the unit test for DS to generate
+                  the required classpath that is required in the env
+                  of the launch container in the mini mr/yarn cluster
+                  -->
+                  <outputFile>${project.build.directory}/test-classes/mrapp-generated-classpath</outputFile>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
new file mode 100644
index 0000000..d145d08
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
@@ -0,0 +1,99 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.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 ZKNodeTracker {
+  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(ZKWatcher 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/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
new file mode 100644
index 0000000..7c02891
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
@@ -0,0 +1,101 @@
+/*
+ * 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 ZKListener {
+
+  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(ZKWatcher 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/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
new file mode 100644
index 0000000..6470faa
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+/**
+ * An empty ZooKeeper watcher
+ */
+@InterfaceAudience.Private
+public class EmptyWatcher implements Watcher {
+  // Used in this package but also by tests so needs to be public
+  public static final EmptyWatcher instance = new EmptyWatcher();
+  private EmptyWatcher() {}
+
+  public void process(WatchedEvent event) {}
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
new file mode 100644
index 0000000..f07b841
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
@@ -0,0 +1,165 @@
+/*
+ *
+ * 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.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT;
+import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.apache.hadoop.hbase.util.DNS;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.zookeeper.server.ServerConfig;
+import org.apache.zookeeper.server.ZooKeeperServerMain;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
+import org.apache.zookeeper.server.quorum.QuorumPeerMain;
+
+/**
+ * HBase's version of ZooKeeper's QuorumPeer. When HBase is set to manage
+ * ZooKeeper, this class is used to start up QuorumPeer instances. By doing
+ * things in here rather than directly calling to ZooKeeper, we have more
+ * control over the process. This class uses {@link ZKConfig} to get settings
+ * from the hbase-site.xml file.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+@InterfaceStability.Evolving
+public class HQuorumPeer {
+
+  /**
+   * Parse ZooKeeper configuration from HBase XML config and run a QuorumPeer.
+   * @param args String[] of command line arguments. Not used.
+   */
+  public static void main(String[] args) {
+    Configuration conf = HBaseConfiguration.create();
+    try {
+      Properties zkProperties = ZKConfig.makeZKProps(conf);
+      writeMyID(zkProperties);
+      QuorumPeerConfig zkConfig = new QuorumPeerConfig();
+      zkConfig.parseProperties(zkProperties);
+
+      // login the zookeeper server principal (if using security)
+      ZKUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE,
+        HConstants.ZK_SERVER_KERBEROS_PRINCIPAL,
+        zkConfig.getClientPortAddress().getHostName());
+
+      runZKServer(zkConfig);
+    } catch (Exception e) {
+      e.printStackTrace();
+      System.exit(-1);
+    }
+  }
+
+  private static void runZKServer(QuorumPeerConfig zkConfig) throws UnknownHostException, IOException {
+    if (zkConfig.isDistributed()) {
+      QuorumPeerMain qp = new QuorumPeerMain();
+      qp.runFromConfig(zkConfig);
+    } else {
+      ZooKeeperServerMain zk = new ZooKeeperServerMain();
+      ServerConfig serverConfig = new ServerConfig();
+      serverConfig.readFrom(zkConfig);
+      zk.runFromConfig(serverConfig);
+    }
+  }
+
+  private static boolean addressIsLocalHost(String address) {
+    return address.equals("localhost") || address.equals("127.0.0.1");
+  }
+
+  static void writeMyID(Properties properties) throws IOException {
+    long myId = -1;
+
+    Configuration conf = HBaseConfiguration.create();
+    String myAddress = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
+        conf.get("hbase.zookeeper.dns.interface","default"),
+        conf.get("hbase.zookeeper.dns.nameserver","default")));
+
+    List<String> ips = new ArrayList<>();
+
+    // Add what could be the best (configured) match
+    ips.add(myAddress.contains(".") ?
+        myAddress :
+        StringUtils.simpleHostname(myAddress));
+
+    // For all nics get all hostnames and IPs
+    Enumeration<?> nics = NetworkInterface.getNetworkInterfaces();
+    while(nics.hasMoreElements()) {
+      Enumeration<?> rawAdrs =
+          ((NetworkInterface)nics.nextElement()).getInetAddresses();
+      while(rawAdrs.hasMoreElements()) {
+        InetAddress inet = (InetAddress) rawAdrs.nextElement();
+        ips.add(StringUtils.simpleHostname(inet.getHostName()));
+        ips.add(inet.getHostAddress());
+      }
+    }
+
+    for (Entry<Object, Object> entry : properties.entrySet()) {
+      String key = entry.getKey().toString().trim();
+      String value = entry.getValue().toString().trim();
+      if (key.startsWith("server.")) {
+        int dot = key.indexOf('.');
+        long id = Long.parseLong(key.substring(dot + 1));
+        String[] parts = value.split(":");
+        String address = parts[0];
+        if (addressIsLocalHost(address) || ips.contains(address)) {
+          myId = id;
+          break;
+        }
+      }
+    }
+
+    // Set the max session timeout from the provided client-side timeout
+    properties.setProperty("maxSessionTimeout",
+      conf.get(HConstants.ZK_SESSION_TIMEOUT, Integer.toString(HConstants.DEFAULT_ZK_SESSION_TIMEOUT)));
+
+    if (myId == -1) {
+      throw new IOException("Could not find my address: " + myAddress +
+                            " in list of ZooKeeper quorum servers");
+    }
+
+    String dataDirStr = properties.get("dataDir").toString().trim();
+    File dataDir = new File(dataDirStr);
+    if (!dataDir.isDirectory()) {
+      if (!dataDir.mkdirs()) {
+        throw new IOException("Unable to create data dir " + dataDir);
+      }
+    }
+
+    File myIdFile = new File(dataDir, "myid");
+    PrintWriter w = new PrintWriter(myIdFile);
+    w.println(myId);
+    w.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
new file mode 100644
index 0000000..e63bfc5
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+/**
+ * Placeholder of an instance which will be accessed by other threads
+ * but is not yet created. Thread safe.
+ */
+class InstancePending<T> {
+  // Based on a subtle part of the Java Language Specification,
+  // in order to avoid a slight overhead of synchronization for each access.
+
+  private final CountDownLatch pendingLatch = new CountDownLatch(1);
+
+  /** Piggybacking on {@code pendingLatch}. */
+  private InstanceHolder<T> instanceHolder;
+
+  private static class InstanceHolder<T> {
+    // The JLS ensures the visibility of a final field and its contents
+    // unless they are exposed to another thread while the construction.
+    final T instance;
+
+    InstanceHolder(T instance) {
+      this.instance = instance;
+    }
+  }
+
+  /**
+   * Returns the instance given by the method {@link #prepare}.
+   * This is an uninterruptible blocking method
+   * and the interruption flag will be set just before returning if any.
+   */
+  T get() {
+    InstanceHolder<T> instanceHolder;
+    boolean interrupted = false;
+
+    while ((instanceHolder = this.instanceHolder) == null) {
+      try {
+        pendingLatch.await();
+      } catch (InterruptedException e) {
+        interrupted = true;
+      }
+    }
+
+    if (interrupted) {
+      Thread.currentThread().interrupt();
+    }
+    return instanceHolder.instance;
+  }
+
+  /**
+   * Associates the given instance for the method {@link #get}.
+   * This method should be called once, and {@code instance} should be non-null.
+   * This method is expected to call as soon as possible
+   * because the method {@code get} is uninterruptibly blocked until this method is called.
+   */
+  void prepare(T instance) {
+    assert instance != null;
+    instanceHolder = new InstanceHolder<>(instance);
+    pendingLatch.countDown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
new file mode 100644
index 0000000..55dafcb
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
@@ -0,0 +1,94 @@
+/*
+ * 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 ZKNodeTracker {
+  private static final Log LOG = LogFactory.getLog(LoadBalancerTracker.class);
+
+  public LoadBalancerTracker(ZKWatcher 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/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
new file mode 100644
index 0000000..85668ad
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
@@ -0,0 +1,281 @@
+/*
+ * 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 org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * Manages the location of the current active Master for the RegionServer.
+ * <p>
+ * Listens for ZooKeeper events related to the master address. The node
+ * <code>/master</code> will contain the address of the current master.
+ * This listener is interested in
+ * <code>NodeDeleted</code> and <code>NodeCreated</code> events on
+ * <code>/master</code>.
+ * <p>
+ * Utilizes {@link ZKNodeTracker} for zk interactions.
+ * <p>
+ * You can get the current master via {@link #getMasterAddress()} or via
+ * {@link #getMasterAddress(ZKWatcher)} if you do not have a running
+ * instance of this Tracker in your context.
+ * <p>
+ * This class also includes utility for interacting with the master znode, for
+ * writing and reading the znode content.
+ */
+@InterfaceAudience.Private
+public class MasterAddressTracker extends ZKNodeTracker {
+  /**
+   * Construct a master address listener with the specified
+   * <code>zookeeper</code> reference.
+   * <p>
+   * This constructor does not trigger any actions, you must call methods
+   * explicitly.  Normally you will just want to execute {@link #start()} to
+   * begin tracking of the master address.
+   *
+   * @param watcher zk reference and watcher
+   * @param abortable abortable in case of fatal error
+   */
+  public MasterAddressTracker(ZKWatcher watcher, Abortable abortable) {
+    super(watcher, watcher.znodePaths.masterAddressZNode, abortable);
+  }
+
+  /**
+   * Get the address of the current master if one is available.  Returns null
+   * if no current master.
+   * @return Server name or null if timed out.
+   */
+  public ServerName getMasterAddress() {
+    return getMasterAddress(false);
+  }
+
+  /**
+   * Get the info port of the current master of one is available.
+   * Return 0 if no current master or zookeeper is unavailable
+   * @return info port or 0 if timed out
+   */
+  public int getMasterInfoPort() {
+    try {
+      final ZooKeeperProtos.Master master = parse(this.getData(false));
+      if (master == null) {
+        return 0;
+      }
+      return master.getInfoPort();
+    } catch (DeserializationException e) {
+      LOG.warn("Failed parse master zk node data", e);
+      return 0;
+    }
+  }
+  /**
+   * Get the info port of the backup master if it is available.
+   * Return 0 if no backup master or zookeeper is unavailable
+   * @param sn server name of backup master
+   * @return info port or 0 if timed out or exceptions
+   */
+  public int getBackupMasterInfoPort(final ServerName sn) {
+    String backupZNode = ZNodePaths.joinZNode(watcher.znodePaths.backupMasterAddressesZNode,
+      sn.toString());
+    try {
+      byte[] data = ZKUtil.getData(watcher, backupZNode);
+      final ZooKeeperProtos.Master backup = parse(data);
+      if (backup == null) {
+        return 0;
+      }
+      return backup.getInfoPort();
+    } catch (Exception e) {
+      LOG.warn("Failed to get backup master: " + sn + "'s info port.", e);
+      return 0;
+    }
+  }
+
+  /**
+   * Get the address of the current master if one is available.  Returns null
+   * if no current master. If refresh is set, try to load the data from ZK again,
+   * otherwise, cached data will be used.
+   *
+   * @param refresh whether to refresh the data by calling ZK directly.
+   * @return Server name or null if timed out.
+   */
+  public ServerName getMasterAddress(final boolean refresh) {
+    try {
+      return ProtobufUtil.parseServerNameFrom(super.getData(refresh));
+    } catch (DeserializationException e) {
+      LOG.warn("Failed parse", e);
+      return null;
+    }
+  }
+
+  /**
+   * Get master address.
+   * Use this instead of {@link #getMasterAddress()} if you do not have an
+   * instance of this tracker in your context.
+   * @param zkw ZKWatcher to use
+   * @return ServerName stored in the the master address znode or null if no
+   * znode present.
+   * @throws KeeperException
+   * @throws IOException
+   */
+  public static ServerName getMasterAddress(final ZKWatcher zkw)
+  throws KeeperException, IOException {
+    byte [] data;
+    try {
+      data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    }
+    // TODO javadoc claims we return null in this case. :/
+    if (data == null){
+      throw new IOException("Can't get master address from ZooKeeper; znode data == null");
+    }
+    try {
+      return ProtobufUtil.parseServerNameFrom(data);
+    } catch (DeserializationException e) {
+      KeeperException ke = new KeeperException.DataInconsistencyException();
+      ke.initCause(e);
+      throw ke;
+    }
+  }
+
+  /**
+   * Get master info port.
+   * Use this instead of {@link #getMasterInfoPort()} if you do not have an
+   * instance of this tracker in your context.
+   * @param zkw ZKWatcher to use
+   * @return master info port in the the master address znode or null if no
+   * znode present.
+   * // TODO can't return null for 'int' return type. non-static verison returns 0
+   * @throws KeeperException
+   * @throws IOException
+   */
+  public static int getMasterInfoPort(final ZKWatcher zkw) throws KeeperException,
+      IOException {
+    byte[] data;
+    try {
+      data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    }
+    // TODO javadoc claims we return null in this case. :/
+    if (data == null) {
+      throw new IOException("Can't get master address from ZooKeeper; znode data == null");
+    }
+    try {
+      return parse(data).getInfoPort();
+    } catch (DeserializationException e) {
+      KeeperException ke = new KeeperException.DataInconsistencyException();
+      ke.initCause(e);
+      throw ke;
+    }
+  }
+
+  /**
+   * Set master address into the <code>master</code> znode or into the backup
+   * subdirectory of backup masters; switch off the passed in <code>znode</code>
+   * path.
+   * @param zkw The ZKWatcher to use.
+   * @param znode Where to create the znode; could be at the top level or it
+   * could be under backup masters
+   * @param master ServerName of the current master must not be null.
+   * @return true if node created, false if not; a watch is set in both cases
+   * @throws KeeperException
+   */
+  public static boolean setMasterAddress(final ZKWatcher zkw,
+      final String znode, final ServerName master, int infoPort)
+  throws KeeperException {
+    return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort));
+  }
+
+  /**
+   * Check if there is a master available.
+   * @return true if there is a master set, false if not.
+   */
+  public boolean hasMaster() {
+    return super.getData(false) != null;
+  }
+
+  /**
+   * @param sn must not be null
+   * @return Content of the master znode as a serialized pb with the pb
+   * magic as prefix.
+   */
+  static byte[] toByteArray(final ServerName sn, int infoPort) {
+    ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
+    HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder();
+    snbuilder.setHostName(sn.getHostname());
+    snbuilder.setPort(sn.getPort());
+    snbuilder.setStartCode(sn.getStartcode());
+    mbuilder.setMaster(snbuilder.build());
+    mbuilder.setRpcVersion(HConstants.RPC_CURRENT_VERSION);
+    mbuilder.setInfoPort(infoPort);
+    return ProtobufUtil.prependPBMagic(mbuilder.build().toByteArray());
+  }
+
+  /**
+   * @param data zookeeper data. may be null
+   * @return pb object of master, null if no active master
+   * @throws DeserializationException
+   */
+  public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException {
+    if (data == null) {
+      return null;
+    }
+    int prefixLen = ProtobufUtil.lengthOfPBMagic();
+    try {
+      return ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+  }
+  /**
+   * delete the master znode if its content is same as the parameter
+   * @param zkw must not be null
+   * @param content must not be null
+   */
+  public static boolean deleteIfEquals(ZKWatcher zkw, final String content) {
+    if (content == null){
+      throw new IllegalArgumentException("Content must not be null");
+    }
+
+    try {
+      Stat stat = new Stat();
+      byte[] data = ZKUtil.getDataNoWatch(zkw, zkw.znodePaths.masterAddressZNode, stat);
+      ServerName sn = ProtobufUtil.parseServerNameFrom(data);
+      if (sn != null && content.equals(sn.toString())) {
+        return (ZKUtil.deleteNode(zkw, zkw.znodePaths.masterAddressZNode, stat.getVersion()));
+      }
+    } catch (KeeperException e) {
+      LOG.warn("Can't get or delete the master znode", e);
+    } catch (DeserializationException e) {
+      LOG.warn("Can't get or delete the master znode", e);
+    }
+
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
new file mode 100644
index 0000000..952da6f
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
@@ -0,0 +1,81 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.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 ZKListener {
+  private boolean hasChildren;
+
+  public MasterMaintenanceModeTracker(ZKWatcher 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);
+  }
+}


[13/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

- Moved DrainingServerTracker and RegionServerTracker to hbase-server:o.a.h.h.master.
- Moved SplitOrMergeTracker to oahh.master (because it depends on a PB)
- Moving hbase-client:oahh.zookeeper.*  to hbase-zookeeper module.  After HBASE-19200, hbase-client doesn't need them anymore (except 3 classes).
- Renamed some classes to use a consistent naming for classes - ZK instead of mix of ZK, Zk , ZooKeeper. Couldn't rename following public classes: MiniZooKeeperCluster, ZooKeeperConnectionException. Left RecoverableZooKeeper for lack of better name. (suggestions?)
- Sadly, can't move tests out because they depend on HBaseTestingUtility (which defeats part of the purpose - trimming down hbase-server tests. We need to promote more use of mocks in our tests)


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

Branch: refs/heads/master
Commit: 330b0d05b99981b4bdc92c81b22ebb5be5ece155
Parents: ca74ec7
Author: Apekshit Sharma <ap...@apache.org>
Authored: Mon Nov 13 11:42:33 2017 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Fri Nov 17 13:20:18 2017 -0800

----------------------------------------------------------------------
 bin/hbase                                       |   14 +-
 bin/hbase.cmd                                   |   10 +-
 conf/log4j.properties                           |    2 +-
 .../src/main/resources/log4j.properties         |    2 +-
 .../src/main/resources/log4j.properties         |    2 +-
 hbase-assembly/pom.xml                          |    4 +
 .../src/main/assembly/hadoop-two-compat.xml     |    1 +
 .../hadoop/hbase/zookeeper/EmptyWatcher.java    |   34 -
 .../hadoop/hbase/zookeeper/HQuorumPeer.java     |  165 --
 .../hadoop/hbase/zookeeper/InstancePending.java |   80 -
 .../hbase/zookeeper/MasterAddressTracker.java   |  281 ---
 .../hbase/zookeeper/MetaTableLocator.java       |  629 ------
 .../hbase/zookeeper/MetricsZooKeeper.java       |  110 -
 .../hadoop/hbase/zookeeper/PendingWatcher.java  |   53 -
 .../hbase/zookeeper/RecoverableZooKeeper.java   |  814 -------
 .../hadoop/hbase/zookeeper/ZKClusterId.java     |   99 -
 .../hadoop/hbase/zookeeper/ZKLeaderManager.java |  183 --
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   | 2072 ------------------
 .../hadoop/hbase/zookeeper/ZkAclReset.java      |  116 -
 .../hbase/zookeeper/ZooKeeperListener.java      |   86 -
 .../zookeeper/ZooKeeperMetricsListener.java     |   91 -
 .../hbase/zookeeper/ZooKeeperNodeTracker.java   |  251 ---
 .../hbase/zookeeper/ZooKeeperWatcher.java       |  638 ------
 .../hbase/zookeeper/TestInstancePending.java    |   49 -
 .../hbase/zookeeper/TestMetricsZooKeeper.java   |   77 -
 .../hadoop/hbase/zookeeper/TestZKUtil.java      |  113 -
 .../hbase/zookeeper/TestZooKeeperWatcher.java   |   57 -
 .../hbase/IntegrationTestMetaReplicas.java      |    6 +-
 .../test/IntegrationTestZKAndFSPermissions.java |    8 +-
 .../apache/hadoop/hbase/mapreduce/Import.java   |    6 +-
 .../replication/VerifyReplication.java          |    6 +-
 hbase-replication/pom.xml                       |    4 +
 .../hbase/replication/ReplicationFactory.java   |   12 +-
 .../replication/ReplicationPeerZKImpl.java      |   14 +-
 .../replication/ReplicationPeersZKImpl.java     |    6 +-
 .../replication/ReplicationQueuesArguments.java |   10 +-
 .../ReplicationQueuesClientArguments.java       |    4 +-
 .../ReplicationQueuesClientZKImpl.java          |   12 +-
 .../replication/ReplicationQueuesZKImpl.java    |   10 +-
 .../replication/ReplicationStateZKBase.java     |   16 +-
 .../replication/ReplicationTrackerZKImpl.java   |   14 +-
 .../TableBasedReplicationQueuesImpl.java        |    5 +-
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java   |    4 +-
 .../rsgroup/VerifyingRSGroupAdminClient.java    |    6 +-
 hbase-server/pom.xml                            |    4 +
 .../java/org/apache/hadoop/hbase/Server.java    |    4 +-
 .../apache/hadoop/hbase/ZKNamespaceManager.java |    8 +-
 .../org/apache/hadoop/hbase/ZNodeClearer.java   |   16 +-
 .../backup/example/HFileArchiveManager.java     |   10 +-
 .../example/TableHFileArchiveTracker.java       |   14 +-
 .../backup/example/ZKTableArchiveClient.java    |    4 +-
 .../ZKSplitLogManagerCoordination.java          |    8 +-
 .../coordination/ZkCoordinatedStateManager.java |    6 +-
 .../ZkSplitLogWorkerCoordination.java           |   14 +-
 .../hbase/master/ActiveMasterManager.java       |   14 +-
 .../hbase/master/DrainingServerTracker.java     |  135 ++
 .../org/apache/hadoop/hbase/master/HMaster.java |    8 +-
 .../hbase/master/MasterMetaBootstrap.java       |    4 +-
 .../hbase/master/MetricsMasterWrapperImpl.java  |    8 +-
 .../hbase/master/RegionServerTracker.java       |  164 ++
 .../hadoop/hbase/master/ServerManager.java      |    7 +-
 .../hbase/master/SplitOrMergeTracker.java       |  153 ++
 .../cleaner/ReplicationZKNodeCleaner.java       |    8 +-
 .../master/replication/ReplicationManager.java  |    6 +-
 .../hbase/procedure/ZKProcedureCoordinator.java |    6 +-
 .../hbase/procedure/ZKProcedureMemberRpcs.java  |    6 +-
 .../hadoop/hbase/procedure/ZKProcedureUtil.java |   10 +-
 .../RegionServerFlushTableProcedureManager.java |    5 +-
 .../hbase/regionserver/HRegionServer.java       |   12 +-
 .../MetricsRegionServerWrapperImpl.java         |    4 +-
 .../snapshot/RegionServerSnapshotManager.java   |    5 +-
 .../replication/HBaseReplicationEndpoint.java   |   14 +-
 .../master/ReplicationHFileCleaner.java         |   11 +-
 .../master/ReplicationLogCleaner.java           |    8 +-
 .../replication/master/TableCFsUpdater.java     |    6 +-
 .../regionserver/DumpReplicationQueues.java     |    8 +-
 .../regionserver/ReplicationSyncUp.java         |   12 +-
 .../hbase/security/access/AccessController.java |    4 +-
 .../hbase/security/access/TableAuthManager.java |    8 +-
 .../security/access/ZKPermissionWatcher.java    |   11 +-
 .../token/AuthenticationTokenSecretManager.java |   14 +-
 .../hadoop/hbase/security/token/TokenUtil.java  |    4 +-
 .../hbase/security/token/ZKSecretWatcher.java   |   12 +-
 .../DefaultVisibilityLabelServiceImpl.java      |    4 +-
 .../visibility/VisibilityLabelsCache.java       |    8 +-
 .../visibility/ZKVisibilityLabelWatcher.java    |   14 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |    8 +-
 .../apache/hadoop/hbase/util/RegionMover.java   |    4 +-
 .../hadoop/hbase/util/ZKDataMigrator.java       |   10 +-
 .../hbase/util/hbck/ReplicationChecker.java     |    6 +-
 .../hbase/zookeeper/ClusterStatusTracker.java   |   99 -
 .../hbase/zookeeper/DeletionListener.java       |  101 -
 .../hbase/zookeeper/DrainingServerTracker.java  |  134 --
 .../hbase/zookeeper/LoadBalancerTracker.java    |   94 -
 .../zookeeper/MasterMaintenanceModeTracker.java |   81 -
 .../hbase/zookeeper/MiniZooKeeperCluster.java   |  472 ----
 .../zookeeper/RegionNormalizerTracker.java      |   94 -
 .../hbase/zookeeper/RegionServerTracker.java    |  162 --
 .../hbase/zookeeper/SplitOrMergeTracker.java    |  149 --
 .../hadoop/hbase/zookeeper/ZKServerTool.java    |   65 -
 .../hadoop/hbase/zookeeper/ZKSplitLog.java      |  124 --
 .../hbase/zookeeper/ZooKeeperMainServer.java    |  126 --
 .../main/resources/hbase-webapps/master/zk.jsp  |    4 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   22 +-
 .../hadoop/hbase/MockRegionServerServices.java  |   11 +-
 .../hbase/TestMetaTableAccessorNoCluster.java   |    4 +-
 .../hadoop/hbase/TestMetaTableLocator.java      |    6 +-
 .../org/apache/hadoop/hbase/TestZooKeeper.java  |   18 +-
 .../TestZooKeeperTableArchiveClient.java        |    4 +-
 .../hbase/client/TestMetaWithReplicas.java      |   10 +-
 .../replication/TestReplicationAdmin.java       |    8 +-
 ...TestMasterCoprocessorExceptionWithAbort.java |   10 +-
 ...estMasterCoprocessorExceptionWithRemove.java |   10 +-
 .../hbase/master/MockNoopMasterServices.java    |    4 +-
 .../hadoop/hbase/master/MockRegionServer.java   |    9 +-
 .../hbase/master/TestActiveMasterManager.java   |   26 +-
 .../hbase/master/TestAssignmentListener.java    |    6 +-
 .../hbase/master/TestClockSkewDetection.java    |    6 -
 .../master/TestDistributedLogSplitting.java     |   22 +-
 .../hbase/master/TestHMasterRPCException.java   |    4 +-
 .../hbase/master/TestMasterNoCluster.java       |    4 +-
 .../hbase/master/TestMasterStatusServlet.java   |    4 +-
 .../hbase/master/TestSplitLogManager.java       |   12 +-
 .../hbase/master/TestTableStateManager.java     |    8 +-
 .../hbase/master/cleaner/TestHFileCleaner.java  |    6 +-
 .../master/cleaner/TestHFileLinkCleaner.java    |    6 +-
 .../hbase/master/cleaner/TestLogsCleaner.java   |   10 +-
 .../cleaner/TestReplicationHFileCleaner.java    |   10 +-
 .../cleaner/TestReplicationZKNodeCleaner.java   |    6 +-
 .../procedure/SimpleRSProcedureManager.java     |    4 +-
 .../hadoop/hbase/procedure/TestZKProcedure.java |   14 +-
 .../procedure/TestZKProcedureControllers.java   |   28 +-
 .../TestCompactionInDeadRegionServer.java       |    5 +-
 .../regionserver/TestHeapMemoryManager.java     |    4 +-
 .../regionserver/TestMasterAddressTracker.java  |   10 +-
 .../hbase/regionserver/TestRSStatusServlet.java |    4 +-
 .../regionserver/TestRegionServerHostname.java  |    6 +-
 .../regionserver/TestRegionServerNoMaster.java  |    4 +-
 .../hbase/regionserver/TestSplitLogWorker.java  |   12 +-
 .../hbase/regionserver/TestWALLockup.java       |    4 +-
 .../replication/TestMasterReplication.java      |    6 +-
 .../replication/TestMultiSlaveReplication.java  |   19 +-
 .../replication/TestPerTableCFReplication.java  |    8 +-
 .../hbase/replication/TestReplicationBase.java  |   12 +-
 .../TestReplicationStateHBaseImpl.java          |    6 +-
 .../replication/TestReplicationStateZKImpl.java |    8 +-
 .../replication/TestReplicationTableBase.java   |    4 +-
 .../TestReplicationTrackerZKImpl.java           |   14 +-
 .../replication/TestSerialReplication.java      |    6 +-
 .../replication/master/TestTableCFsUpdater.java |    6 +-
 .../regionserver/TestGlobalThrottler.java       |    6 +-
 .../TestReplicationSourceManager.java           |    8 +-
 .../security/access/TestAccessController2.java  |    4 +-
 .../security/access/TestTablePermissions.java   |    6 +-
 .../access/TestZKPermissionWatcher.java         |    6 +-
 .../security/token/TestTokenAuthentication.java |    8 +-
 .../security/token/TestZKSecretWatcher.java     |   26 +-
 .../token/TestZKSecretWatcherRefreshKeys.java   |   16 +-
 ...sibilityLabelReplicationWithExpAsString.java |    6 +-
 .../TestVisibilityLabelsReplication.java        |   10 +-
 .../apache/hadoop/hbase/util/MockServer.java    |   10 +-
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   |    4 +-
 .../zookeeper/TestRecoverableZooKeeper.java     |    2 +-
 .../hbase/zookeeper/TestZKLeaderManager.java    |   14 +-
 .../hbase/zookeeper/TestZKMainServer.java       |  118 +
 .../hadoop/hbase/zookeeper/TestZKMulti.java     |    4 +-
 .../hbase/zookeeper/TestZKNodeTracker.java      |  351 +++
 .../hbase/zookeeper/TestZooKeeperACL.java       |   12 +-
 .../zookeeper/TestZooKeeperMainServer.java      |  118 -
 .../zookeeper/TestZooKeeperNodeTracker.java     |  351 ---
 hbase-shell/src/main/ruby/hbase/admin.rb        |    2 +-
 hbase-zookeeper/pom.xml                         |  412 ++++
 .../hbase/zookeeper/ClusterStatusTracker.java   |   99 +
 .../hbase/zookeeper/DeletionListener.java       |  101 +
 .../hadoop/hbase/zookeeper/EmptyWatcher.java    |   34 +
 .../hadoop/hbase/zookeeper/HQuorumPeer.java     |  165 ++
 .../hadoop/hbase/zookeeper/InstancePending.java |   80 +
 .../hbase/zookeeper/LoadBalancerTracker.java    |   94 +
 .../hbase/zookeeper/MasterAddressTracker.java   |  281 +++
 .../zookeeper/MasterMaintenanceModeTracker.java |   81 +
 .../hbase/zookeeper/MetaTableLocator.java       |  629 ++++++
 .../hbase/zookeeper/MiniZooKeeperCluster.java   |  472 ++++
 .../hadoop/hbase/zookeeper/PendingWatcher.java  |   53 +
 .../hbase/zookeeper/RecoverableZooKeeper.java   |  810 +++++++
 .../zookeeper/RegionNormalizerTracker.java      |   94 +
 .../hadoop/hbase/zookeeper/ZKAclReset.java      |  116 +
 .../hadoop/hbase/zookeeper/ZKClusterId.java     |   99 +
 .../hadoop/hbase/zookeeper/ZKLeaderManager.java |  183 ++
 .../hadoop/hbase/zookeeper/ZKListener.java      |   85 +
 .../hadoop/hbase/zookeeper/ZKMainServer.java    |  126 ++
 .../hadoop/hbase/zookeeper/ZKMetrics.java       |  108 +
 .../hbase/zookeeper/ZKMetricsListener.java      |   91 +
 .../hadoop/hbase/zookeeper/ZKNodeTracker.java   |  251 +++
 .../hadoop/hbase/zookeeper/ZKServerTool.java    |   65 +
 .../hadoop/hbase/zookeeper/ZKSplitLog.java      |  122 ++
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   | 2072 ++++++++++++++++++
 .../hadoop/hbase/zookeeper/ZKWatcher.java       |  634 ++++++
 .../hbase/zookeeper/TestInstancePending.java    |   49 +
 .../hadoop/hbase/zookeeper/TestZKMetrics.java   |   80 +
 .../hadoop/hbase/zookeeper/TestZKUtil.java      |  113 +
 .../hadoop/hbase/zookeeper/TestZKWatcher.java   |   57 +
 pom.xml                                         |    6 +
 202 files changed, 9129 insertions(+), 8735 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 998bdbe..d98e7bc 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -17,7 +17,7 @@
 # * See the License for the specific language governing permissions and
 # * limitations under the License.
 # */
-# 
+#
 # The hbase command script.  Based on the hadoop command script putting
 # in hbase classes, libs and configurations ahead of hadoop's.
 #
@@ -91,10 +91,10 @@ if [ $# = 0 ]; then
   echo "  hfile           Store file analyzer"
   echo "  zkcli           Run the ZooKeeper shell"
   echo "  master          Run an HBase HMaster node"
-  echo "  regionserver    Run an HBase HRegionServer node" 
+  echo "  regionserver    Run an HBase HRegionServer node"
   echo "  zookeeper       Run a ZooKeeper server"
-  echo "  rest            Run an HBase REST server" 
-  echo "  thrift          Run the HBase Thrift server" 
+  echo "  rest            Run an HBase REST server"
+  echo "  thrift          Run the HBase Thrift server"
   echo "  thrift2         Run the HBase Thrift2 server"
   echo "  clean           Run the HBase clean up script"
   echo "  classpath       Dump hbase CLASSPATH"
@@ -274,7 +274,7 @@ fi
 if $cygwin; then
   JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`
 fi
- 
+
 # restore ordinary behaviour
 unset IFS
 
@@ -362,7 +362,7 @@ elif [ "$COMMAND" = "wal" ] ; then
 elif [ "$COMMAND" = "hfile" ] ; then
   CLASS='org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter'
 elif [ "$COMMAND" = "zkcli" ] ; then
-  CLASS="org.apache.hadoop.hbase.zookeeper.ZooKeeperMainServer"
+  CLASS="org.apache.hadoop.hbase.zookeeper.ZKMainServer"
 elif [ "$COMMAND" = "backup" ] ; then
   CLASS='org.apache.hadoop.hbase.backup.BackupDriver'
 elif [ "$COMMAND" = "restore" ] ; then
@@ -422,7 +422,7 @@ elif [ "$COMMAND" = "zookeeper" ] ; then
   fi
 elif [ "$COMMAND" = "clean" ] ; then
   case $1 in
-    --cleanZk|--cleanHdfs|--cleanAll) 
+    --cleanZk|--cleanHdfs|--cleanAll)
       matches="yes" ;;
     *) ;;
   esac

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/bin/hbase.cmd
----------------------------------------------------------------------
diff --git a/bin/hbase.cmd b/bin/hbase.cmd
index 070cb61..8e4a59f 100644
--- a/bin/hbase.cmd
+++ b/bin/hbase.cmd
@@ -16,7 +16,7 @@
 @rem * See the License for the specific language governing permissions and
 @rem * limitations under the License.
 @rem */
-@rem 
+@rem
 @rem The hbase command script.  Based on the hadoop command script putting
 @rem in hbase classes, libs and configurations ahead of hadoop's.
 @rem
@@ -197,7 +197,7 @@ if exist "%HBASE_HOME%\build\native" (
 rem This loop would set %hbase-command-arguments%
 set _hbasearguments=
 :MakeCmdArgsLoop
-  if [%1]==[] goto :EndLoop 
+  if [%1]==[] goto :EndLoop
 
   if not defined _hbasearguments (
     set _hbasearguments=%1
@@ -205,8 +205,8 @@ set _hbasearguments=
     set _hbasearguments=!_hbasearguments! %1
   )
   shift
-goto :MakeCmdArgsLoop 
-:EndLoop 
+goto :MakeCmdArgsLoop
+:EndLoop
 
 set hbase-command-arguments=%_hbasearguments%
 
@@ -424,7 +424,7 @@ goto :eof
   goto :eof
 
 :zkcli
-  set CLASS=org.apache.hadoop.hbase.zookeeper.ZooKeeperMainServer
+  set CLASS=org.apache.hadoop.hbase.zookeeper.ZKMainServer
   goto :eof
 
 :mapredcp

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/conf/log4j.properties b/conf/log4j.properties
index 15545ff..6367dbe 100644
--- a/conf/log4j.properties
+++ b/conf/log4j.properties
@@ -98,7 +98,7 @@ log4j.logger.org.apache.hadoop.hbase=INFO
 log4j.logger.org.apache.hadoop.hbase.META=INFO
 # Make these two classes INFO-level. Make them DEBUG to see more zk debug.
 log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
-log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKWatcher=INFO
 #log4j.logger.org.apache.hadoop.dfs=DEBUG
 # Set this class to log INFO only otherwise its OTT
 # Enable this to get detailed connection error/retry logging.

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties b/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties
index 11f2b75..0b01e57 100644
--- a/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties
+++ b/hbase-archetypes/hbase-client-project/src/main/resources/log4j.properties
@@ -95,7 +95,7 @@ log4j.logger.org.apache.zookeeper=INFO
 log4j.logger.org.apache.hadoop.hbase=INFO
 # Make these two classes INFO-level. Make them DEBUG to see more zk debug.
 log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
-log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKWatcher=INFO
 #log4j.logger.org.apache.hadoop.dfs=DEBUG
 # Set this class to log INFO only otherwise its OTT
 # Enable this to get detailed connection error/retry logging.

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties b/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
index 11f2b75..0b01e57 100644
--- a/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
@@ -95,7 +95,7 @@ log4j.logger.org.apache.zookeeper=INFO
 log4j.logger.org.apache.hadoop.hbase=INFO
 # Make these two classes INFO-level. Make them DEBUG to see more zk debug.
 log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
-log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKWatcher=INFO
 #log4j.logger.org.apache.hadoop.dfs=DEBUG
 # Set this class to log INFO only otherwise its OTT
 # Enable this to get detailed connection error/retry logging.

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 71e90f3..75cba4d 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -303,6 +303,10 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-examples</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-zookeeper</artifactId>
+    </dependency>
   </dependencies>
   <profiles>
     <profile>

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
index 03951bd..2aa3e57 100644
--- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
+++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
@@ -59,6 +59,7 @@
         <include>org.apache.hbase:hbase-spark-it</include>
         <include>org.apache.hbase:hbase-testing-util</include>
         <include>org.apache.hbase:hbase-thrift</include>
+        <include>org.apache.hbase:hbase-zookeeper</include>
       </includes>
       <!-- Binaries for the dependencies also go in the hbase-jars directory -->
       <binaries>

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java
deleted file mode 100644
index 191943a..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.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.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-
-/**
- * An empty ZooKeeper watcher
- */
-@InterfaceAudience.Private
-public class EmptyWatcher implements Watcher {
-  // Used in this package but also by tests so needs to be public
-  public static final EmptyWatcher instance = new EmptyWatcher();
-  private EmptyWatcher() {}
-
-  public void process(WatchedEvent event) {}
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
deleted file mode 100644
index 5f3904a..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
+++ /dev/null
@@ -1,165 +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.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT;
-import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Properties;
-
-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.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.server.ServerConfig;
-import org.apache.zookeeper.server.ZooKeeperServerMain;
-import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
-import org.apache.zookeeper.server.quorum.QuorumPeerMain;
-
-/**
- * HBase's version of ZooKeeper's QuorumPeer. When HBase is set to manage
- * ZooKeeper, this class is used to start up QuorumPeer instances. By doing
- * things in here rather than directly calling to ZooKeeper, we have more
- * control over the process. This class uses {@link ZKConfig} to get settings
- * from the hbase-site.xml file.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-@InterfaceStability.Evolving
-public class HQuorumPeer {
-
-  /**
-   * Parse ZooKeeper configuration from HBase XML config and run a QuorumPeer.
-   * @param args String[] of command line arguments. Not used.
-   */
-  public static void main(String[] args) {
-    Configuration conf = HBaseConfiguration.create();
-    try {
-      Properties zkProperties = ZKConfig.makeZKProps(conf);
-      writeMyID(zkProperties);
-      QuorumPeerConfig zkConfig = new QuorumPeerConfig();
-      zkConfig.parseProperties(zkProperties);
-
-      // login the zookeeper server principal (if using security)
-      ZKUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE,
-        HConstants.ZK_SERVER_KERBEROS_PRINCIPAL,
-        zkConfig.getClientPortAddress().getHostName());
-
-      runZKServer(zkConfig);
-    } catch (Exception e) {
-      e.printStackTrace();
-      System.exit(-1);
-    }
-  }
-
-  private static void runZKServer(QuorumPeerConfig zkConfig) throws UnknownHostException, IOException {
-    if (zkConfig.isDistributed()) {
-      QuorumPeerMain qp = new QuorumPeerMain();
-      qp.runFromConfig(zkConfig);
-    } else {
-      ZooKeeperServerMain zk = new ZooKeeperServerMain();
-      ServerConfig serverConfig = new ServerConfig();
-      serverConfig.readFrom(zkConfig);
-      zk.runFromConfig(serverConfig);
-    }
-  }
-
-  private static boolean addressIsLocalHost(String address) {
-    return address.equals("localhost") || address.equals("127.0.0.1");
-  }
-
-  static void writeMyID(Properties properties) throws IOException {
-    long myId = -1;
-
-    Configuration conf = HBaseConfiguration.create();
-    String myAddress = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
-        conf.get("hbase.zookeeper.dns.interface","default"),
-        conf.get("hbase.zookeeper.dns.nameserver","default")));
-
-    List<String> ips = new ArrayList<>();
-
-    // Add what could be the best (configured) match
-    ips.add(myAddress.contains(".") ?
-        myAddress :
-        StringUtils.simpleHostname(myAddress));
-
-    // For all nics get all hostnames and IPs
-    Enumeration<?> nics = NetworkInterface.getNetworkInterfaces();
-    while(nics.hasMoreElements()) {
-      Enumeration<?> rawAdrs =
-          ((NetworkInterface)nics.nextElement()).getInetAddresses();
-      while(rawAdrs.hasMoreElements()) {
-        InetAddress inet = (InetAddress) rawAdrs.nextElement();
-        ips.add(StringUtils.simpleHostname(inet.getHostName()));
-        ips.add(inet.getHostAddress());
-      }
-    }
-
-    for (Entry<Object, Object> entry : properties.entrySet()) {
-      String key = entry.getKey().toString().trim();
-      String value = entry.getValue().toString().trim();
-      if (key.startsWith("server.")) {
-        int dot = key.indexOf('.');
-        long id = Long.parseLong(key.substring(dot + 1));
-        String[] parts = value.split(":");
-        String address = parts[0];
-        if (addressIsLocalHost(address) || ips.contains(address)) {
-          myId = id;
-          break;
-        }
-      }
-    }
-
-    // Set the max session timeout from the provided client-side timeout
-    properties.setProperty("maxSessionTimeout",
-      conf.get(ZK_SESSION_TIMEOUT, Integer.toString(DEFAULT_ZK_SESSION_TIMEOUT)));
-
-    if (myId == -1) {
-      throw new IOException("Could not find my address: " + myAddress +
-                            " in list of ZooKeeper quorum servers");
-    }
-
-    String dataDirStr = properties.get("dataDir").toString().trim();
-    File dataDir = new File(dataDirStr);
-    if (!dataDir.isDirectory()) {
-      if (!dataDir.mkdirs()) {
-        throw new IOException("Unable to create data dir " + dataDir);
-      }
-    }
-
-    File myIdFile = new File(dataDir, "myid");
-    PrintWriter w = new PrintWriter(myIdFile);
-    w.println(myId);
-    w.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
deleted file mode 100644
index e63bfc5..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java
+++ /dev/null
@@ -1,80 +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.concurrent.CountDownLatch;
-
-/**
- * Placeholder of an instance which will be accessed by other threads
- * but is not yet created. Thread safe.
- */
-class InstancePending<T> {
-  // Based on a subtle part of the Java Language Specification,
-  // in order to avoid a slight overhead of synchronization for each access.
-
-  private final CountDownLatch pendingLatch = new CountDownLatch(1);
-
-  /** Piggybacking on {@code pendingLatch}. */
-  private InstanceHolder<T> instanceHolder;
-
-  private static class InstanceHolder<T> {
-    // The JLS ensures the visibility of a final field and its contents
-    // unless they are exposed to another thread while the construction.
-    final T instance;
-
-    InstanceHolder(T instance) {
-      this.instance = instance;
-    }
-  }
-
-  /**
-   * Returns the instance given by the method {@link #prepare}.
-   * This is an uninterruptible blocking method
-   * and the interruption flag will be set just before returning if any.
-   */
-  T get() {
-    InstanceHolder<T> instanceHolder;
-    boolean interrupted = false;
-
-    while ((instanceHolder = this.instanceHolder) == null) {
-      try {
-        pendingLatch.await();
-      } catch (InterruptedException e) {
-        interrupted = true;
-      }
-    }
-
-    if (interrupted) {
-      Thread.currentThread().interrupt();
-    }
-    return instanceHolder.instance;
-  }
-
-  /**
-   * Associates the given instance for the method {@link #get}.
-   * This method should be called once, and {@code instance} should be non-null.
-   * This method is expected to call as soon as possible
-   * because the method {@code get} is uninterruptibly blocked until this method is called.
-   */
-  void prepare(T instance) {
-    assert instance != null;
-    instanceHolder = new InstanceHolder<>(instance);
-    pendingLatch.countDown();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
deleted file mode 100644
index fba637f..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
+++ /dev/null
@@ -1,281 +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 org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Manages the location of the current active Master for the RegionServer.
- * <p>
- * Listens for ZooKeeper events related to the master address. The node
- * <code>/master</code> will contain the address of the current master.
- * This listener is interested in
- * <code>NodeDeleted</code> and <code>NodeCreated</code> events on
- * <code>/master</code>.
- * <p>
- * Utilizes {@link ZooKeeperNodeTracker} for zk interactions.
- * <p>
- * You can get the current master via {@link #getMasterAddress()} or via
- * {@link #getMasterAddress(ZooKeeperWatcher)} if you do not have a running
- * instance of this Tracker in your context.
- * <p>
- * This class also includes utility for interacting with the master znode, for
- * writing and reading the znode content.
- */
-@InterfaceAudience.Private
-public class MasterAddressTracker extends ZooKeeperNodeTracker {
-  /**
-   * Construct a master address listener with the specified
-   * <code>zookeeper</code> reference.
-   * <p>
-   * This constructor does not trigger any actions, you must call methods
-   * explicitly.  Normally you will just want to execute {@link #start()} to
-   * begin tracking of the master address.
-   *
-   * @param watcher zk reference and watcher
-   * @param abortable abortable in case of fatal error
-   */
-  public MasterAddressTracker(ZooKeeperWatcher watcher, Abortable abortable) {
-    super(watcher, watcher.znodePaths.masterAddressZNode, abortable);
-  }
-
-  /**
-   * Get the address of the current master if one is available.  Returns null
-   * if no current master.
-   * @return Server name or null if timed out.
-   */
-  public ServerName getMasterAddress() {
-    return getMasterAddress(false);
-  }
-
-  /**
-   * Get the info port of the current master of one is available.
-   * Return 0 if no current master or zookeeper is unavailable
-   * @return info port or 0 if timed out
-   */
-  public int getMasterInfoPort() {
-    try {
-      final ZooKeeperProtos.Master master = parse(this.getData(false));
-      if (master == null) {
-        return 0;
-      }
-      return master.getInfoPort();
-    } catch (DeserializationException e) {
-      LOG.warn("Failed parse master zk node data", e);
-      return 0;
-    }
-  }
-  /**
-   * Get the info port of the backup master if it is available.
-   * Return 0 if no backup master or zookeeper is unavailable
-   * @param sn server name of backup master
-   * @return info port or 0 if timed out or exceptions
-   */
-  public int getBackupMasterInfoPort(final ServerName sn) {
-    String backupZNode = ZNodePaths.joinZNode(watcher.znodePaths.backupMasterAddressesZNode,
-      sn.toString());
-    try {
-      byte[] data = ZKUtil.getData(watcher, backupZNode);
-      final ZooKeeperProtos.Master backup = parse(data);
-      if (backup == null) {
-        return 0;
-      }
-      return backup.getInfoPort();
-    } catch (Exception e) {
-      LOG.warn("Failed to get backup master: " + sn + "'s info port.", e);
-      return 0;
-    }
-  }
-
-  /**
-   * Get the address of the current master if one is available.  Returns null
-   * if no current master. If refresh is set, try to load the data from ZK again,
-   * otherwise, cached data will be used.
-   *
-   * @param refresh whether to refresh the data by calling ZK directly.
-   * @return Server name or null if timed out.
-   */
-  public ServerName getMasterAddress(final boolean refresh) {
-    try {
-      return ProtobufUtil.parseServerNameFrom(super.getData(refresh));
-    } catch (DeserializationException e) {
-      LOG.warn("Failed parse", e);
-      return null;
-    }
-  }
-
-  /**
-   * Get master address.
-   * Use this instead of {@link #getMasterAddress()} if you do not have an
-   * instance of this tracker in your context.
-   * @param zkw ZooKeeperWatcher to use
-   * @return ServerName stored in the the master address znode or null if no
-   * znode present.
-   * @throws KeeperException 
-   * @throws IOException 
-   */
-  public static ServerName getMasterAddress(final ZooKeeperWatcher zkw)
-  throws KeeperException, IOException {
-    byte [] data;
-    try {
-      data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException();
-    }
-    // TODO javadoc claims we return null in this case. :/
-    if (data == null){
-      throw new IOException("Can't get master address from ZooKeeper; znode data == null");
-    }
-    try {
-      return ProtobufUtil.parseServerNameFrom(data);
-    } catch (DeserializationException e) {
-      KeeperException ke = new KeeperException.DataInconsistencyException();
-      ke.initCause(e);
-      throw ke;
-    }
-  }
-
-  /**
-   * Get master info port.
-   * Use this instead of {@link #getMasterInfoPort()} if you do not have an
-   * instance of this tracker in your context.
-   * @param zkw ZooKeeperWatcher to use
-   * @return master info port in the the master address znode or null if no
-   * znode present.
-   * // TODO can't return null for 'int' return type. non-static verison returns 0
-   * @throws KeeperException
-   * @throws IOException
-   */
-  public static int getMasterInfoPort(final ZooKeeperWatcher zkw) throws KeeperException,
-      IOException {
-    byte[] data;
-    try {
-      data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException();
-    }
-    // TODO javadoc claims we return null in this case. :/
-    if (data == null) {
-      throw new IOException("Can't get master address from ZooKeeper; znode data == null");
-    }
-    try {
-      return parse(data).getInfoPort();
-    } catch (DeserializationException e) {
-      KeeperException ke = new KeeperException.DataInconsistencyException();
-      ke.initCause(e);
-      throw ke;
-    }
-  }
-
-  /**
-   * Set master address into the <code>master</code> znode or into the backup
-   * subdirectory of backup masters; switch off the passed in <code>znode</code>
-   * path.
-   * @param zkw The ZooKeeperWatcher to use.
-   * @param znode Where to create the znode; could be at the top level or it
-   * could be under backup masters
-   * @param master ServerName of the current master must not be null.
-   * @return true if node created, false if not; a watch is set in both cases
-   * @throws KeeperException
-   */
-  public static boolean setMasterAddress(final ZooKeeperWatcher zkw,
-      final String znode, final ServerName master, int infoPort)
-  throws KeeperException {
-    return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort));
-  }
-
-  /**
-   * Check if there is a master available.
-   * @return true if there is a master set, false if not.
-   */
-  public boolean hasMaster() {
-    return super.getData(false) != null;
-  }
-
-  /**
-   * @param sn must not be null
-   * @return Content of the master znode as a serialized pb with the pb
-   * magic as prefix.
-   */
-  static byte[] toByteArray(final ServerName sn, int infoPort) {
-    ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
-    HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder();
-    snbuilder.setHostName(sn.getHostname());
-    snbuilder.setPort(sn.getPort());
-    snbuilder.setStartCode(sn.getStartcode());
-    mbuilder.setMaster(snbuilder.build());
-    mbuilder.setRpcVersion(HConstants.RPC_CURRENT_VERSION);
-    mbuilder.setInfoPort(infoPort);
-    return ProtobufUtil.prependPBMagic(mbuilder.build().toByteArray());
-  }
-
-  /**
-   * @param data zookeeper data. may be null
-   * @return pb object of master, null if no active master
-   * @throws DeserializationException
-   */
-  public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException {
-    if (data == null) {
-      return null;
-    }
-    int prefixLen = ProtobufUtil.lengthOfPBMagic();
-    try {
-      return ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen);
-    } catch (InvalidProtocolBufferException e) {
-      throw new DeserializationException(e);
-    }
-  }
-  /**
-   * delete the master znode if its content is same as the parameter
-   * @param zkw must not be null
-   * @param content must not be null
-   */
-  public static boolean deleteIfEquals(ZooKeeperWatcher zkw, final String content) {
-    if (content == null){
-      throw new IllegalArgumentException("Content must not be null");
-    }
-
-    try {
-      Stat stat = new Stat();
-      byte[] data = ZKUtil.getDataNoWatch(zkw, zkw.znodePaths.masterAddressZNode, stat);
-      ServerName sn = ProtobufUtil.parseServerNameFrom(data);
-      if (sn != null && content.equals(sn.toString())) {
-        return (ZKUtil.deleteNode(zkw, zkw.znodePaths.masterAddressZNode, stat.getVersion()));
-      }
-    } catch (KeeperException e) {
-      LOG.warn("Can't get or delete the master znode", e);
-    } catch (DeserializationException e) {
-      LOG.warn("Can't get or delete the master znode", e);
-    }
-
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
deleted file mode 100644
index cddde2f..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ /dev/null
@@ -1,629 +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.EOFException;
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.NoRouteToHostException;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.ipc.FailedServerException;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
-
-/**
- * Utility class to perform operation (get/wait for/verify/set/delete) on znode in ZooKeeper
- * which keeps hbase:meta region server location.
- *
- * Stateless class with a bunch of static methods. Doesn't manage resources passed in
- * (e.g. Connection, ZooKeeperWatcher etc).
- *
- * Meta region location is set by <code>RegionServerServices</code>.
- * This class doesn't use ZK watchers, rather accesses ZK directly.
- *
- * This class it stateless. The only reason it's not made a non-instantiable util class
- * with a collection of static methods is that it'd be rather hard to mock properly in tests.
- *
- * TODO: rewrite using RPC calls to master to find out about hbase:meta.
- */
-@InterfaceAudience.Private
-public class MetaTableLocator {
-  private static final Log LOG = LogFactory.getLog(MetaTableLocator.class);
-
-  // only needed to allow non-timeout infinite waits to stop when cluster shuts down
-  private volatile boolean stopped = false;
-
-  /**
-   * Checks if the meta region location is available.
-   * @return true if meta region location is available, false if not
-   */
-  public boolean isLocationAvailable(ZooKeeperWatcher zkw) {
-    return getMetaRegionLocation(zkw) != null;
-  }
-
-  /**
-   * @param zkw ZooKeeper watcher to be used
-   * @return meta table regions and their locations.
-   */
-  public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw) {
-    return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
-  }
-
-  /**
-   *
-   * @param zkw
-   * @param replicaId
-   * @return meta table regions and their locations.
-   */
-  public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw,
-      int replicaId) {
-    ServerName serverName = getMetaRegionLocation(zkw, replicaId);
-    List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
-    list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
-        RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
-    return list;
-  }
-
-  /**
-   * @param zkw ZooKeeper watcher to be used
-   * @return List of meta regions
-   */
-  public List<RegionInfo> getMetaRegions(ZooKeeperWatcher zkw) {
-    return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
-  }
-
-  /**
-   *
-   * @param zkw
-   * @param replicaId
-   * @return List of meta regions
-   */
-  public List<RegionInfo> getMetaRegions(ZooKeeperWatcher zkw, int replicaId) {
-    List<Pair<RegionInfo, ServerName>> result;
-    result = getMetaRegionsAndLocations(zkw, replicaId);
-    return getListOfRegionInfos(result);
-  }
-
-  private List<RegionInfo> getListOfRegionInfos(
-      final List<Pair<RegionInfo, ServerName>> pairs) {
-    if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
-    List<RegionInfo> result = new ArrayList<>(pairs.size());
-    for (Pair<RegionInfo, ServerName> pair: pairs) {
-      result.add(pair.getFirst());
-    }
-    return result;
-  }
-
-  /**
-   * Gets the meta region location, if available.  Does not block.
-   * @param zkw zookeeper connection to use
-   * @return server name or null if we failed to get the data.
-   */
-  public ServerName getMetaRegionLocation(final ZooKeeperWatcher zkw) {
-    try {
-      RegionState state = getMetaRegionState(zkw);
-      return state.isOpened() ? state.getServerName() : null;
-    } catch (KeeperException ke) {
-      return null;
-    }
-  }
-
-  /**
-   * Gets the meta region location, if available.  Does not block.
-   * @param zkw
-   * @param replicaId
-   * @return server name
-   */
-  public ServerName getMetaRegionLocation(final ZooKeeperWatcher zkw, int replicaId) {
-    try {
-      RegionState state = getMetaRegionState(zkw, replicaId);
-      return state.isOpened() ? state.getServerName() : null;
-    } catch (KeeperException ke) {
-      return null;
-    }
-  }
-
-  /**
-   * Gets the meta region location, if available, and waits for up to the
-   * specified timeout if not immediately available.
-   * Given the zookeeper notification could be delayed, we will try to
-   * get the latest data.
-   * @param zkw
-   * @param timeout maximum time to wait, in millis
-   * @return server name for server hosting meta region formatted as per
-   * {@link ServerName}, or null if none available
-   * @throws InterruptedException if interrupted while waiting
-   * @throws NotAllMetaRegionsOnlineException
-   */
-  public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, long timeout)
-  throws InterruptedException, NotAllMetaRegionsOnlineException {
-    return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
-  }
-
-  /**
-   * Gets the meta region location, if available, and waits for up to the
-   * specified timeout if not immediately available.
-   * Given the zookeeper notification could be delayed, we will try to
-   * get the latest data.
-   * @param zkw
-   * @param replicaId
-   * @param timeout maximum time to wait, in millis
-   * @return server name for server hosting meta region formatted as per
-   * {@link ServerName}, or null if none available
-   * @throws InterruptedException
-   * @throws NotAllMetaRegionsOnlineException
-   */
-  public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, int replicaId, long timeout)
-  throws InterruptedException, NotAllMetaRegionsOnlineException {
-    try {
-      if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) {
-        String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
-            + "There could be a mismatch with the one configured in the master.";
-        LOG.error(errorMsg);
-        throw new IllegalArgumentException(errorMsg);
-      }
-    } catch (KeeperException e) {
-      throw new IllegalStateException("KeeperException while trying to check baseZNode:", e);
-    }
-    ServerName sn = blockUntilAvailable(zkw, replicaId, timeout);
-
-    if (sn == null) {
-      throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
-    }
-
-    return sn;
-  }
-
-  /**
-   * Waits indefinitely for availability of <code>hbase:meta</code>.  Used during
-   * cluster startup.  Does not verify meta, just that something has been
-   * set up in zk.
-   * @see #waitMetaRegionLocation(org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher, long)
-   * @throws InterruptedException if interrupted while waiting
-   */
-  public void waitMetaRegionLocation(ZooKeeperWatcher zkw) throws InterruptedException {
-    long startTime = System.currentTimeMillis();
-    while (!stopped) {
-      try {
-        if (waitMetaRegionLocation(zkw, 100) != null) break;
-        long sleepTime = System.currentTimeMillis() - startTime;
-        // +1 in case sleepTime=0
-        if ((sleepTime + 1) % 10000 == 0) {
-          LOG.warn("Have been waiting for meta to be assigned for " + sleepTime + "ms");
-        }
-      } catch (NotAllMetaRegionsOnlineException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("hbase:meta still not available, sleeping and retrying." +
-            " Reason: " + e.getMessage());
-        }
-      }
-    }
-  }
-
-  /**
-   * Verify <code>hbase:meta</code> is deployed and accessible.
-   * @param hConnection
-   * @param zkw
-   * @param timeout How long to wait on zk for meta address (passed through to
-   * the internal call to {@link #getMetaServerConnection}.
-   * @return True if the <code>hbase:meta</code> location is healthy.
-   * @throws java.io.IOException
-   * @throws InterruptedException
-   */
-  public boolean verifyMetaRegionLocation(ClusterConnection hConnection,
-      ZooKeeperWatcher zkw, final long timeout)
-  throws InterruptedException, IOException {
-    return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID);
-  }
-
-  /**
-   * Verify <code>hbase:meta</code> is deployed and accessible.
-   * @param connection
-   * @param zkw
-   * @param timeout How long to wait on zk for meta address (passed through to
-   * @param replicaId
-   * @return True if the <code>hbase:meta</code> location is healthy.
-   * @throws InterruptedException
-   * @throws IOException
-   */
-  public boolean verifyMetaRegionLocation(ClusterConnection connection,
-      ZooKeeperWatcher zkw, final long timeout, int replicaId)
-  throws InterruptedException, IOException {
-    AdminProtos.AdminService.BlockingInterface service = null;
-    try {
-      service = getMetaServerConnection(connection, zkw, timeout, replicaId);
-    } catch (NotAllMetaRegionsOnlineException e) {
-      // Pass
-    } catch (ServerNotRunningYetException e) {
-      // Pass -- remote server is not up so can't be carrying root
-    } catch (UnknownHostException e) {
-      // Pass -- server name doesn't resolve so it can't be assigned anything.
-    } catch (RegionServerStoppedException e) {
-      // Pass -- server name sends us to a server that is dying or already dead.
-    }
-    return (service != null) && verifyRegionLocation(connection, service,
-            getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
-                RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId).getRegionName());
-  }
-
-  /**
-   * Verify we can connect to <code>hostingServer</code> and that its carrying
-   * <code>regionName</code>.
-   * @param hostingServer Interface to the server hosting <code>regionName</code>
-   * @param address The servername that goes with the <code>metaServer</code>
-   * Interface.  Used logging.
-   * @param regionName The regionname we are interested in.
-   * @return True if we were able to verify the region located at other side of
-   * the Interface.
-   * @throws IOException
-   */
-  // TODO: We should be able to get the ServerName from the AdminProtocol
-  // rather than have to pass it in.  Its made awkward by the fact that the
-  // HRI is likely a proxy against remote server so the getServerName needs
-  // to be fixed to go to a local method or to a cache before we can do this.
-  private boolean verifyRegionLocation(final ClusterConnection connection,
-      AdminService.BlockingInterface hostingServer, final ServerName address,
-      final byte [] regionName)
-  throws IOException {
-    if (hostingServer == null) {
-      LOG.info("Passed hostingServer is null");
-      return false;
-    }
-    Throwable t;
-    HBaseRpcController controller = connection.getRpcControllerFactory().newController();
-    try {
-      // Try and get regioninfo from the hosting server.
-      return ProtobufUtil.getRegionInfo(controller, hostingServer, regionName) != null;
-    } catch (ConnectException e) {
-      t = e;
-    } catch (RetriesExhaustedException e) {
-      t = e;
-    } catch (RemoteException e) {
-      IOException ioe = e.unwrapRemoteException();
-      t = ioe;
-    } catch (IOException e) {
-      Throwable cause = e.getCause();
-      if (cause != null && cause instanceof EOFException) {
-        t = cause;
-      } else if (cause != null && cause.getMessage() != null
-          && cause.getMessage().contains("Connection reset")) {
-        t = cause;
-      } else {
-        t = e;
-      }
-    }
-    LOG.info("Failed verification of " + Bytes.toStringBinary(regionName) +
-      " at address=" + address + ", exception=" + t.getMessage());
-    return false;
-  }
-
-  /**
-   * Gets a connection to the server hosting meta, as reported by ZooKeeper,
-   * waiting up to the specified timeout for availability.
-   * <p>WARNING: Does not retry.  Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
-   * @param connection
-   * @param zkw
-   * @param timeout How long to wait on meta location
-   * @param replicaId
-   * @return connection to server hosting meta
-   * @throws InterruptedException
-   * @throws NotAllMetaRegionsOnlineException if timed out waiting
-   * @throws IOException
-   */
-  private AdminService.BlockingInterface getMetaServerConnection(ClusterConnection connection,
-      ZooKeeperWatcher zkw, long timeout, int replicaId)
-  throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
-    return getCachedConnection(connection, waitMetaRegionLocation(zkw, replicaId, timeout));
-  }
-
-  /**
-   * @param sn ServerName to get a connection against.
-   * @return The AdminProtocol we got when we connected to <code>sn</code>
-   * May have come from cache, may not be good, may have been setup by this
-   * invocation, or may be null.
-   * @throws IOException
-   */
-  private static AdminService.BlockingInterface getCachedConnection(ClusterConnection connection,
-    ServerName sn)
-  throws IOException {
-    if (sn == null) {
-      return null;
-    }
-    AdminService.BlockingInterface service = null;
-    try {
-      service = connection.getAdmin(sn);
-    } catch (RetriesExhaustedException e) {
-      if (e.getCause() != null && e.getCause() instanceof ConnectException) {
-        // Catch this; presume it means the cached connection has gone bad.
-      } else {
-        throw e;
-      }
-    } catch (SocketTimeoutException e) {
-      LOG.debug("Timed out connecting to " + sn);
-    } catch (NoRouteToHostException e) {
-      LOG.debug("Connecting to " + sn, e);
-    } catch (SocketException e) {
-      LOG.debug("Exception connecting to " + sn);
-    } catch (UnknownHostException e) {
-      LOG.debug("Unknown host exception connecting to  " + sn);
-    } catch (FailedServerException e) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Server " + sn + " is in failed server list.");
-      }
-    } catch (IOException ioe) {
-      Throwable cause = ioe.getCause();
-      if (ioe instanceof ConnectException) {
-        // Catch. Connect refused.
-      } else if (cause != null && cause instanceof EOFException) {
-        // Catch. Other end disconnected us.
-      } else if (cause != null && cause.getMessage() != null &&
-        cause.getMessage().toLowerCase(Locale.ROOT).contains("connection reset")) {
-        // Catch. Connection reset.
-      } else {
-        throw ioe;
-      }
-
-    }
-    return service;
-  }
-
-  /**
-   * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
-   * specified server address.
-   * @param zookeeper zookeeper reference
-   * @param serverName The server hosting <code>hbase:meta</code>
-   * @param state The region transition state
-   * @throws KeeperException unexpected zookeeper exception
-   */
-  public static void setMetaLocation(ZooKeeperWatcher zookeeper,
-      ServerName serverName, RegionState.State state) throws KeeperException {
-    setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
-  }
-
-  /**
-   * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
-   * specified server address.
-   * @param zookeeper
-   * @param serverName
-   * @param replicaId
-   * @param state
-   * @throws KeeperException
-   */
-  public static void setMetaLocation(ZooKeeperWatcher zookeeper,
-      ServerName serverName, int replicaId, RegionState.State state) throws KeeperException {
-    if (serverName == null) {
-      LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
-      return;
-    }
-    LOG.info("Setting hbase:meta (replicaId=" + replicaId + ") location in ZooKeeper as " +
-      serverName);
-    // Make the MetaRegionServer pb and then get its bytes and save this as
-    // the znode content.
-    MetaRegionServer pbrsr = MetaRegionServer.newBuilder()
-      .setServer(ProtobufUtil.toServerName(serverName))
-      .setRpcVersion(HConstants.RPC_CURRENT_VERSION)
-      .setState(state.convert()).build();
-    byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
-    try {
-      ZKUtil.setData(zookeeper,
-          zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
-    } catch(KeeperException.NoNodeException nne) {
-      if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
-        LOG.debug("META region location doesn't exist, create it");
-      } else {
-        LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
-            ", create it");
-      }
-      ZKUtil.createAndWatch(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
-    }
-  }
-
-  /**
-   * Load the meta region state from the meta server ZNode.
-   */
-  public static RegionState getMetaRegionState(ZooKeeperWatcher zkw) throws KeeperException {
-    return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
-  }
-
-  /**
-   * Load the meta region state from the meta server ZNode.
-   * @param zkw
-   * @param replicaId
-   * @return regionstate
-   * @throws KeeperException
-   */
-  public static RegionState getMetaRegionState(ZooKeeperWatcher zkw, int replicaId)
-      throws KeeperException {
-    RegionState.State state = RegionState.State.OPEN;
-    ServerName serverName = null;
-    try {
-      byte[] data = ZKUtil.getData(zkw, zkw.znodePaths.getZNodeForReplica(replicaId));
-      if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
-        try {
-          int prefixLen = ProtobufUtil.lengthOfPBMagic();
-          ZooKeeperProtos.MetaRegionServer rl =
-            ZooKeeperProtos.MetaRegionServer.PARSER.parseFrom
-              (data, prefixLen, data.length - prefixLen);
-          if (rl.hasState()) {
-            state = RegionState.State.convert(rl.getState());
-          }
-          HBaseProtos.ServerName sn = rl.getServer();
-          serverName = ServerName.valueOf(
-            sn.getHostName(), sn.getPort(), sn.getStartCode());
-        } catch (InvalidProtocolBufferException e) {
-          throw new DeserializationException("Unable to parse meta region location");
-        }
-      } else {
-        // old style of meta region location?
-        serverName = ProtobufUtil.parseServerNameFrom(data);
-      }
-    } catch (DeserializationException e) {
-      throw ZKUtil.convert(e);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-    }
-    if (serverName == null) {
-      state = RegionState.State.OFFLINE;
-    }
-    return new RegionState(
-        RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId),
-      state, serverName);
-  }
-
-  /**
-   * Deletes the location of <code>hbase:meta</code> in ZooKeeper.
-   * @param zookeeper zookeeper reference
-   * @throws KeeperException unexpected zookeeper exception
-   */
-  public void deleteMetaLocation(ZooKeeperWatcher zookeeper)
-  throws KeeperException {
-    deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
-  }
-
-  public void deleteMetaLocation(ZooKeeperWatcher zookeeper, int replicaId)
-  throws KeeperException {
-    if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
-      LOG.info("Deleting hbase:meta region location in ZooKeeper");
-    } else {
-      LOG.info("Deleting hbase:meta for " + replicaId + " region location in ZooKeeper");
-    }
-    try {
-      // Just delete the node.  Don't need any watches.
-      ZKUtil.deleteNode(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId));
-    } catch(KeeperException.NoNodeException nne) {
-      // Has already been deleted
-    }
-  }
-  /**
-   * Wait until the primary meta region is available. Get the secondary
-   * locations as well but don't block for those.
-   * @param zkw
-   * @param timeout
-   * @param conf
-   * @return ServerName or null if we timed out.
-   * @throws InterruptedException
-   */
-  public List<ServerName> blockUntilAvailable(final ZooKeeperWatcher zkw,
-      final long timeout, Configuration conf)
-          throws InterruptedException {
-    int numReplicasConfigured = 1;
-
-    List<ServerName> servers = new ArrayList<>();
-    // Make the blocking call first so that we do the wait to know
-    // the znodes are all in place or timeout.
-    ServerName server = blockUntilAvailable(zkw, timeout);
-    if (server == null) return null;
-    servers.add(server);
-
-    try {
-      List<String> metaReplicaNodes = zkw.getMetaReplicaNodes();
-      numReplicasConfigured = metaReplicaNodes.size();
-    } catch (KeeperException e) {
-      LOG.warn("Got ZK exception " + e);
-    }
-    for (int replicaId = 1; replicaId < numReplicasConfigured; replicaId++) {
-      // return all replica locations for the meta
-      servers.add(getMetaRegionLocation(zkw, replicaId));
-    }
-    return servers;
-  }
-
-  /**
-   * Wait until the meta region is available and is not in transition.
-   * @param zkw zookeeper connection to use
-   * @param timeout maximum time to wait, in millis
-   * @return ServerName or null if we timed out.
-   * @throws InterruptedException
-   */
-  public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
-      final long timeout)
-  throws InterruptedException {
-    return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
-  }
-
-  /**
-   * Wait until the meta region is available and is not in transition.
-   * @param zkw
-   * @param replicaId
-   * @param timeout
-   * @return ServerName or null if we timed out.
-   * @throws InterruptedException
-   */
-  public ServerName blockUntilAvailable(final ZooKeeperWatcher zkw, int replicaId,
-      final long timeout)
-  throws InterruptedException {
-    if (timeout < 0) throw new IllegalArgumentException();
-    if (zkw == null) throw new IllegalArgumentException();
-    long startTime = System.currentTimeMillis();
-    ServerName sn = null;
-    while (true) {
-      sn = getMetaRegionLocation(zkw, replicaId);
-      if (sn != null || (System.currentTimeMillis() - startTime)
-          > timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
-        break;
-      }
-      Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);
-    }
-    return sn;
-  }
-
-  /**
-   * Stop working.
-   * Interrupts any ongoing waits.
-   */
-  public void stop() {
-    if (!stopped) {
-      LOG.debug("Stopping MetaTableLocator");
-      stopped = true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/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 5632031..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.yetus.audience.InterfaceAudience;
-
-import org.apache.hadoop.hbase.shaded.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);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
deleted file mode 100644
index da7d176..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
+++ /dev/null
@@ -1,53 +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.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-
-/**
- * Placeholder of a watcher which might be triggered before the instance is not yet created.
- * <p>
- * {@code ZooKeeper} starts its event thread within its constructor (and that is an anti-pattern),
- * and the watcher passed to the constructor might be called back by the event thread
- * before you get the instance of {@code ZooKeeper} from the constructor.
- * If your watcher calls methods of {@code ZooKeeper},
- * pass this placeholder to the constructor of the {@code ZooKeeper},
- * create your watcher using the instance of {@code ZooKeeper},
- * and then call the method {@code PendingWatcher.prepare}.
- */
-class PendingWatcher implements Watcher {
-  private final InstancePending<Watcher> pending = new InstancePending<>();
-
-  @Override
-  public void process(WatchedEvent event) {
-    pending.get().process(event);
-  }
-
-  /**
-   * Associates the substantial watcher of processing events.
-   * This method should be called once, and {@code watcher} should be non-null.
-   * This method is expected to call as soon as possible
-   * because the event processing, being invoked by the ZooKeeper event thread,
-   * is uninterruptibly blocked until this method is called.
-   */
-  void prepare(Watcher watcher) {
-    pending.prepare(watcher);
-  }
-}


[04/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
new file mode 100644
index 0000000..f6c7a2d
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -0,0 +1,629 @@
+/*
+ * 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.EOFException;
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.NoRouteToHostException;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.ipc.FailedServerException;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
+
+/**
+ * Utility class to perform operation (get/wait for/verify/set/delete) on znode in ZooKeeper
+ * which keeps hbase:meta region server location.
+ *
+ * Stateless class with a bunch of static methods. Doesn't manage resources passed in
+ * (e.g. Connection, ZKWatcher etc).
+ *
+ * Meta region location is set by <code>RegionServerServices</code>.
+ * This class doesn't use ZK watchers, rather accesses ZK directly.
+ *
+ * This class it stateless. The only reason it's not made a non-instantiable util class
+ * with a collection of static methods is that it'd be rather hard to mock properly in tests.
+ *
+ * TODO: rewrite using RPC calls to master to find out about hbase:meta.
+ */
+@InterfaceAudience.Private
+public class MetaTableLocator {
+  private static final Log LOG = LogFactory.getLog(MetaTableLocator.class);
+
+  // only needed to allow non-timeout infinite waits to stop when cluster shuts down
+  private volatile boolean stopped = false;
+
+  /**
+   * Checks if the meta region location is available.
+   * @return true if meta region location is available, false if not
+   */
+  public boolean isLocationAvailable(ZKWatcher zkw) {
+    return getMetaRegionLocation(zkw) != null;
+  }
+
+  /**
+   * @param zkw ZooKeeper watcher to be used
+   * @return meta table regions and their locations.
+   */
+  public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw) {
+    return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+  }
+
+  /**
+   *
+   * @param zkw
+   * @param replicaId
+   * @return meta table regions and their locations.
+   */
+  public List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw,
+      int replicaId) {
+    ServerName serverName = getMetaRegionLocation(zkw, replicaId);
+    List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
+    list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
+        RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
+    return list;
+  }
+
+  /**
+   * @param zkw ZooKeeper watcher to be used
+   * @return List of meta regions
+   */
+  public List<RegionInfo> getMetaRegions(ZKWatcher zkw) {
+    return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+  }
+
+  /**
+   *
+   * @param zkw
+   * @param replicaId
+   * @return List of meta regions
+   */
+  public List<RegionInfo> getMetaRegions(ZKWatcher zkw, int replicaId) {
+    List<Pair<RegionInfo, ServerName>> result;
+    result = getMetaRegionsAndLocations(zkw, replicaId);
+    return getListOfRegionInfos(result);
+  }
+
+  private List<RegionInfo> getListOfRegionInfos(
+      final List<Pair<RegionInfo, ServerName>> pairs) {
+    if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
+    List<RegionInfo> result = new ArrayList<>(pairs.size());
+    for (Pair<RegionInfo, ServerName> pair: pairs) {
+      result.add(pair.getFirst());
+    }
+    return result;
+  }
+
+  /**
+   * Gets the meta region location, if available.  Does not block.
+   * @param zkw zookeeper connection to use
+   * @return server name or null if we failed to get the data.
+   */
+  public ServerName getMetaRegionLocation(final ZKWatcher zkw) {
+    try {
+      RegionState state = getMetaRegionState(zkw);
+      return state.isOpened() ? state.getServerName() : null;
+    } catch (KeeperException ke) {
+      return null;
+    }
+  }
+
+  /**
+   * Gets the meta region location, if available.  Does not block.
+   * @param zkw
+   * @param replicaId
+   * @return server name
+   */
+  public ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) {
+    try {
+      RegionState state = getMetaRegionState(zkw, replicaId);
+      return state.isOpened() ? state.getServerName() : null;
+    } catch (KeeperException ke) {
+      return null;
+    }
+  }
+
+  /**
+   * Gets the meta region location, if available, and waits for up to the
+   * specified timeout if not immediately available.
+   * Given the zookeeper notification could be delayed, we will try to
+   * get the latest data.
+   * @param zkw
+   * @param timeout maximum time to wait, in millis
+   * @return server name for server hosting meta region formatted as per
+   * {@link ServerName}, or null if none available
+   * @throws InterruptedException if interrupted while waiting
+   * @throws NotAllMetaRegionsOnlineException
+   */
+  public ServerName waitMetaRegionLocation(ZKWatcher zkw, long timeout)
+  throws InterruptedException, NotAllMetaRegionsOnlineException {
+    return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
+  }
+
+  /**
+   * Gets the meta region location, if available, and waits for up to the
+   * specified timeout if not immediately available.
+   * Given the zookeeper notification could be delayed, we will try to
+   * get the latest data.
+   * @param zkw
+   * @param replicaId
+   * @param timeout maximum time to wait, in millis
+   * @return server name for server hosting meta region formatted as per
+   * {@link ServerName}, or null if none available
+   * @throws InterruptedException
+   * @throws NotAllMetaRegionsOnlineException
+   */
+  public ServerName waitMetaRegionLocation(ZKWatcher zkw, int replicaId, long timeout)
+  throws InterruptedException, NotAllMetaRegionsOnlineException {
+    try {
+      if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) {
+        String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
+            + "There could be a mismatch with the one configured in the master.";
+        LOG.error(errorMsg);
+        throw new IllegalArgumentException(errorMsg);
+      }
+    } catch (KeeperException e) {
+      throw new IllegalStateException("KeeperException while trying to check baseZNode:", e);
+    }
+    ServerName sn = blockUntilAvailable(zkw, replicaId, timeout);
+
+    if (sn == null) {
+      throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
+    }
+
+    return sn;
+  }
+
+  /**
+   * Waits indefinitely for availability of <code>hbase:meta</code>.  Used during
+   * cluster startup.  Does not verify meta, just that something has been
+   * set up in zk.
+   * @see #waitMetaRegionLocation(ZKWatcher, long)
+   * @throws InterruptedException if interrupted while waiting
+   */
+  public void waitMetaRegionLocation(ZKWatcher zkw) throws InterruptedException {
+    long startTime = System.currentTimeMillis();
+    while (!stopped) {
+      try {
+        if (waitMetaRegionLocation(zkw, 100) != null) break;
+        long sleepTime = System.currentTimeMillis() - startTime;
+        // +1 in case sleepTime=0
+        if ((sleepTime + 1) % 10000 == 0) {
+          LOG.warn("Have been waiting for meta to be assigned for " + sleepTime + "ms");
+        }
+      } catch (NotAllMetaRegionsOnlineException e) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("hbase:meta still not available, sleeping and retrying." +
+            " Reason: " + e.getMessage());
+        }
+      }
+    }
+  }
+
+  /**
+   * Verify <code>hbase:meta</code> is deployed and accessible.
+   * @param hConnection
+   * @param zkw
+   * @param timeout How long to wait on zk for meta address (passed through to
+   * the internal call to {@link #getMetaServerConnection}.
+   * @return True if the <code>hbase:meta</code> location is healthy.
+   * @throws java.io.IOException
+   * @throws InterruptedException
+   */
+  public boolean verifyMetaRegionLocation(ClusterConnection hConnection,
+                                          ZKWatcher zkw, final long timeout)
+  throws InterruptedException, IOException {
+    return verifyMetaRegionLocation(hConnection, zkw, timeout, RegionInfo.DEFAULT_REPLICA_ID);
+  }
+
+  /**
+   * Verify <code>hbase:meta</code> is deployed and accessible.
+   * @param connection
+   * @param zkw
+   * @param timeout How long to wait on zk for meta address (passed through to
+   * @param replicaId
+   * @return True if the <code>hbase:meta</code> location is healthy.
+   * @throws InterruptedException
+   * @throws IOException
+   */
+  public boolean verifyMetaRegionLocation(ClusterConnection connection,
+                                          ZKWatcher zkw, final long timeout, int replicaId)
+  throws InterruptedException, IOException {
+    AdminProtos.AdminService.BlockingInterface service = null;
+    try {
+      service = getMetaServerConnection(connection, zkw, timeout, replicaId);
+    } catch (NotAllMetaRegionsOnlineException e) {
+      // Pass
+    } catch (ServerNotRunningYetException e) {
+      // Pass -- remote server is not up so can't be carrying root
+    } catch (UnknownHostException e) {
+      // Pass -- server name doesn't resolve so it can't be assigned anything.
+    } catch (RegionServerStoppedException e) {
+      // Pass -- server name sends us to a server that is dying or already dead.
+    }
+    return (service != null) && verifyRegionLocation(connection, service,
+            getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
+                RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId).getRegionName());
+  }
+
+  /**
+   * Verify we can connect to <code>hostingServer</code> and that its carrying
+   * <code>regionName</code>.
+   * @param hostingServer Interface to the server hosting <code>regionName</code>
+   * @param address The servername that goes with the <code>metaServer</code>
+   * Interface.  Used logging.
+   * @param regionName The regionname we are interested in.
+   * @return True if we were able to verify the region located at other side of
+   * the Interface.
+   * @throws IOException
+   */
+  // TODO: We should be able to get the ServerName from the AdminProtocol
+  // rather than have to pass it in.  Its made awkward by the fact that the
+  // HRI is likely a proxy against remote server so the getServerName needs
+  // to be fixed to go to a local method or to a cache before we can do this.
+  private boolean verifyRegionLocation(final ClusterConnection connection,
+      AdminService.BlockingInterface hostingServer, final ServerName address,
+      final byte [] regionName)
+  throws IOException {
+    if (hostingServer == null) {
+      LOG.info("Passed hostingServer is null");
+      return false;
+    }
+    Throwable t;
+    HBaseRpcController controller = connection.getRpcControllerFactory().newController();
+    try {
+      // Try and get regioninfo from the hosting server.
+      return ProtobufUtil.getRegionInfo(controller, hostingServer, regionName) != null;
+    } catch (ConnectException e) {
+      t = e;
+    } catch (RetriesExhaustedException e) {
+      t = e;
+    } catch (RemoteException e) {
+      IOException ioe = e.unwrapRemoteException();
+      t = ioe;
+    } catch (IOException e) {
+      Throwable cause = e.getCause();
+      if (cause != null && cause instanceof EOFException) {
+        t = cause;
+      } else if (cause != null && cause.getMessage() != null
+          && cause.getMessage().contains("Connection reset")) {
+        t = cause;
+      } else {
+        t = e;
+      }
+    }
+    LOG.info("Failed verification of " + Bytes.toStringBinary(regionName) +
+      " at address=" + address + ", exception=" + t.getMessage());
+    return false;
+  }
+
+  /**
+   * Gets a connection to the server hosting meta, as reported by ZooKeeper,
+   * waiting up to the specified timeout for availability.
+   * <p>WARNING: Does not retry.  Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
+   * @param connection
+   * @param zkw
+   * @param timeout How long to wait on meta location
+   * @param replicaId
+   * @return connection to server hosting meta
+   * @throws InterruptedException
+   * @throws NotAllMetaRegionsOnlineException if timed out waiting
+   * @throws IOException
+   */
+  private AdminService.BlockingInterface getMetaServerConnection(ClusterConnection connection,
+                                                                 ZKWatcher zkw, long timeout, int replicaId)
+  throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
+    return getCachedConnection(connection, waitMetaRegionLocation(zkw, replicaId, timeout));
+  }
+
+  /**
+   * @param sn ServerName to get a connection against.
+   * @return The AdminProtocol we got when we connected to <code>sn</code>
+   * May have come from cache, may not be good, may have been setup by this
+   * invocation, or may be null.
+   * @throws IOException
+   */
+  private static AdminService.BlockingInterface getCachedConnection(ClusterConnection connection,
+    ServerName sn)
+  throws IOException {
+    if (sn == null) {
+      return null;
+    }
+    AdminService.BlockingInterface service = null;
+    try {
+      service = connection.getAdmin(sn);
+    } catch (RetriesExhaustedException e) {
+      if (e.getCause() != null && e.getCause() instanceof ConnectException) {
+        // Catch this; presume it means the cached connection has gone bad.
+      } else {
+        throw e;
+      }
+    } catch (SocketTimeoutException e) {
+      LOG.debug("Timed out connecting to " + sn);
+    } catch (NoRouteToHostException e) {
+      LOG.debug("Connecting to " + sn, e);
+    } catch (SocketException e) {
+      LOG.debug("Exception connecting to " + sn);
+    } catch (UnknownHostException e) {
+      LOG.debug("Unknown host exception connecting to  " + sn);
+    } catch (FailedServerException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Server " + sn + " is in failed server list.");
+      }
+    } catch (IOException ioe) {
+      Throwable cause = ioe.getCause();
+      if (ioe instanceof ConnectException) {
+        // Catch. Connect refused.
+      } else if (cause != null && cause instanceof EOFException) {
+        // Catch. Other end disconnected us.
+      } else if (cause != null && cause.getMessage() != null &&
+        cause.getMessage().toLowerCase(Locale.ROOT).contains("connection reset")) {
+        // Catch. Connection reset.
+      } else {
+        throw ioe;
+      }
+
+    }
+    return service;
+  }
+
+  /**
+   * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
+   * specified server address.
+   * @param zookeeper zookeeper reference
+   * @param serverName The server hosting <code>hbase:meta</code>
+   * @param state The region transition state
+   * @throws KeeperException unexpected zookeeper exception
+   */
+  public static void setMetaLocation(ZKWatcher zookeeper,
+      ServerName serverName, RegionState.State state) throws KeeperException {
+    setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
+  }
+
+  /**
+   * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
+   * specified server address.
+   * @param zookeeper
+   * @param serverName
+   * @param replicaId
+   * @param state
+   * @throws KeeperException
+   */
+  public static void setMetaLocation(ZKWatcher zookeeper,
+      ServerName serverName, int replicaId, RegionState.State state) throws KeeperException {
+    if (serverName == null) {
+      LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
+      return;
+    }
+    LOG.info("Setting hbase:meta (replicaId=" + replicaId + ") location in ZooKeeper as " +
+      serverName);
+    // Make the MetaRegionServer pb and then get its bytes and save this as
+    // the znode content.
+    MetaRegionServer pbrsr = MetaRegionServer.newBuilder()
+      .setServer(ProtobufUtil.toServerName(serverName))
+      .setRpcVersion(HConstants.RPC_CURRENT_VERSION)
+      .setState(state.convert()).build();
+    byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
+    try {
+      ZKUtil.setData(zookeeper,
+          zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
+    } catch(KeeperException.NoNodeException nne) {
+      if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
+        LOG.debug("META region location doesn't exist, create it");
+      } else {
+        LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
+            ", create it");
+      }
+      ZKUtil.createAndWatch(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
+    }
+  }
+
+  /**
+   * Load the meta region state from the meta server ZNode.
+   */
+  public static RegionState getMetaRegionState(ZKWatcher zkw) throws KeeperException {
+    return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+  }
+
+  /**
+   * Load the meta region state from the meta server ZNode.
+   * @param zkw
+   * @param replicaId
+   * @return regionstate
+   * @throws KeeperException
+   */
+  public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId)
+      throws KeeperException {
+    RegionState.State state = RegionState.State.OPEN;
+    ServerName serverName = null;
+    try {
+      byte[] data = ZKUtil.getData(zkw, zkw.znodePaths.getZNodeForReplica(replicaId));
+      if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
+        try {
+          int prefixLen = ProtobufUtil.lengthOfPBMagic();
+          ZooKeeperProtos.MetaRegionServer rl =
+            ZooKeeperProtos.MetaRegionServer.PARSER.parseFrom
+              (data, prefixLen, data.length - prefixLen);
+          if (rl.hasState()) {
+            state = RegionState.State.convert(rl.getState());
+          }
+          HBaseProtos.ServerName sn = rl.getServer();
+          serverName = ServerName.valueOf(
+            sn.getHostName(), sn.getPort(), sn.getStartCode());
+        } catch (InvalidProtocolBufferException e) {
+          throw new DeserializationException("Unable to parse meta region location");
+        }
+      } else {
+        // old style of meta region location?
+        serverName = ProtobufUtil.parseServerNameFrom(data);
+      }
+    } catch (DeserializationException e) {
+      throw ZKUtil.convert(e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+    if (serverName == null) {
+      state = RegionState.State.OFFLINE;
+    }
+    return new RegionState(
+        RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId),
+      state, serverName);
+  }
+
+  /**
+   * Deletes the location of <code>hbase:meta</code> in ZooKeeper.
+   * @param zookeeper zookeeper reference
+   * @throws KeeperException unexpected zookeeper exception
+   */
+  public void deleteMetaLocation(ZKWatcher zookeeper)
+  throws KeeperException {
+    deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
+  }
+
+  public void deleteMetaLocation(ZKWatcher zookeeper, int replicaId)
+  throws KeeperException {
+    if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
+      LOG.info("Deleting hbase:meta region location in ZooKeeper");
+    } else {
+      LOG.info("Deleting hbase:meta for " + replicaId + " region location in ZooKeeper");
+    }
+    try {
+      // Just delete the node.  Don't need any watches.
+      ZKUtil.deleteNode(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId));
+    } catch(KeeperException.NoNodeException nne) {
+      // Has already been deleted
+    }
+  }
+  /**
+   * Wait until the primary meta region is available. Get the secondary
+   * locations as well but don't block for those.
+   * @param zkw
+   * @param timeout
+   * @param conf
+   * @return ServerName or null if we timed out.
+   * @throws InterruptedException
+   */
+  public List<ServerName> blockUntilAvailable(final ZKWatcher zkw,
+      final long timeout, Configuration conf)
+          throws InterruptedException {
+    int numReplicasConfigured = 1;
+
+    List<ServerName> servers = new ArrayList<>();
+    // Make the blocking call first so that we do the wait to know
+    // the znodes are all in place or timeout.
+    ServerName server = blockUntilAvailable(zkw, timeout);
+    if (server == null) return null;
+    servers.add(server);
+
+    try {
+      List<String> metaReplicaNodes = zkw.getMetaReplicaNodes();
+      numReplicasConfigured = metaReplicaNodes.size();
+    } catch (KeeperException e) {
+      LOG.warn("Got ZK exception " + e);
+    }
+    for (int replicaId = 1; replicaId < numReplicasConfigured; replicaId++) {
+      // return all replica locations for the meta
+      servers.add(getMetaRegionLocation(zkw, replicaId));
+    }
+    return servers;
+  }
+
+  /**
+   * Wait until the meta region is available and is not in transition.
+   * @param zkw zookeeper connection to use
+   * @param timeout maximum time to wait, in millis
+   * @return ServerName or null if we timed out.
+   * @throws InterruptedException
+   */
+  public ServerName blockUntilAvailable(final ZKWatcher zkw,
+      final long timeout)
+  throws InterruptedException {
+    return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
+  }
+
+  /**
+   * Wait until the meta region is available and is not in transition.
+   * @param zkw
+   * @param replicaId
+   * @param timeout
+   * @return ServerName or null if we timed out.
+   * @throws InterruptedException
+   */
+  public ServerName blockUntilAvailable(final ZKWatcher zkw, int replicaId,
+                                        final long timeout)
+  throws InterruptedException {
+    if (timeout < 0) throw new IllegalArgumentException();
+    if (zkw == null) throw new IllegalArgumentException();
+    long startTime = System.currentTimeMillis();
+    ServerName sn = null;
+    while (true) {
+      sn = getMetaRegionLocation(zkw, replicaId);
+      if (sn != null || (System.currentTimeMillis() - startTime)
+          > timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
+        break;
+      }
+      Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);
+    }
+    return sn;
+  }
+
+  /**
+   * Stop working.
+   * Interrupts any ongoing waits.
+   */
+  public void stop() {
+    if (!stopped) {
+      LOG.debug("Stopping MetaTableLocator");
+      stopped = true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
new file mode 100644
index 0000000..ef643bf
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
@@ -0,0 +1,472 @@
+/*
+ *
+ * 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/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
new file mode 100644
index 0000000..da7d176
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+/**
+ * Placeholder of a watcher which might be triggered before the instance is not yet created.
+ * <p>
+ * {@code ZooKeeper} starts its event thread within its constructor (and that is an anti-pattern),
+ * and the watcher passed to the constructor might be called back by the event thread
+ * before you get the instance of {@code ZooKeeper} from the constructor.
+ * If your watcher calls methods of {@code ZooKeeper},
+ * pass this placeholder to the constructor of the {@code ZooKeeper},
+ * create your watcher using the instance of {@code ZooKeeper},
+ * and then call the method {@code PendingWatcher.prepare}.
+ */
+class PendingWatcher implements Watcher {
+  private final InstancePending<Watcher> pending = new InstancePending<>();
+
+  @Override
+  public void process(WatchedEvent event) {
+    pending.get().process(event);
+  }
+
+  /**
+   * Associates the substantial watcher of processing events.
+   * This method should be called once, and {@code watcher} should be non-null.
+   * This method is expected to call as soon as possible
+   * because the event processing, being invoked by the ZooKeeper event thread,
+   * is uninterruptibly blocked until this method is called.
+   */
+  void prepare(Watcher watcher) {
+    pending.prepare(watcher);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
new file mode 100644
index 0000000..d6c11af
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -0,0 +1,810 @@
+/*
+ *
+ * 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.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.htrace.core.TraceScope;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooKeeper.States;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.CreateRequest;
+import org.apache.zookeeper.proto.SetDataRequest;
+
+/**
+ * A zookeeper that can handle 'recoverable' errors.
+ * To handle recoverable errors, developers need to realize that there are two
+ * classes of requests: idempotent and non-idempotent requests. Read requests
+ * and unconditional sets and deletes are examples of idempotent requests, they
+ * can be reissued with the same results.
+ * (Although, the delete may throw a NoNodeException on reissue its effect on
+ * the ZooKeeper state is the same.) Non-idempotent requests need special
+ * handling, application and library writers need to keep in mind that they may
+ * need to encode information in the data or name of znodes to detect
+ * retries. A simple example is a create that uses a sequence flag.
+ * If a process issues a create("/x-", ..., SEQUENCE) and gets a connection
+ * loss exception, that process will reissue another
+ * create("/x-", ..., SEQUENCE) and get back x-111. When the process does a
+ * getChildren("/"), it sees x-1,x-30,x-109,x-110,x-111, now it could be
+ * that x-109 was the result of the previous create, so the process actually
+ * owns both x-109 and x-111. An easy way around this is to use "x-process id-"
+ * when doing the create. If the process is using an id of 352, before reissuing
+ * the create it will do a getChildren("/") and see "x-222-1", "x-542-30",
+ * "x-352-109", x-333-110". The process will know that the original create
+ * succeeded an the znode it created is "x-352-109".
+ * @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling"
+ */
+@InterfaceAudience.Private
+public class RecoverableZooKeeper {
+  private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
+  // the actual ZooKeeper client instance
+  private ZooKeeper zk;
+  private final RetryCounterFactory retryCounterFactory;
+  // An identifier of this process in the cluster
+  private final String identifier;
+  private final byte[] id;
+  private Watcher watcher;
+  private int sessionTimeout;
+  private String quorumServers;
+  private final ZKMetricsListener metrics;
+
+  public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
+      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime)
+  throws IOException {
+    this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis, maxSleepTime,
+        null);
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
+      justification="None. Its always been this way.")
+  public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
+      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier)
+  throws IOException {
+    // TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as we should.
+    this.retryCounterFactory =
+      new RetryCounterFactory(maxRetries+1, retryIntervalMillis, maxSleepTime);
+
+    if (identifier == null || identifier.length() == 0) {
+      // the identifier = processID@hostName
+      identifier = ManagementFactory.getRuntimeMXBean().getName();
+    }
+    LOG.info("Process identifier=" + identifier +
+      " connecting to ZooKeeper ensemble=" + quorumServers);
+    this.identifier = identifier;
+    this.id = Bytes.toBytes(identifier);
+
+    this.watcher = watcher;
+    this.sessionTimeout = sessionTimeout;
+    this.quorumServers = quorumServers;
+    this.metrics = new ZKMetrics();
+    try {checkZk();} catch (Exception x) {/* ignore */}
+  }
+
+  /**
+   * Try to create a ZooKeeper connection. Turns any exception encountered into a
+   * KeeperException.OperationTimeoutException so it can retried.
+   * @return The created ZooKeeper connection object
+   * @throws KeeperException
+   */
+  protected synchronized ZooKeeper checkZk() throws KeeperException {
+    if (this.zk == null) {
+      try {
+        this.zk = new ZooKeeper(quorumServers, sessionTimeout, watcher);
+      } catch (IOException ex) {
+        LOG.warn("Unable to create ZooKeeper Connection", ex);
+        throw new KeeperException.OperationTimeoutException();
+      }
+    }
+    return zk;
+  }
+
+  public synchronized void reconnectAfterExpiration()
+        throws IOException, KeeperException, InterruptedException {
+    if (zk != null) {
+      LOG.info("Closing dead ZooKeeper connection, session" +
+        " was: 0x"+Long.toHexString(zk.getSessionId()));
+      zk.close();
+      // reset the ZooKeeper connection
+      zk = null;
+    }
+    checkZk();
+    LOG.info("Recreated a ZooKeeper, session" +
+      " is: 0x"+Long.toHexString(zk.getSessionId()));
+  }
+
+  /**
+   * delete is an idempotent operation. Retry before throwing exception.
+   * This function will not throw NoNodeException if the path does not
+   * exist.
+   */
+  public void delete(String path, int version) throws InterruptedException, KeeperException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.delete")) {
+      RetryCounter retryCounter = retryCounterFactory.create();
+      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) {
+                LOG.debug("Node " + path + " already deleted. Assuming a " +
+                    "previous attempt succeeded.");
+                return;
+              }
+              LOG.debug("Node " + path + " already deleted, retry=" + isRetry);
+              throw e;
+
+            case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "delete");
+              break;
+            case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
+              retryOrThrow(retryCounter, e, "delete");
+              break;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+        isRetry = true;
+      }
+    }
+  }
+
+  /**
+   * exists is an idempotent operation. Retry before throwing exception
+   * @return A Stat instance
+   */
+  public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
+      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;
+        } 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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  /**
+   * exists is an idempotent operation. Retry before throwing exception
+   * @return A Stat instance
+   */
+  public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
+      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;
+        } 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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
+      String opName) throws KeeperException {
+    if (!retryCounter.shouldRetry()) {
+      LOG.error("ZooKeeper " + opName + " failed after "
+        + retryCounter.getMaxAttempts() + " attempts");
+      throw e;
+    }
+    LOG.debug("Retry, connectivity issue (JVM Pause?); quorum=" + quorumServers + "," +
+        "exception=" + e);
+  }
+
+  /**
+   * getChildren is an idempotent operation. Retry before throwing exception
+   * @return List of children znodes
+   */
+  public List<String> getChildren(String path, Watcher watcher)
+    throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
+      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;
+        } 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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  /**
+   * getChildren is an idempotent operation. Retry before throwing exception
+   * @return List of children znodes
+   */
+  public List<String> getChildren(String path, boolean watch)
+  throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
+      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;
+        } 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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  /**
+   * getData is an idempotent operation. Retry before throwing exception
+   * @return Data
+   */
+  public byte[] getData(String path, Watcher watcher, Stat stat)
+  throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
+      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 ZKMetadata.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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  /**
+   * getData is an idempotent operation. Retry before throwing exception
+   * @return Data
+   */
+  public byte[] getData(String path, boolean watch, Stat stat)
+  throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
+      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 ZKMetadata.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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  /**
+   * setData is NOT an idempotent operation. Retry may cause BadVersion Exception
+   * Adding an identifier field into the data to check whether
+   * badversion is caused by the result of previous correctly setData
+   * @return Stat instance
+   */
+  public Stat setData(String path, byte[] data, int version)
+  throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setData")) {
+      RetryCounter retryCounter = retryCounterFactory.create();
+      byte[] newData = ZKMetadata.appendMetaData(id, 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;
+        } 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:
+              if (isRetry) {
+                // 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;
+                }
+              }
+            // throw other exceptions and verified bad version exceptions
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+        isRetry = true;
+      }
+    }
+  }
+
+  /**
+   * getAcl is an idempotent operation. Retry before throwing exception
+   * @return list of ACLs
+   */
+  public List<ACL> getAcl(String path, Stat stat)
+  throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getAcl")) {
+      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;
+        } 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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  /**
+   * setAcl is an idempotent operation. Retry before throwing exception
+   * @return list of ACLs
+   */
+  public Stat setAcl(String path, List<ACL> acls, int version)
+  throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setAcl")) {
+      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;
+        } 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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  /**
+   * <p>
+   * NONSEQUENTIAL create is idempotent operation.
+   * Retry before throwing exceptions.
+   * But this function will not throw the NodeExist exception back to the
+   * application.
+   * </p>
+   * <p>
+   * But SEQUENTIAL is NOT idempotent operation. It is necessary to add
+   * identifier to the path to verify, whether the previous one is successful
+   * or not.
+   * </p>
+   *
+   * @return Path
+   */
+  public String create(String path, byte[] data, List<ACL> acl,
+      CreateMode createMode)
+  throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.create")) {
+      byte[] newData = ZKMetadata.appendMetaData(id, data);
+      switch (createMode) {
+        case EPHEMERAL:
+        case PERSISTENT:
+          return createNonSequential(path, newData, acl, createMode);
+
+        case EPHEMERAL_SEQUENTIAL:
+        case PERSISTENT_SEQUENTIAL:
+          return createSequential(path, newData, acl, createMode);
+
+        default:
+          throw new IllegalArgumentException("Unrecognized CreateMode: " +
+              createMode);
+      }
+    }
+  }
+
+  private String createNonSequential(String path, byte[] data, List<ACL> acl,
+      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;
+      } 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
+                return path;
+              }
+              LOG.error("Node " + path + " already exists with " +
+                  Bytes.toStringBinary(currentData) + ", could not write " +
+                  Bytes.toStringBinary(data));
+              throw e;
+            }
+            LOG.debug("Node " + path + " already exists");
+            throw e;
+
+          case CONNECTIONLOSS:
+            this.metrics.registerConnectionLossException();
+            retryOrThrow(retryCounter, e, "create");
+            break;
+          case OPERATIONTIMEOUT:
+            this.metrics.registerOperationTimeoutException();
+            retryOrThrow(retryCounter, e, "create");
+            break;
+
+          default:
+            throw e;
+        }
+      }
+      retryCounter.sleepUntilNextRetry();
+      isRetry = true;
+    }
+  }
+
+  private String createSequential(String path, byte[] data,
+      List<ACL> acl, CreateMode createMode)
+  throws KeeperException, InterruptedException {
+    RetryCounter retryCounter = retryCounterFactory.create();
+    boolean first = true;
+    String newPath = path+this.identifier;
+    while (true) {
+      try {
+        if (!first) {
+          // Check if we succeeded on a previous attempt
+          String previousResult = findPreviousSequentialNode(newPath);
+          if (previousResult != null) {
+            return previousResult;
+          }
+        }
+        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;
+      } 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;
+
+          default:
+            throw e;
+        }
+      }
+      retryCounter.sleepUntilNextRetry();
+    }
+  }
+  /**
+   * Convert Iterable of {@link org.apache.zookeeper.Op} we got into the ZooKeeper.Op
+   * instances to actually pass to multi (need to do this in order to appendMetaData).
+   */
+  private Iterable<Op> prepareZKMulti(Iterable<Op> ops)
+  throws UnsupportedOperationException {
+    if(ops == null) return null;
+
+    List<Op> preparedOps = new LinkedList<>();
+    for (Op op : ops) {
+      if (op.getType() == ZooDefs.OpCode.create) {
+        CreateRequest create = (CreateRequest)op.toRequestRecord();
+        preparedOps.add(Op.create(create.getPath(), ZKMetadata.appendMetaData(id, create.getData()),
+          create.getAcl(), create.getFlags()));
+      } else if (op.getType() == ZooDefs.OpCode.delete) {
+        // no need to appendMetaData for delete
+        preparedOps.add(op);
+      } else if (op.getType() == ZooDefs.OpCode.setData) {
+        SetDataRequest setData = (SetDataRequest)op.toRequestRecord();
+        preparedOps.add(Op.setData(setData.getPath(), ZKMetadata.appendMetaData(id, setData.getData()),
+          setData.getVersion()));
+      } else {
+        throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName());
+      }
+    }
+    return preparedOps;
+  }
+
+  /**
+   * Run multiple operations in a transactional manner. Retry before throwing exception
+   */
+  public List<OpResult> multi(Iterable<Op> ops)
+  throws KeeperException, InterruptedException {
+    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.multi")) {
+      RetryCounter retryCounter = retryCounterFactory.create();
+      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;
+        } 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;
+
+            default:
+              throw e;
+          }
+        }
+        retryCounter.sleepUntilNextRetry();
+      }
+    }
+  }
+
+  private String findPreviousSequentialNode(String path)
+    throws KeeperException, InterruptedException {
+    int lastSlashIdx = path.lastIndexOf('/');
+    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;
+      }
+    }
+    return null;
+  }
+
+  public synchronized long getSessionId() {
+    return zk == null ? -1 : zk.getSessionId();
+  }
+
+  public synchronized void close() throws InterruptedException {
+    if (zk != null) zk.close();
+  }
+
+  public synchronized States getState() {
+    return zk == null ? null : zk.getState();
+  }
+
+  public synchronized ZooKeeper getZooKeeper() {
+    return zk;
+  }
+
+  public synchronized byte[] getSessionPasswd() {
+    return zk == null ? null : zk.getSessionPasswd();
+  }
+
+  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));
+  }
+
+  /**
+   * Filters the given node list by the given prefixes.
+   * This method is all-inclusive--if any element in the node list starts
+   * with any of the given prefixes, then it is included in the result.
+   *
+   * @param nodes the nodes to filter
+   * @param prefixes the prefixes to include in the result
+   * @return list of every element that starts with one of the prefixes
+   */
+  private static List<String> filterByPrefix(List<String> nodes,
+      String... prefixes) {
+    List<String> lockChildren = new ArrayList<>();
+    for (String child : nodes){
+      for (String prefix : prefixes){
+        if (child.startsWith(prefix)){
+          lockChildren.add(child);
+          break;
+        }
+      }
+    }
+    return lockChildren;
+  }
+
+  public String getIdentifier() {
+    return identifier;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
new file mode 100644
index 0000000..93545ee
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
@@ -0,0 +1,94 @@
+/*
+ * 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 ZKNodeTracker {
+  private static final Log LOG = LogFactory.getLog(RegionNormalizerTracker.class);
+
+  public RegionNormalizerTracker(ZKWatcher 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/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java
new file mode 100644
index 0000000..4150f54
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAclReset.java
@@ -0,0 +1,116 @@
+/*
+ *
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+
+/**
+ * You may add the jaas.conf option
+ *    -Djava.security.auth.login.config=/PATH/jaas.conf
+ *
+ * You may also specify -D to set options
+ *    "hbase.zookeeper.quorum"    (it should be in hbase-site.xml)
+ *    "zookeeper.znode.parent"    (it should be in hbase-site.xml)
+ *
+ * Use -set-acls to set the ACLs, no option to erase ACLs
+ */
+@InterfaceAudience.Private
+public class ZKAclReset extends Configured implements Tool {
+  private static final Log LOG = LogFactory.getLog(ZKAclReset.class);
+
+  private static void resetAcls(final ZKWatcher zkw, final String znode,
+                                final boolean eraseAcls) throws Exception {
+    List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
+    if (children != null) {
+      for (String child: children) {
+        resetAcls(zkw, ZNodePaths.joinZNode(znode, child), eraseAcls);
+      }
+    }
+
+    ZooKeeper zk = zkw.getRecoverableZooKeeper().getZooKeeper();
+    if (eraseAcls) {
+      LOG.info(" - erase ACLs for " + znode);
+      zk.setACL(znode, ZooDefs.Ids.OPEN_ACL_UNSAFE, -1);
+    } else {
+      LOG.info(" - set ACLs for " + znode);
+      zk.setACL(znode, ZKUtil.createACL(zkw, znode, true), -1);
+    }
+  }
+
+  private static void resetAcls(final Configuration conf, boolean eraseAcls)
+      throws Exception {
+    ZKWatcher zkw = new ZKWatcher(conf, "ZKAclReset", null);
+    try {
+      LOG.info((eraseAcls ? "Erase" : "Set") + " HBase ACLs for " +
+                zkw.getQuorum() + " " + zkw.znodePaths.baseZNode);
+      resetAcls(zkw, zkw.znodePaths.baseZNode, eraseAcls);
+    } finally {
+      zkw.close();
+    }
+  }
+
+  private void printUsageAndExit() {
+    System.err.printf("Usage: hbase %s [options]%n", getClass().getName());
+    System.err.println(" where [options] are:");
+    System.err.println("  -h|-help                Show this help and exit.");
+    System.err.println("  -set-acls               Setup the hbase znode ACLs for a secure cluster");
+    System.err.println();
+    System.err.println("Examples:");
+    System.err.println("  To reset the ACLs to the unsecure cluster behavior:");
+    System.err.println("  hbase " + getClass().getName());
+    System.err.println();
+    System.err.println("  To reset the ACLs to the secure cluster behavior:");
+    System.err.println("  hbase " + getClass().getName() + " -set-acls");
+    System.exit(1);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    boolean eraseAcls = true;
+
+    for (int i = 0; i < args.length; ++i) {
+      if (args[i].equals("-help")) {
+        printUsageAndExit();
+      } else if (args[i].equals("-set-acls")) {
+        eraseAcls = false;
+      } else {
+        printUsageAndExit();
+      }
+    }
+
+    resetAcls(getConf(), eraseAcls);
+    return(0);
+  }
+
+  public static void main(String[] args) throws Exception {
+    System.exit(ToolRunner.run(HBaseConfiguration.create(), new ZKAclReset(), args));
+  }
+}


[07/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 3b3d568..bb7eca2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
@@ -133,7 +134,6 @@ import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -2752,7 +2752,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     }
   }
 
-  public void expireSession(ZooKeeperWatcher nodeZK) throws Exception {
+  public void expireSession(ZKWatcher nodeZK) throws Exception {
    expireSession(nodeZK, false);
   }
 
@@ -2767,7 +2767,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param checkStatus - true to check if we can create a Table with the
    *                    current configuration.
    */
-  public void expireSession(ZooKeeperWatcher nodeZK, boolean checkStatus)
+  public void expireSession(ZKWatcher nodeZK, boolean checkStatus)
     throws Exception {
     Configuration c = new Configuration(this.conf);
     String quorumServers = ZKConfig.getZKQuorumServersString(c);
@@ -2882,18 +2882,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   private HBaseAdmin hbaseAdmin = null;
 
   /**
-   * Returns a ZooKeeperWatcher instance.
+   * Returns a ZKWatcher instance.
    * This instance is shared between HBaseTestingUtility instance users.
    * Don't close it, it will be closed automatically when the
    * cluster shutdowns
    *
-   * @return The ZooKeeperWatcher instance.
+   * @return The ZKWatcher instance.
    * @throws IOException
    */
-  public synchronized ZooKeeperWatcher getZooKeeperWatcher()
+  public synchronized ZKWatcher getZooKeeperWatcher()
     throws IOException {
     if (zooKeeperWatcher == null) {
-      zooKeeperWatcher = new ZooKeeperWatcher(conf, "testing utility",
+      zooKeeperWatcher = new ZKWatcher(conf, "testing utility",
         new Abortable() {
         @Override public void abort(String why, Throwable e) {
           throw new RuntimeException("Unexpected abort in HBaseTestingUtility:"+why, e);
@@ -2903,7 +2903,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     }
     return zooKeeperWatcher;
   }
-  private ZooKeeperWatcher zooKeeperWatcher;
+  private ZKWatcher zooKeeperWatcher;
 
 
 
@@ -3508,13 +3508,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   }
 
   /**
-   * Gets a ZooKeeperWatcher.
+   * Gets a ZKWatcher.
    * @param TEST_UTIL
    */
-  public static ZooKeeperWatcher getZooKeeperWatcher(
+  public static ZKWatcher getZooKeeperWatcher(
       HBaseTestingUtility TEST_UTIL) throws ZooKeeperConnectionException,
       IOException {
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+    ZKWatcher zkw = new ZKWatcher(TEST_UTIL.getConfiguration(),
         "unittest", new Abortable() {
           boolean aborted = false;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index cac6fd6..53f27d1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -23,7 +23,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -54,7 +53,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@@ -71,18 +70,18 @@ public class MockRegionServerServices implements RegionServerServices {
     new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
   private HFileSystem hfs = null;
   private final Configuration conf;
-  private ZooKeeperWatcher zkw = null;
+  private ZKWatcher zkw = null;
   private ServerName serverName = null;
   private RpcServerInterface rpcServer = null;
   private volatile boolean abortRequested;
   private volatile boolean stopping = false;
   private final AtomicBoolean running = new AtomicBoolean(true);
 
-  MockRegionServerServices(ZooKeeperWatcher zkw) {
+  MockRegionServerServices(ZKWatcher zkw) {
     this(zkw, null);
   }
 
-  MockRegionServerServices(ZooKeeperWatcher zkw, ServerName serverName) {
+  MockRegionServerServices(ZKWatcher zkw, ServerName serverName) {
     this.zkw = zkw;
     this.serverName = serverName;
     this.conf = (zkw == null ? new Configuration() : zkw.getConfiguration());
@@ -167,7 +166,7 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public ZooKeeperWatcher getZooKeeper() {
+  public ZKWatcher getZooKeeper() {
     return zkw;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
index fb1c1e2..961677b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRespon
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -124,7 +124,7 @@ public class TestMetaTableAccessorNoCluster {
   public void testRideOverServerNotRunning()
       throws IOException, InterruptedException, ServiceException {
     // Need a zk watcher.
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
+    ZKWatcher zkw = new ZKWatcher(UTIL.getConfiguration(),
       this.getClass().getSimpleName(), ABORTABLE, true);
     // This is a servername we use in a few places below.
     ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
index 6a904a7..c2a4945 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -66,7 +66,7 @@ public class TestMetaTableLocator {
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static final ServerName SN =
       ServerName.valueOf("example.org", 1234, System.currentTimeMillis());
-  private ZooKeeperWatcher watcher;
+  private ZKWatcher watcher;
   private Abortable abortable;
 
   @BeforeClass public static void beforeClass() throws Exception {
@@ -91,7 +91,7 @@ public class TestMetaTableLocator {
         return false;
       }
     };
-    this.watcher = new ZooKeeperWatcher(UTIL.getConfiguration(),
+    this.watcher = new ZKWatcher(UTIL.getConfiguration(),
       this.getClass().getSimpleName(), this.abortable, true);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index d546d5d..ed93b17 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
@@ -184,8 +184,8 @@ public class TestZooKeeper {
    */
   @Test
   public void testCreateWithParents() throws Exception {
-    ZooKeeperWatcher zkw =
-        new ZooKeeperWatcher(new Configuration(TEST_UTIL.getConfiguration()),
+    ZKWatcher zkw =
+        new ZKWatcher(new Configuration(TEST_UTIL.getConfiguration()),
             TestZooKeeper.class.getName(), null);
     byte[] expectedData = new byte[] { 1, 2, 3 };
     ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4/testCreateWithParents", expectedData);
@@ -206,7 +206,7 @@ public class TestZooKeeper {
    */
   @Test
   public void testZNodeDeletes() throws Exception {
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(
+    ZKWatcher zkw = new ZKWatcher(
       new Configuration(TEST_UTIL.getConfiguration()),
       TestZooKeeper.class.getName(), null);
     ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4");
@@ -247,7 +247,7 @@ public class TestZooKeeper {
 
     // Assumes the  root of the ZooKeeper space is writable as it creates a node
     // wherever the cluster home is defined.
-    ZooKeeperWatcher zk2 = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+    ZKWatcher zk2 = new ZKWatcher(TEST_UTIL.getConfiguration(),
       "testCreateSilentIsReallySilent", null);
 
     // Save the previous ACL
@@ -330,7 +330,7 @@ public class TestZooKeeper {
   @SuppressWarnings("deprecation")
   public void testGetChildDataAndWatchForNewChildrenShouldNotThrowNPE()
       throws Exception {
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), name.getMethodName(), null);
+    ZKWatcher zkw = new ZKWatcher(TEST_UTIL.getConfiguration(), name.getMethodName(), null);
     ZKUtil.getChildDataAndWatchForNewChildren(zkw, "/wrongNode");
   }
 
@@ -345,7 +345,7 @@ public class TestZooKeeper {
     cluster.startRegionServer();
     cluster.waitForActiveAndReadyMaster(10000);
     HMaster m = cluster.getMaster();
-    final ZooKeeperWatcher zkw = m.getZooKeeper();
+    final ZKWatcher zkw = m.getZooKeeper();
     // now the cluster is up. So assign some regions.
     try (Admin admin = TEST_UTIL.getAdmin()) {
       byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"),
@@ -376,14 +376,14 @@ public class TestZooKeeper {
    * Count listeners in zkw excluding listeners, that belongs to workers or other
    * temporary processes.
    */
-  private int countPermanentListeners(ZooKeeperWatcher watcher) {
+  private int countPermanentListeners(ZKWatcher watcher) {
     return countListeners(watcher, ZkSplitLogWorkerCoordination.class);
   }
 
   /**
    * Count listeners in zkw excluding provided classes
    */
-  private int countListeners(ZooKeeperWatcher watcher, Class<?>... exclude) {
+  private int countListeners(ZKWatcher watcher, Class<?>... exclude) {
     int cnt = 0;
     for (Object o : watcher.getListeners()) {
       boolean skip = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
index aa246c2..1c2279c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -93,7 +93,7 @@ public class TestZooKeeperTableArchiveClient {
     CONNECTION = (ClusterConnection)ConnectionFactory.createConnection(UTIL.getConfiguration());
     archivingClient = new ZKTableArchiveClient(UTIL.getConfiguration(), CONNECTION);
     // make hfile archiving node so we can archive files
-    ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+    ZKWatcher watcher = UTIL.getZooKeeperWatcher();
     String archivingZNode = ZKTableArchiveClient.getArchiveZNode(UTIL.getConfiguration(), watcher);
     ZKUtil.createWithParents(watcher, archivingZNode);
     rss = mock(RegionServerServices.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 6a4ada3..5660a01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -56,8 +56,8 @@ import org.apache.hadoop.hbase.util.HBaseFsckRepair;
 import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
 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.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
@@ -132,7 +132,7 @@ public class TestMetaWithReplicas {
   @Test
   public void testZookeeperNodesForReplicas() throws Exception {
     // Checks all the znodes exist when meta's replicas are enabled
-    ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+    ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
     Configuration conf = TEST_UTIL.getConfiguration();
     String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
         HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
@@ -166,7 +166,7 @@ public class TestMetaWithReplicas {
     // server holding the primary meta replica. Then it does a put/get into/from
     // the test table. The put/get operations would use the replicas to locate the
     // location of the test table's region
-    ZooKeeperWatcher zkw = util.getZooKeeperWatcher();
+    ZKWatcher zkw = util.getZooKeeperWatcher();
     Configuration conf = util.getConfiguration();
     conf.setBoolean(HConstants.USE_META_REPLICAS, true);
 
@@ -368,7 +368,7 @@ public class TestMetaWithReplicas {
         false, false);
     HBaseFsckRepair.closeRegionSilentlyAndWait(c,
         rl.getRegionLocation(2).getServerName(), rl.getRegionLocation(2).getRegionInfo());
-    ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+    ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
     ZKUtil.deleteNode(zkw, zkw.znodePaths.getZNodeForReplica(2));
     // check that problem exists
     HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false);
@@ -400,7 +400,7 @@ public class TestMetaWithReplicas {
     // caches update themselves. Uses the master operations to test
     // this
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+    ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
     String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
         HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
     String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index 62951ef..c98a02c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -34,17 +34,15 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -172,7 +170,7 @@ public class TestReplicationAdmin {
     ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
     rpc2.setClusterKey(KEY_SECOND);
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "Test HBaseAdmin", null);
+    ZKWatcher zkw = new ZKWatcher(conf, "Test HBaseAdmin", null);
     ReplicationQueues repQueues =
         ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null, zkw));
     repQueues.init("server1");
@@ -187,7 +185,7 @@ public class TestReplicationAdmin {
     }
     repQueues.removeQueue(ID_ONE);
     assertEquals(0, repQueues.getAllQueues().size());
-    
+
     // add recovered queue for ID_ONE
     repQueues.addLog(ID_ONE + "-server2", "file1");
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
index 0595a67..2a35365 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -60,10 +60,10 @@ import static org.junit.Assert.fail;
 @Category({CoprocessorTests.class, MediumTests.class})
 public class TestMasterCoprocessorExceptionWithAbort {
 
-  public static class MasterTracker extends ZooKeeperNodeTracker {
+  public static class MasterTracker extends ZKNodeTracker {
     public boolean masterZKNodeWasDeleted = false;
 
-    public MasterTracker(ZooKeeperWatcher zkw, String masterNode, Abortable abortable) {
+    public MasterTracker(ZKWatcher zkw, String masterNode, Abortable abortable) {
       super(zkw, masterNode, abortable);
     }
 
@@ -174,7 +174,7 @@ public class TestMasterCoprocessorExceptionWithAbort {
 
     // set a watch on the zookeeper /hbase/master node. If the master dies,
     // the node will be deleted.
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
+    ZKWatcher zkw = new ZKWatcher(UTIL.getConfiguration(),
       "unittest", new Abortable() {
       @Override
       public void abort(String why, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
index d4c6e4f..c4defa2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
@@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -59,10 +59,10 @@ import static org.junit.Assert.fail;
 @Category({CoprocessorTests.class, MediumTests.class})
 public class TestMasterCoprocessorExceptionWithRemove {
 
-  public static class MasterTracker extends ZooKeeperNodeTracker {
+  public static class MasterTracker extends ZKNodeTracker {
     public boolean masterZKNodeWasDeleted = false;
 
-    public MasterTracker(ZooKeeperWatcher zkw, String masterNode, Abortable abortable) {
+    public MasterTracker(ZKWatcher zkw, String masterNode, Abortable abortable) {
       super(zkw, masterNode, abortable);
     }
 
@@ -159,7 +159,7 @@ public class TestMasterCoprocessorExceptionWithRemove {
     // we are testing that the default setting of hbase.coprocessor.abortonerror
     // =false
     // is respected.
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
+    ZKWatcher zkw = new ZKWatcher(UTIL.getConfiguration(),
       "unittest", new Abortable() {
       @Override
       public void abort(String why, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 85d2b0b..dadec1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
 import com.google.protobuf.Service;
 
@@ -150,7 +150,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public ZooKeeperWatcher getZooKeeper() {
+  public ZKWatcher getZooKeeper() {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 25cfbd9..8358e0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -25,7 +25,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 
@@ -65,7 +64,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
@@ -138,7 +137,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuo
 class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
 ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   private final ServerName sn;
-  private final ZooKeeperWatcher zkw;
+  private final ZKWatcher zkw;
   private final Configuration conf;
   private final Random random = new Random();
 
@@ -191,7 +190,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   throws ZooKeeperConnectionException, IOException {
     this.sn = sn;
     this.conf = conf;
-    this.zkw = new ZooKeeperWatcher(conf, sn.toString(), this, true);
+    this.zkw = new ZKWatcher(conf, sn.toString(), this, true);
   }
 
   /**
@@ -284,7 +283,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public ZooKeeperWatcher getZooKeeper() {
+  public ZKWatcher getZooKeeper() {
     return this.zkw;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
index f1feef5..6fcd8ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
@@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -70,7 +70,7 @@ public class TestActiveMasterManager {
   }
 
   @Test public void testRestartMaster() throws IOException, KeeperException {
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+    ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
       "testActiveMasterManagerFromZK", null, true);
     try {
       ZKUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode);
@@ -112,7 +112,7 @@ public class TestActiveMasterManager {
    */
   @Test
   public void testActiveMasterManagerFromZK() throws Exception {
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+    ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
       "testActiveMasterManagerFromZK", null, true);
     try {
       ZKUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode);
@@ -135,7 +135,7 @@ public class TestActiveMasterManager {
     ClusterStatusTracker clusterStatusTracker =
       ms1.getClusterStatusTracker();
     clusterStatusTracker.setClusterUp();
-    activeMasterManager.blockUntilBecomingActiveMaster(100, 
+    activeMasterManager.blockUntilBecomingActiveMaster(100,
         Mockito.mock(MonitoredTask.class));
     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
     assertMaster(zk, firstMasterAddress);
@@ -193,9 +193,9 @@ public class TestActiveMasterManager {
    * @param zk
    * @param thisMasterAddress
    * @throws KeeperException
-   * @throws IOException 
+   * @throws IOException
    */
-  private void assertMaster(ZooKeeperWatcher zk,
+  private void assertMaster(ZKWatcher zk,
       ServerName expectedAddress)
   throws KeeperException, IOException {
     ServerName readAddress = MasterAddressTracker.getMasterAddress(zk);
@@ -209,7 +209,7 @@ public class TestActiveMasterManager {
     DummyMaster dummyMaster;
     boolean isActiveMaster;
 
-    public WaitToBeMasterThread(ZooKeeperWatcher zk, ServerName address) {
+    public WaitToBeMasterThread(ZKWatcher zk, ServerName address) {
       this.dummyMaster = new DummyMaster(zk,address);
       this.manager = this.dummyMaster.getActiveMasterManager();
       isActiveMaster = false;
@@ -224,13 +224,13 @@ public class TestActiveMasterManager {
     }
   }
 
-  public static class NodeDeletionListener extends ZooKeeperListener {
+  public static class NodeDeletionListener extends ZKListener {
     private static final Log LOG = LogFactory.getLog(NodeDeletionListener.class);
 
     private Semaphore lock;
     private String node;
 
-    public NodeDeletionListener(ZooKeeperWatcher watcher, String node) {
+    public NodeDeletionListener(ZKWatcher watcher, String node) {
       super(watcher);
       lock = new Semaphore(0);
       this.node = node;
@@ -257,7 +257,7 @@ public class TestActiveMasterManager {
     private ClusterStatusTracker clusterStatusTracker;
     private ActiveMasterManager activeMasterManager;
 
-    public DummyMaster(ZooKeeperWatcher zk, ServerName master) {
+    public DummyMaster(ZKWatcher zk, ServerName master) {
       this.clusterStatusTracker =
         new ClusterStatusTracker(zk, this);
       clusterStatusTracker.start();
@@ -269,7 +269,7 @@ public class TestActiveMasterManager {
 
     @Override
     public void abort(final String msg, final Throwable t) {}
-    
+
     @Override
     public boolean isAborted() {
       return false;
@@ -281,7 +281,7 @@ public class TestActiveMasterManager {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return null;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
index d8707c3..7f8e9c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
@@ -47,11 +47,9 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
-import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
 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.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -319,7 +317,7 @@ public class TestAssignmentListener {
     // are properly added to the ServerManager.drainingServers when they
     // register with the ServerManager under these circumstances.
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZooKeeperWatcher zooKeeper = new ZooKeeperWatcher(conf,
+    ZKWatcher zooKeeper = new ZKWatcher(conf,
         "zkWatcher-NewServerDrainTest", abortable, true);
     String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
         HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
index 852b139..bd7c507 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
@@ -27,19 +27,13 @@ import java.net.InetAddress;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ClockOutOfSyncException;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index 3d59639..bc68b28 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -41,7 +41,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.commons.logging.Log;
@@ -93,8 +92,7 @@ import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -210,7 +208,7 @@ public class TestDistributedLogSplitting {
     Path rootdir = FSUtils.getRootDir(conf);
 
     int numRegions = 50;
-    Table t = installTable(new ZooKeeperWatcher(conf, "table-creation", null),
+    Table t = installTable(new ZKWatcher(conf, "table-creation", null),
         "table", "family", numRegions);
     try {
       TableName table = t.getName();
@@ -283,7 +281,7 @@ public class TestDistributedLogSplitting {
     // they will consume recovered.edits
     master.balanceSwitch(false);
 
-    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    final ZKWatcher zkw = new ZKWatcher(conf, "table-creation", null);
     Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
     try {
       HRegionServer hrs = findRSToKill(false, "table");
@@ -352,7 +350,7 @@ public class TestDistributedLogSplitting {
     final Path logDir = new Path(rootdir,
       AbstractFSWALProvider.getWALDirectoryName(hrs.getServerName().toString()));
 
-    Table t = installTable(new ZooKeeperWatcher(conf, "table-creation", null),
+    Table t = installTable(new ZKWatcher(conf, "table-creation", null),
         "table", "family", 40);
     try {
       makeWAL(hrs, ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()),
@@ -407,7 +405,7 @@ public class TestDistributedLogSplitting {
 
     startCluster(NUM_RS); // NUM_RS=6.
 
-    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+    final ZKWatcher zkw = new ZKWatcher(conf,
         "distributed log splitting test", null);
 
     Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
@@ -520,7 +518,7 @@ public class TestDistributedLogSplitting {
   public void testReadWriteSeqIdFiles() throws Exception {
     LOG.info("testReadWriteSeqIdFiles");
     startCluster(2);
-    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    final ZKWatcher zkw = new ZKWatcher(conf, "table-creation", null);
     Table ht = installTable(zkw, name.getMethodName(), "family", 10);
     try {
       FileSystem fs = master.getMasterFileSystem().getFileSystem();
@@ -551,12 +549,12 @@ public class TestDistributedLogSplitting {
     }
   }
 
-  Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
+  Table installTable(ZKWatcher zkw, String tname, String fname, int nrs) throws Exception {
     return installTable(zkw, tname, fname, nrs, 0);
   }
 
-  Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
-      int existingRegions) throws Exception {
+  Table installTable(ZKWatcher zkw, String tname, String fname, int nrs,
+                     int existingRegions) throws Exception {
     // Create a table with regions
     TableName table = TableName.valueOf(tname);
     byte [] family = Bytes.toBytes(fname);
@@ -737,7 +735,7 @@ public class TestDistributedLogSplitting {
     return count;
   }
 
-  private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master) throws Exception {
+  private void blockUntilNoRIT(ZKWatcher zkw, HMaster master) throws Exception {
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
index 64d5a02..4b2c911 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
@@ -66,7 +66,7 @@ public class TestHMasterRPCException {
     conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 2000);
     testUtil.startMiniZKCluster();
 
-    ZooKeeperWatcher watcher = testUtil.getZooKeeperWatcher();
+    ZKWatcher watcher = testUtil.getZooKeeperWatcher();
     ZKUtil.createWithParents(watcher, watcher.znodePaths.masterAddressZNode, Bytes.toBytes("fake:123"));
     master = new HMaster(conf);
     rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 25671fc..20c9fe1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -53,7 +53,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 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.metrics2.lib.DefaultMetricsSystem;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
@@ -105,7 +105,7 @@ public class TestMasterNoCluster {
   public void tearDown()
   throws KeeperException, ZooKeeperConnectionException, IOException {
     // Make sure zk is clean before we run the next test.
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(TESTUTIL.getConfiguration(),
+    ZKWatcher zkw = new ZKWatcher(TESTUTIL.getConfiguration(),
         "@Before", new Abortable() {
       @Override
       public void abort(String why, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
index ea57e15..e2a2312 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
@@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -95,7 +95,7 @@ public class TestMasterStatusServlet {
     Mockito.doReturn(serverManager).when(master).getServerManager();
 
     // Fake ZKW
-    ZooKeeperWatcher zkw = Mockito.mock(ZooKeeperWatcher.class);
+    ZKWatcher zkw = Mockito.mock(ZKWatcher.class);
     Mockito.doReturn(new ZNodePaths(conf)).when(zkw).getZNodePaths();
     Mockito.doReturn("fakequorum").when(zkw).getQuorum();
     Mockito.doReturn(zkw).when(master).getZooKeeper();

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
index ca64326..d74b732 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
@@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
@@ -84,7 +84,7 @@ public class TestSplitLogManager {
     Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
   }
 
-  private ZooKeeperWatcher zkw;
+  private ZKWatcher zkw;
   private DummyMasterServices master;
   private SplitLogManager slm;
   private Configuration conf;
@@ -93,17 +93,17 @@ public class TestSplitLogManager {
   private static HBaseTestingUtility TEST_UTIL;
 
   class DummyMasterServices extends MockNoopMasterServices {
-    private ZooKeeperWatcher zkw;
+    private ZKWatcher zkw;
     private CoordinatedStateManager cm;
 
-    public DummyMasterServices(ZooKeeperWatcher zkw, Configuration conf) {
+    public DummyMasterServices(ZKWatcher zkw, Configuration conf) {
       super(conf);
       this.zkw = zkw;
       cm = new ZkCoordinatedStateManager(this);
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return zkw;
     }
 
@@ -125,7 +125,7 @@ public class TestSplitLogManager {
     conf = TEST_UTIL.getConfiguration();
     // Use a different ZK wrapper instance for each tests.
     zkw =
-        new ZooKeeperWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null);
+        new ZKWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null);
     master = new DummyMasterServices(zkw, conf);
 
     ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
index 6cf259a..1f61ee7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
@@ -29,8 +29,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 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.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Assert;
@@ -60,7 +60,7 @@ public class TestTableStateManager {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.startMiniCluster(2, 1);
     TEST_UTIL.shutdownMiniHBaseCluster();
-    ZooKeeperWatcher watcher = TEST_UTIL.getZooKeeperWatcher();
+    ZKWatcher watcher = TEST_UTIL.getZooKeeperWatcher();
     setTableStateInZK(watcher, tableName, ZooKeeperProtos.DeprecatedTableState.State.DISABLED);
     TEST_UTIL.restartHBaseCluster(1);
 
@@ -70,8 +70,8 @@ public class TestTableStateManager {
         TableState.State.DISABLED);
   }
 
-  private void setTableStateInZK(ZooKeeperWatcher watcher, final TableName tableName,
-      final ZooKeeperProtos.DeprecatedTableState.State state)
+  private void setTableStateInZK(ZKWatcher watcher, final TableName tableName,
+                                 final ZooKeeperProtos.DeprecatedTableState.State state)
       throws KeeperException, IOException {
     String znode = ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString());
     if (ZKUtil.checkExists(watcher, znode) == -1) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 572816d..97d6189 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -205,9 +205,9 @@ public class TestHFileCleaner {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       try {
-        return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+        return new ZKWatcher(getConfiguration(), "dummy server", this);
       } catch (IOException e) {
         e.printStackTrace();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index 773d0fc..9820fd0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -146,9 +146,9 @@ public class TestHFileLinkCleaner {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       try {
-        return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+        return new ZKWatcher(getConfiguration(), "dummy server", this);
       } catch (IOException e) {
         e.printStackTrace();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index b5ca894..4ccc677 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
@@ -234,7 +234,7 @@ public class TestLogsCleaner {
 
     // when zk is working both files should be returned
     cleaner = new ReplicationLogCleaner();
-    try (ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null)) {
+    try (ZKWatcher zkw = new ZKWatcher(conf, "testZooKeeperAbort-normal", null)) {
       cleaner.setConf(conf, zkw);
       cleaner.preClean();
       Iterable<FileStatus> filesToDelete = cleaner.getDeletableFiles(dummyFiles);
@@ -255,9 +255,9 @@ public class TestLogsCleaner {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       try {
-        return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+        return new ZKWatcher(getConfiguration(), "dummy server", this);
       } catch (IOException e) {
         e.printStackTrace();
       }
@@ -321,7 +321,7 @@ public class TestLogsCleaner {
     }
   }
 
-  static class FaultyZooKeeperWatcher extends ZooKeeperWatcher {
+  static class FaultyZooKeeperWatcher extends ZKWatcher {
     private RecoverableZooKeeper zk;
 
     public FaultyZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable)

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index be7f35e..b2ae3bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.junit.After;
@@ -240,7 +240,7 @@ public class TestReplicationHFileCleaner {
 
     // when zk is working both files should be returned
     cleaner = new ReplicationHFileCleaner();
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null);
+    ZKWatcher zkw = new ZKWatcher(conf, "testZooKeeperAbort-normal", null);
     try {
       cleaner.setConf(conf, zkw);
       Iterable<FileStatus> filesToDelete = cleaner.getDeletableFiles(dummyFiles);
@@ -263,9 +263,9 @@ public class TestReplicationHFileCleaner {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       try {
-        return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+        return new ZKWatcher(getConfiguration(), "dummy server", this);
       } catch (IOException e) {
         e.printStackTrace();
       }
@@ -332,7 +332,7 @@ public class TestReplicationHFileCleaner {
     }
   }
 
-  static class FaultyZooKeeperWatcher extends ZooKeeperWatcher {
+  static class FaultyZooKeeperWatcher extends ZKWatcher {
     private RecoverableZooKeeper zk;
     public FaultyZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable)
         throws ZooKeeperConnectionException, IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
index e11143d..6aa59cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -48,12 +48,12 @@ public class TestReplicationZKNodeCleaner {
   private final String SERVER_TWO = "server2";
 
   private final Configuration conf;
-  private final ZooKeeperWatcher zkw;
+  private final ZKWatcher zkw;
   private final ReplicationQueues repQueues;
 
   public TestReplicationZKNodeCleaner() throws Exception {
     conf = TEST_UTIL.getConfiguration();
-    zkw = new ZooKeeperWatcher(conf, "TestReplicationZKNodeCleaner", null);
+    zkw = new ZKWatcher(conf, "TestReplicationZKNodeCleaner", null);
     repQueues = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null,
         zkw));
     assertTrue(repQueues instanceof ReplicationQueuesZKImpl);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
index 58efa87..7d6f80a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.DaemonThreadFactory;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.zookeeper.KeeperException;
@@ -51,7 +51,7 @@ public class SimpleRSProcedureManager extends RegionServerProcedureManager {
   @Override
   public void initialize(RegionServerServices rss) throws KeeperException {
     this.rss = rss;
-    ZooKeeperWatcher zkw = rss.getZooKeeper();
+    ZKWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw, getProcedureSignature());
 
     ThreadPoolExecutor pool =

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
index 14d7eab..36ea086 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.errorhandling.TimeoutException;
 import org.apache.hadoop.hbase.procedure.Subprocedure.SubprocedureImpl;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -85,8 +85,8 @@ public class TestZKProcedure {
     UTIL.shutdownMiniZKCluster();
   }
 
-  private static ZooKeeperWatcher newZooKeeperWatcher() throws IOException {
-    return new ZooKeeperWatcher(UTIL.getConfiguration(), "testing utility", new Abortable() {
+  private static ZKWatcher newZooKeeperWatcher() throws IOException {
+    return new ZKWatcher(UTIL.getConfiguration(), "testing utility", new Abortable() {
       @Override
       public void abort(String why, Throwable e) {
         throw new RuntimeException(
@@ -123,7 +123,7 @@ public class TestZKProcedure {
     List<String> expected = Arrays.asList(members);
 
     // setup the constants
-    ZooKeeperWatcher coordZkw = newZooKeeperWatcher();
+    ZKWatcher coordZkw = newZooKeeperWatcher();
     String opDescription = "coordination test - " + members.length + " cohort members";
 
     // start running the controller
@@ -144,7 +144,7 @@ public class TestZKProcedure {
     List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> procMembers = new ArrayList<>(members.length);
     // start each member
     for (String member : members) {
-      ZooKeeperWatcher watcher = newZooKeeperWatcher();
+      ZKWatcher watcher = newZooKeeperWatcher();
       ZKProcedureMemberRpcs comms = new ZKProcedureMemberRpcs(watcher, opDescription);
       ThreadPoolExecutor pool2 = ProcedureMember.defaultPool(member, 1, KEEP_ALIVE);
       ProcedureMember procMember = new ProcedureMember(comms, pool2, subprocFactory);
@@ -207,7 +207,7 @@ public class TestZKProcedure {
     final CountDownLatch coordinatorReceivedErrorLatch = new CountDownLatch(1);
 
     // start running the coordinator and its controller
-    ZooKeeperWatcher coordinatorWatcher = newZooKeeperWatcher();
+    ZKWatcher coordinatorWatcher = newZooKeeperWatcher();
     ZKProcedureCoordinator coordinatorController = new ZKProcedureCoordinator(
         coordinatorWatcher, opDescription, COORDINATOR_NODE_NAME);
     ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE);
@@ -217,7 +217,7 @@ public class TestZKProcedure {
     SubprocedureFactory subprocFactory = Mockito.mock(SubprocedureFactory.class);
     List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> members = new ArrayList<>(expected.size());
     for (String member : expected) {
-      ZooKeeperWatcher watcher = newZooKeeperWatcher();
+      ZKWatcher watcher = newZooKeeperWatcher();
       ZKProcedureMemberRpcs controller = new ZKProcedureMemberRpcs(watcher, opDescription);
       ThreadPoolExecutor pool2 = ProcedureMember.defaultPool(member, 1, KEEP_ALIVE);
       ProcedureMember mem = new ProcedureMember(controller, pool2, subprocFactory);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
index 4ab9685..5d68074 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -79,7 +79,7 @@ public class TestZKProcedureControllers {
    */
   @Test(timeout = 60000)
   public void testSimpleZKCohortMemberController() throws Exception {
-    ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+    ZKWatcher watcher = UTIL.getZooKeeperWatcher();
     final String operationName = "instanceTest";
 
     final Subprocedure sub = Mockito.mock(Subprocedure.class);
@@ -171,7 +171,7 @@ public class TestZKProcedureControllers {
 
   private void runMockCommitWithOrchestratedControllers(StartControllers controllers,
       String operationName, byte[] data, String... cohort) throws Exception {
-    ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+    ZKWatcher watcher = UTIL.getZooKeeperWatcher();
     List<String> expected = Lists.newArrayList(cohort);
 
     final Subprocedure sub = Mockito.mock(Subprocedure.class);
@@ -247,7 +247,7 @@ public class TestZKProcedureControllers {
 
   public void runEarlyPrepareNodes(StartControllers controllers, String operationName, byte[] data,
       String... cohort) throws Exception {
-    ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+    ZKWatcher watcher = UTIL.getZooKeeperWatcher();
     List<String> expected = Lists.newArrayList(cohort);
 
     final Subprocedure sub = Mockito.mock(Subprocedure.class);
@@ -344,7 +344,7 @@ public class TestZKProcedureControllers {
   /**
    * Verify that the prepare, commit and abort nodes for the operation are removed from zookeeper
    */
-  private void verifyZooKeeperClean(String operationName, ZooKeeperWatcher watcher,
+  private void verifyZooKeeperClean(String operationName, ZKWatcher watcher,
       ZKProcedureUtil controller) throws Exception {
     String prepare = ZKProcedureUtil.getAcquireBarrierNode(controller, operationName);
     String commit = ZKProcedureUtil.getReachedBarrierNode(controller, operationName);
@@ -383,18 +383,18 @@ public class TestZKProcedureControllers {
    */
   private abstract class StartControllers {
     public abstract Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
-        ZooKeeperWatcher watcher, String operationName,
-        ProcedureCoordinator coordinator, String controllerName,
-        ProcedureMember member, List<String> cohortNames) throws Exception;
+            ZKWatcher watcher, String operationName,
+            ProcedureCoordinator coordinator, String controllerName,
+            ProcedureMember member, List<String> cohortNames) throws Exception;
   }
 
   private final StartControllers startCoordinatorFirst = new StartControllers() {
 
     @Override
     public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
-        ZooKeeperWatcher watcher, String operationName,
-        ProcedureCoordinator coordinator, String controllerName,
-        ProcedureMember member, List<String> expected) throws Exception {
+            ZKWatcher watcher, String operationName,
+            ProcedureCoordinator coordinator, String controllerName,
+            ProcedureMember member, List<String> expected) throws Exception {
       // start the controller
       ZKProcedureCoordinator controller = new ZKProcedureCoordinator(
           watcher, operationName, CONTROLLER_NODE_NAME);
@@ -420,9 +420,9 @@ public class TestZKProcedureControllers {
 
     @Override
     public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
-        ZooKeeperWatcher watcher, String operationName,
-        ProcedureCoordinator coordinator, String controllerName,
-        ProcedureMember member, List<String> expected) throws Exception {
+            ZKWatcher watcher, String operationName,
+            ProcedureCoordinator coordinator, String controllerName,
+            ProcedureMember member, List<String> expected) throws Exception {
 
       // make a cohort controller for each expected node
       List<ZKProcedureMemberRpcs> cohortControllers = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
index 5c24c2c..9bd74d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
@@ -25,7 +25,6 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
@@ -42,8 +41,8 @@ import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
 import org.apache.hadoop.hbase.wal.FSHLogProvider;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALProvider;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -122,7 +121,7 @@ public class TestCompactionInDeadRegionServer {
   public void test() throws Exception {
     HRegionServer rsToSuspend = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
     HRegion region = (HRegion) rsToSuspend.getRegions(TABLE_NAME).get(0);
-    ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
+    ZKWatcher watcher = UTIL.getZooKeeperWatcher();
     watcher.getRecoverableZooKeeper().delete(
       ZNodePaths.joinZNode(watcher.getZNodePaths().rsZNode, rsToSuspend.getServerName().toString()),
       -1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index b16feca..2a3ce00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerResult;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -813,7 +813,7 @@ public class TestHeapMemoryManager {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return null;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
index 84865cc..f1b4441 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java
@@ -31,9 +31,9 @@ import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -80,7 +80,7 @@ public class TestMasterAddressTracker {
    */
   private MasterAddressTracker setupMasterTracker(final ServerName sn, final int infoPort)
       throws Exception {
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+    ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
         name.getMethodName(), null);
     ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode);
 
@@ -156,13 +156,13 @@ public class TestMasterAddressTracker {
     assertEquals("Should receive 0 for backup not found.", 0, addressTracker.getMasterInfoPort());
   }
 
-  public static class NodeCreationListener extends ZooKeeperListener {
+  public static class NodeCreationListener extends ZKListener {
     private static final Log LOG = LogFactory.getLog(NodeCreationListener.class);
 
     private Semaphore lock;
     private String node;
 
-    public NodeCreationListener(ZooKeeperWatcher watcher, String node) {
+    public NodeCreationListener(ZKWatcher watcher, String node) {
       super(watcher);
       lock = new Semaphore(0);
       this.node = node;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
index cfecd9f..c9cae7e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.tmpl.regionserver.RSStatusTmpl;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -92,7 +92,7 @@ public class TestRSStatusServlet {
     Mockito.doReturn(fakeResponse).when(rpcServices).getServerInfo(
       (RpcController)Mockito.any(), (GetServerInfoRequest)Mockito.any());
     // Fake ZKW
-    ZooKeeperWatcher zkw = Mockito.mock(ZooKeeperWatcher.class);
+    ZKWatcher zkw = Mockito.mock(ZKWatcher.class);
     Mockito.doReturn("fakequorum").when(zkw).getQuorum();
     Mockito.doReturn(zkw).when(rs).getZooKeeper();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
index 872fec6..abcc497 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -99,7 +99,7 @@ public class TestRegionServerHostname {
         TEST_UTIL.getConfiguration().set(HRegionServer.RS_HOSTNAME_KEY, hostName);
         TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
         try {
-          ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+          ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
           List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
           // there would be NUM_RS+1 children - one for the master
           assertTrue(servers.size() ==
@@ -160,7 +160,7 @@ public class TestRegionServerHostname {
     TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
     boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
     int expectedRS = NUM_RS + (tablesOnMaster? 1: 0);
-    try (ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher()) {
+    try (ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher()) {
       List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
       assertEquals(expectedRS, servers.size());
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index 1f7320e..6e20612 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -106,7 +106,7 @@ public class TestRegionServerNoMaster {
     // so that regions can be assigned during the mocking phase.
     HRegionServer hrs = HTU.getHBaseCluster()
       .getLiveRegionServerThreads().get(0).getRegionServer();
-    ZooKeeperWatcher zkw = hrs.getZooKeeper();
+    ZKWatcher zkw = hrs.getZooKeeper();
     MetaTableLocator mtl = new MetaTableLocator();
     ServerName sn = mtl.getMetaRegionLocation(zkw);
     if (sn != null && !masterAddr.equals(sn)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index 0bf9d03..f241f85 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -52,8 +52,8 @@ import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 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.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
@@ -74,16 +74,16 @@ public class TestSplitLogWorker {
   private final static HBaseTestingUtility TEST_UTIL =
     new HBaseTestingUtility();
   private DummyServer ds;
-  private ZooKeeperWatcher zkw;
+  private ZKWatcher zkw;
   private SplitLogWorker slw;
   private ExecutorService executorService;
 
   class DummyServer implements Server {
-    private ZooKeeperWatcher zkw;
+    private ZKWatcher zkw;
     private Configuration conf;
     private CoordinatedStateManager cm;
 
-    public DummyServer(ZooKeeperWatcher zkw, Configuration conf) {
+    public DummyServer(ZKWatcher zkw, Configuration conf) {
       this.zkw = zkw;
       this.conf = conf;
       cm = new ZkCoordinatedStateManager(this);
@@ -113,7 +113,7 @@ public class TestSplitLogWorker {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return zkw;
     }
 
@@ -193,7 +193,7 @@ public class TestSplitLogWorker {
   public void setup() throws Exception {
     TEST_UTIL.startMiniZKCluster();
     Configuration conf = TEST_UTIL.getConfiguration();
-    zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
+    zkw = new ZKWatcher(TEST_UTIL.getConfiguration(),
         "split-log-worker-tests", null);
     ds = new DummyServer(zkw, conf);
     ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 2e5c552..10a84b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -505,7 +505,7 @@ public class TestWALLockup {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return null;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index 6b7d36b..58b22c8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -75,8 +75,8 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 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.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -203,7 +203,7 @@ public class TestMasterReplication {
     Table[] htables = getHTablesOnClusters(tableName);
     putAndWait(row, famName, htables[0], htables[0]);
     rollWALAndWait(utilities[0], table.getTableName(), row);
-    ZooKeeperWatcher zkw = utilities[0].getZooKeeperWatcher();
+    ZKWatcher zkw = utilities[0].getZooKeeperWatcher();
     String queuesZnode = ZNodePaths.joinZNode(zkw.getZNodePaths().baseZNode,
       ZNodePaths.joinZNode("replication", "rs"));
     List<String> listChildrenNoWatch =
@@ -493,7 +493,7 @@ public class TestMasterReplication {
       utility.startMiniCluster();
       utilities[i] = utility;
       configurations[i] = conf;
-      new ZooKeeperWatcher(conf, "cluster" + i, null, true);
+      new ZKWatcher(conf, "cluster" + i, null, true);
     }
   }
 


[08/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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/330b0d05/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" ?>


[12/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/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
deleted file mode 100644
index 04f709f..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ /dev/null
@@ -1,814 +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.apache.hadoop.hbase.zookeeper.ZKMetadata.appendMetaData;
-import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
-
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.RetryCounterFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.trace.TraceUtil;
-import org.apache.htrace.core.TraceScope;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.ZooKeeper.States;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.CreateRequest;
-import org.apache.zookeeper.proto.SetDataRequest;
-
-/**
- * A zookeeper that can handle 'recoverable' errors.
- * To handle recoverable errors, developers need to realize that there are two
- * classes of requests: idempotent and non-idempotent requests. Read requests
- * and unconditional sets and deletes are examples of idempotent requests, they
- * can be reissued with the same results.
- * (Although, the delete may throw a NoNodeException on reissue its effect on
- * the ZooKeeper state is the same.) Non-idempotent requests need special
- * handling, application and library writers need to keep in mind that they may
- * need to encode information in the data or name of znodes to detect
- * retries. A simple example is a create that uses a sequence flag.
- * If a process issues a create("/x-", ..., SEQUENCE) and gets a connection
- * loss exception, that process will reissue another
- * create("/x-", ..., SEQUENCE) and get back x-111. When the process does a
- * getChildren("/"), it sees x-1,x-30,x-109,x-110,x-111, now it could be
- * that x-109 was the result of the previous create, so the process actually
- * owns both x-109 and x-111. An easy way around this is to use "x-process id-"
- * when doing the create. If the process is using an id of 352, before reissuing
- * the create it will do a getChildren("/") and see "x-222-1", "x-542-30",
- * "x-352-109", x-333-110". The process will know that the original create
- * succeeded an the znode it created is "x-352-109".
- * @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling"
- */
-@InterfaceAudience.Private
-public class RecoverableZooKeeper {
-  private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
-  // the actual ZooKeeper client instance
-  private ZooKeeper zk;
-  private final RetryCounterFactory retryCounterFactory;
-  // An identifier of this process in the cluster
-  private final String identifier;
-  private final byte[] id;
-  private Watcher watcher;
-  private int sessionTimeout;
-  private String quorumServers;
-  private final ZooKeeperMetricsListener metrics;
-
-  public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
-      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime)
-  throws IOException {
-    this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis, maxSleepTime,
-        null);
-  }
-
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
-      justification="None. Its always been this way.")
-  public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
-      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier)
-  throws IOException {
-    // TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as we should.
-    this.retryCounterFactory =
-      new RetryCounterFactory(maxRetries+1, retryIntervalMillis, maxSleepTime);
-
-    if (identifier == null || identifier.length() == 0) {
-      // the identifier = processID@hostName
-      identifier = ManagementFactory.getRuntimeMXBean().getName();
-    }
-    LOG.info("Process identifier=" + identifier +
-      " connecting to ZooKeeper ensemble=" + quorumServers);
-    this.identifier = identifier;
-    this.id = Bytes.toBytes(identifier);
-
-    this.watcher = watcher;
-    this.sessionTimeout = sessionTimeout;
-    this.quorumServers = quorumServers;
-    this.metrics = new MetricsZooKeeper();
-    try {checkZk();} catch (Exception x) {/* ignore */}
-  }
-
-  /**
-   * Try to create a ZooKeeper connection. Turns any exception encountered into a
-   * KeeperException.OperationTimeoutException so it can retried.
-   * @return The created ZooKeeper connection object
-   * @throws KeeperException
-   */
-  protected synchronized ZooKeeper checkZk() throws KeeperException {
-    if (this.zk == null) {
-      try {
-        this.zk = new ZooKeeper(quorumServers, sessionTimeout, watcher);
-      } catch (IOException ex) {
-        LOG.warn("Unable to create ZooKeeper Connection", ex);
-        throw new KeeperException.OperationTimeoutException();
-      }
-    }
-    return zk;
-  }
-
-  public synchronized void reconnectAfterExpiration()
-        throws IOException, KeeperException, InterruptedException {
-    if (zk != null) {
-      LOG.info("Closing dead ZooKeeper connection, session" +
-        " was: 0x"+Long.toHexString(zk.getSessionId()));
-      zk.close();
-      // reset the ZooKeeper connection
-      zk = null;
-    }
-    checkZk();
-    LOG.info("Recreated a ZooKeeper, session" +
-      " is: 0x"+Long.toHexString(zk.getSessionId()));
-  }
-
-  /**
-   * delete is an idempotent operation. Retry before throwing exception.
-   * This function will not throw NoNodeException if the path does not
-   * exist.
-   */
-  public void delete(String path, int version) throws InterruptedException, KeeperException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.delete")) {
-      RetryCounter retryCounter = retryCounterFactory.create();
-      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) {
-                LOG.debug("Node " + path + " already deleted. Assuming a " +
-                    "previous attempt succeeded.");
-                return;
-              }
-              LOG.debug("Node " + path + " already deleted, retry=" + isRetry);
-              throw e;
-
-            case CONNECTIONLOSS:
-              this.metrics.registerConnectionLossException();
-              retryOrThrow(retryCounter, e, "delete");
-              break;
-            case OPERATIONTIMEOUT:
-              this.metrics.registerOperationTimeoutException();
-              retryOrThrow(retryCounter, e, "delete");
-              break;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-        isRetry = true;
-      }
-    }
-  }
-
-  /**
-   * exists is an idempotent operation. Retry before throwing exception
-   * @return A Stat instance
-   */
-  public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
-      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;
-        } 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;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  /**
-   * exists is an idempotent operation. Retry before throwing exception
-   * @return A Stat instance
-   */
-  public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
-      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;
-        } 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;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
-      String opName) throws KeeperException {
-    if (!retryCounter.shouldRetry()) {
-      LOG.error("ZooKeeper " + opName + " failed after "
-        + retryCounter.getMaxAttempts() + " attempts");
-      throw e;
-    }
-    LOG.debug("Retry, connectivity issue (JVM Pause?); quorum=" + quorumServers + "," +
-        "exception=" + e);
-  }
-
-  /**
-   * getChildren is an idempotent operation. Retry before throwing exception
-   * @return List of children znodes
-   */
-  public List<String> getChildren(String path, Watcher watcher)
-    throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
-      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;
-        } 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;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  /**
-   * getChildren is an idempotent operation. Retry before throwing exception
-   * @return List of children znodes
-   */
-  public List<String> getChildren(String path, boolean watch)
-  throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
-      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;
-        } 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;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  /**
-   * getData is an idempotent operation. Retry before throwing exception
-   * @return Data
-   */
-  public byte[] getData(String path, Watcher watcher, Stat stat)
-  throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
-      RetryCounter retryCounter = retryCounterFactory.create();
-      while (true) {
-        try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          byte[] revData = checkZk().getData(path, watcher, stat);
-          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return removeMetaData(revData);
-        } catch (KeeperException e) {
-          this.metrics.registerFailedZKCall();
-          switch (e.code()) {
-            case CONNECTIONLOSS:
-              this.metrics.registerConnectionLossException();
-              retryOrThrow(retryCounter, e, "getData");
-              break;
-            case OPERATIONTIMEOUT:
-              this.metrics.registerOperationTimeoutException();
-              retryOrThrow(retryCounter, e, "getData");
-              break;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  /**
-   * getData is an idempotent operation. Retry before throwing exception
-   * @return Data
-   */
-  public byte[] getData(String path, boolean watch, Stat stat)
-  throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
-      RetryCounter retryCounter = retryCounterFactory.create();
-      while (true) {
-        try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          byte[] revData = checkZk().getData(path, watch, stat);
-          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return removeMetaData(revData);
-        } catch (KeeperException e) {
-          this.metrics.registerFailedZKCall();
-          switch (e.code()) {
-            case CONNECTIONLOSS:
-              this.metrics.registerConnectionLossException();
-              retryOrThrow(retryCounter, e, "getData");
-              break;
-            case OPERATIONTIMEOUT:
-              this.metrics.registerOperationTimeoutException();
-              retryOrThrow(retryCounter, e, "getData");
-              break;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  /**
-   * setData is NOT an idempotent operation. Retry may cause BadVersion Exception
-   * Adding an identifier field into the data to check whether
-   * badversion is caused by the result of previous correctly setData
-   * @return Stat instance
-   */
-  public Stat setData(String path, byte[] data, int version)
-  throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setData")) {
-      RetryCounter retryCounter = retryCounterFactory.create();
-      byte[] newData = appendMetaData(id, 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;
-        } 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:
-              if (isRetry) {
-                // 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;
-                }
-              }
-            // throw other exceptions and verified bad version exceptions
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-        isRetry = true;
-      }
-    }
-  }
-
-  /**
-   * getAcl is an idempotent operation. Retry before throwing exception
-   * @return list of ACLs
-   */
-  public List<ACL> getAcl(String path, Stat stat)
-  throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getAcl")) {
-      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;
-        } 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;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  /**
-   * setAcl is an idempotent operation. Retry before throwing exception
-   * @return list of ACLs
-   */
-  public Stat setAcl(String path, List<ACL> acls, int version)
-  throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setAcl")) {
-      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;
-        } 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;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  /**
-   * <p>
-   * NONSEQUENTIAL create is idempotent operation.
-   * Retry before throwing exceptions.
-   * But this function will not throw the NodeExist exception back to the
-   * application.
-   * </p>
-   * <p>
-   * But SEQUENTIAL is NOT idempotent operation. It is necessary to add
-   * identifier to the path to verify, whether the previous one is successful
-   * or not.
-   * </p>
-   *
-   * @return Path
-   */
-  public String create(String path, byte[] data, List<ACL> acl,
-      CreateMode createMode)
-  throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.create")) {
-      byte[] newData = appendMetaData(id, data);
-      switch (createMode) {
-        case EPHEMERAL:
-        case PERSISTENT:
-          return createNonSequential(path, newData, acl, createMode);
-
-        case EPHEMERAL_SEQUENTIAL:
-        case PERSISTENT_SEQUENTIAL:
-          return createSequential(path, newData, acl, createMode);
-
-        default:
-          throw new IllegalArgumentException("Unrecognized CreateMode: " +
-              createMode);
-      }
-    }
-  }
-
-  private String createNonSequential(String path, byte[] data, List<ACL> acl,
-      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;
-      } 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
-                return path;
-              }
-              LOG.error("Node " + path + " already exists with " +
-                  Bytes.toStringBinary(currentData) + ", could not write " +
-                  Bytes.toStringBinary(data));
-              throw e;
-            }
-            LOG.debug("Node " + path + " already exists");
-            throw e;
-
-          case CONNECTIONLOSS:
-            this.metrics.registerConnectionLossException();
-            retryOrThrow(retryCounter, e, "create");
-            break;
-          case OPERATIONTIMEOUT:
-            this.metrics.registerOperationTimeoutException();
-            retryOrThrow(retryCounter, e, "create");
-            break;
-
-          default:
-            throw e;
-        }
-      }
-      retryCounter.sleepUntilNextRetry();
-      isRetry = true;
-    }
-  }
-
-  private String createSequential(String path, byte[] data,
-      List<ACL> acl, CreateMode createMode)
-  throws KeeperException, InterruptedException {
-    RetryCounter retryCounter = retryCounterFactory.create();
-    boolean first = true;
-    String newPath = path+this.identifier;
-    while (true) {
-      try {
-        if (!first) {
-          // Check if we succeeded on a previous attempt
-          String previousResult = findPreviousSequentialNode(newPath);
-          if (previousResult != null) {
-            return previousResult;
-          }
-        }
-        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;
-      } 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;
-
-          default:
-            throw e;
-        }
-      }
-      retryCounter.sleepUntilNextRetry();
-    }
-  }
-  /**
-   * Convert Iterable of {@link org.apache.zookeeper.Op} we got into the ZooKeeper.Op
-   * instances to actually pass to multi (need to do this in order to appendMetaData).
-   */
-  private Iterable<Op> prepareZKMulti(Iterable<Op> ops)
-  throws UnsupportedOperationException {
-    if(ops == null) return null;
-
-    List<Op> preparedOps = new LinkedList<>();
-    for (Op op : ops) {
-      if (op.getType() == ZooDefs.OpCode.create) {
-        CreateRequest create = (CreateRequest)op.toRequestRecord();
-        preparedOps.add(Op.create(create.getPath(), appendMetaData(id, create.getData()),
-          create.getAcl(), create.getFlags()));
-      } else if (op.getType() == ZooDefs.OpCode.delete) {
-        // no need to appendMetaData for delete
-        preparedOps.add(op);
-      } else if (op.getType() == ZooDefs.OpCode.setData) {
-        SetDataRequest setData = (SetDataRequest)op.toRequestRecord();
-        preparedOps.add(Op.setData(setData.getPath(), appendMetaData(id, setData.getData()),
-          setData.getVersion()));
-      } else {
-        throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName());
-      }
-    }
-    return preparedOps;
-  }
-
-  /**
-   * Run multiple operations in a transactional manner. Retry before throwing exception
-   */
-  public List<OpResult> multi(Iterable<Op> ops)
-  throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.multi")) {
-      RetryCounter retryCounter = retryCounterFactory.create();
-      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;
-        } 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;
-
-            default:
-              throw e;
-          }
-        }
-        retryCounter.sleepUntilNextRetry();
-      }
-    }
-  }
-
-  private String findPreviousSequentialNode(String path)
-    throws KeeperException, InterruptedException {
-    int lastSlashIdx = path.lastIndexOf('/');
-    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;
-      }
-    }
-    return null;
-  }
-
-  public synchronized long getSessionId() {
-    return zk == null ? -1 : zk.getSessionId();
-  }
-
-  public synchronized void close() throws InterruptedException {
-    if (zk != null) zk.close();
-  }
-
-  public synchronized States getState() {
-    return zk == null ? null : zk.getState();
-  }
-
-  public synchronized ZooKeeper getZooKeeper() {
-    return zk;
-  }
-
-  public synchronized byte[] getSessionPasswd() {
-    return zk == null ? null : zk.getSessionPasswd();
-  }
-
-  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));
-  }
-
-  /**
-   * Filters the given node list by the given prefixes.
-   * This method is all-inclusive--if any element in the node list starts
-   * with any of the given prefixes, then it is included in the result.
-   *
-   * @param nodes the nodes to filter
-   * @param prefixes the prefixes to include in the result
-   * @return list of every element that starts with one of the prefixes
-   */
-  private static List<String> filterByPrefix(List<String> nodes,
-      String... prefixes) {
-    List<String> lockChildren = new ArrayList<>();
-    for (String child : nodes){
-      for (String prefix : prefixes){
-        if (child.startsWith(prefix)){
-          lockChildren.add(child);
-          break;
-        }
-      }
-    }
-    return lockChildren;
-  }
-
-  public String getIdentifier() {
-    return identifier;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
deleted file mode 100644
index 9ef7691..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.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 java.util.UUID;
-
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ClusterId;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Publishes and synchronizes a unique identifier specific to a given HBase
- * cluster.  The stored identifier is read from the file system by the active
- * master on startup, and is subsequently available to all watchers (including
- * clients).
- */
-@InterfaceAudience.Private
-public class ZKClusterId {
-  private ZooKeeperWatcher watcher;
-  private Abortable abortable;
-  private String id;
-
-  public ZKClusterId(ZooKeeperWatcher watcher, Abortable abortable) {
-    this.watcher = watcher;
-    this.abortable = abortable;
-  }
-
-  public boolean hasId() {
-    return getId() != null;
-  }
-
-  public String getId() {
-    try {
-      if (id == null) {
-        id = readClusterIdZNode(watcher);
-      }
-    } catch (KeeperException ke) {
-      abortable.abort("Unexpected exception from ZooKeeper reading cluster ID",
-          ke);
-    }
-    return id;
-  }
-
-  public static String readClusterIdZNode(ZooKeeperWatcher watcher)
-  throws KeeperException {
-    if (ZKUtil.checkExists(watcher, watcher.znodePaths.clusterIdZNode) != -1) {
-      byte [] data;
-      try {
-        data = ZKUtil.getData(watcher, watcher.znodePaths.clusterIdZNode);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        return null;
-      }
-      if (data != null) {
-        try {
-          return ClusterId.parseFrom(data).toString();
-        } catch (DeserializationException e) {
-          throw ZKUtil.convert(e);
-        }
-      }
-    }
-    return null;
-  }
-
-  public static void setClusterId(ZooKeeperWatcher watcher, ClusterId id)
-      throws KeeperException {
-    ZKUtil.createSetData(watcher, watcher.znodePaths.clusterIdZNode, id.toByteArray());
-  }
-
-  /**
-   * Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions
-   * @param zkw watcher connected to an ensemble
-   * @return the UUID read from zookeeper
-   * @throws KeeperException
-   */
-  public static UUID getUUIDForCluster(ZooKeeperWatcher zkw) throws KeeperException {
-    String uuid = readClusterIdZNode(zkw);
-    return uuid == null ? null : UUID.fromString(uuid);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
deleted file mode 100644
index b0610b0..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
+++ /dev/null
@@ -1,183 +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.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Handles coordination of a single "leader" instance among many possible
- * candidates.  The first {@link ZKLeaderManager} to successfully create
- * the given znode becomes the leader, allowing the instance to continue
- * with whatever processing must be protected.  Other {@link ZKLeaderManager}
- * instances will wait to be notified of changes to the leader znode.
- * If the current master instance fails, the ephemeral leader znode will
- * be removed, and all waiting instances will be notified, with the race
- * to claim the leader znode beginning all over again.
- * @deprecated Not used
- */
-@Deprecated
-@InterfaceAudience.Private
-public class ZKLeaderManager extends ZooKeeperListener {
-  private static final Log LOG = LogFactory.getLog(ZKLeaderManager.class);
-
-  private final AtomicBoolean leaderExists = new AtomicBoolean();
-  private String leaderZNode;
-  private byte[] nodeId;
-  private Stoppable candidate;
-
-  public ZKLeaderManager(ZooKeeperWatcher watcher, String leaderZNode,
-      byte[] identifier, Stoppable candidate) {
-    super(watcher);
-    this.leaderZNode = leaderZNode;
-    this.nodeId = identifier;
-    this.candidate = candidate;
-  }
-
-  public void start() {
-    try {
-      watcher.registerListener(this);
-      String parent = ZKUtil.getParent(leaderZNode);
-      if (ZKUtil.checkExists(watcher, parent) < 0) {
-        ZKUtil.createWithParents(watcher, parent);
-      }
-    } catch (KeeperException ke) {
-      watcher.abort("Unhandled zk exception when starting", ke);
-      candidate.stop("Unhandled zk exception starting up: "+ke.getMessage());
-    }
-  }
-
-  @Override
-  public void nodeCreated(String path) {
-    if (leaderZNode.equals(path) && !candidate.isStopped()) {
-      handleLeaderChange();
-    }
-  }
-
-  @Override
-  public void nodeDeleted(String path) {
-    if (leaderZNode.equals(path) && !candidate.isStopped()) {
-      handleLeaderChange();
-    }
-  }
-
-  private void handleLeaderChange() {
-    try {
-      synchronized(leaderExists) {
-        if (ZKUtil.watchAndCheckExists(watcher, leaderZNode)) {
-          LOG.info("Found new leader for znode: "+leaderZNode);
-          leaderExists.set(true);
-        } else {
-          LOG.info("Leader change, but no new leader found");
-          leaderExists.set(false);
-          leaderExists.notifyAll();
-        }
-      }
-    } catch (KeeperException ke) {
-      watcher.abort("ZooKeeper error checking for leader znode", ke);
-      candidate.stop("ZooKeeper error checking for leader: "+ke.getMessage());
-    }
-  }
-
-  /**
-   * Blocks until this instance has claimed the leader ZNode in ZooKeeper
-   */
-  public void waitToBecomeLeader() {
-    while (!candidate.isStopped()) {
-      try {
-        if (ZKUtil.createEphemeralNodeAndWatch(watcher, leaderZNode, nodeId)) {
-          // claimed the leader znode
-          leaderExists.set(true);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Claimed the leader znode as '"+
-                Bytes.toStringBinary(nodeId)+"'");
-          }
-          return;
-        }
-
-        // if claiming the node failed, there should be another existing node
-        byte[] currentId = ZKUtil.getDataAndWatch(watcher, leaderZNode);
-        if (currentId != null && Bytes.equals(currentId, nodeId)) {
-          // claimed with our ID, but we didn't grab it, possibly restarted?
-          LOG.info("Found existing leader with our ID ("+
-              Bytes.toStringBinary(nodeId)+"), removing");
-          ZKUtil.deleteNode(watcher, leaderZNode);
-          leaderExists.set(false);
-        } else {
-          LOG.info("Found existing leader with ID: "+Bytes.toStringBinary(nodeId));
-          leaderExists.set(true);
-        }
-      } catch (KeeperException ke) {
-        watcher.abort("Unexpected error from ZK, stopping candidate", ke);
-        candidate.stop("Unexpected error from ZK: "+ke.getMessage());
-        return;
-      }
-
-      // wait for next chance
-      synchronized(leaderExists) {
-        while (leaderExists.get() && !candidate.isStopped()) {
-          try {
-            leaderExists.wait();
-          } catch (InterruptedException ie) {
-            LOG.debug("Interrupted waiting on leader", ie);
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Removes the leader znode, if it is currently claimed by this instance.
-   */
-  public void stepDownAsLeader() {
-    try {
-      synchronized(leaderExists) {
-        if (!leaderExists.get()) {
-          return;
-        }
-        byte[] leaderId = ZKUtil.getData(watcher, leaderZNode);
-        if (leaderId != null && Bytes.equals(nodeId, leaderId)) {
-          LOG.info("Stepping down as leader");
-          ZKUtil.deleteNodeFailSilent(watcher, leaderZNode);
-          leaderExists.set(false);
-        } else {
-          LOG.info("Not current leader, no need to step down");
-        }
-      }
-    } catch (KeeperException ke) {
-      watcher.abort("Unhandled zookeeper exception removing leader node", ke);
-      candidate.stop("Unhandled zookeeper exception removing leader node: "
-          + ke.getMessage());
-    } catch (InterruptedException e) {
-      watcher.abort("Unhandled zookeeper exception removing leader node", e);
-      candidate.stop("Unhandled zookeeper exception removing leader node: "
-          + e.getMessage());
-    }
-  }
-
-  public boolean hasLeader() {
-    return leaderExists.get();
-  }
-}


[09/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
index d261993..389dea7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
@@ -23,12 +23,12 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -42,12 +42,12 @@ class HFileArchiveManager {
 
   private final String archiveZnode;
   private static final Log LOG = LogFactory.getLog(HFileArchiveManager.class);
-  private final ZooKeeperWatcher zooKeeper;
+  private final ZKWatcher zooKeeper;
   private volatile boolean stopped = false;
 
   public HFileArchiveManager(Connection connection, Configuration conf)
       throws ZooKeeperConnectionException, IOException {
-    this.zooKeeper = new ZooKeeperWatcher(conf, "hfileArchiveManager-on-" + connection.toString(),
+    this.zooKeeper = new ZKWatcher(conf, "hfileArchiveManager-on-" + connection.toString(),
         connection);
     this.archiveZnode = ZKTableArchiveClient.getArchiveZNode(this.zooKeeper.getConfiguration(),
       this.zooKeeper);
@@ -104,7 +104,7 @@ class HFileArchiveManager {
    * @param table table name on which to enable archiving
    * @throws KeeperException
    */
-  private void enable(ZooKeeperWatcher zooKeeper, byte[] table)
+  private void enable(ZKWatcher zooKeeper, byte[] table)
       throws KeeperException {
     LOG.debug("Ensuring archiving znode exists");
     ZKUtil.createAndFailSilent(zooKeeper, archiveZnode);
@@ -123,7 +123,7 @@ class HFileArchiveManager {
    * @param table name of the table to disable
    * @throws KeeperException if an unexpected ZK connection issues occurs
    */
-  private void disable(ZooKeeperWatcher zooKeeper, byte[] table) throws KeeperException {
+  private void disable(ZKWatcher zooKeeper, byte[] table) throws KeeperException {
     // ensure the latest state of the archive node is found
     zooKeeper.sync(archiveZnode);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java
index 72c292f..73b50a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java
@@ -22,12 +22,12 @@ import java.util.List;
 
 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.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -38,14 +38,14 @@ import org.apache.zookeeper.KeeperException;
  * archive.
  */
 @InterfaceAudience.Private
-public class TableHFileArchiveTracker extends ZooKeeperListener {
+public class TableHFileArchiveTracker extends ZKListener {
   private static final Log LOG = LogFactory.getLog(TableHFileArchiveTracker.class);
   public static final String HFILE_ARCHIVE_ZNODE_PARENT = "hfilearchive";
   private HFileArchiveTableMonitor monitor;
   private String archiveHFileZNode;
   private boolean stopped = false;
 
-  private TableHFileArchiveTracker(ZooKeeperWatcher watcher, HFileArchiveTableMonitor monitor) {
+  private TableHFileArchiveTracker(ZKWatcher watcher, HFileArchiveTableMonitor monitor) {
     super(watcher);
     watcher.registerListener(this);
     this.monitor = monitor;
@@ -235,7 +235,7 @@ public class TableHFileArchiveTracker extends ZooKeeperListener {
    */
   public static TableHFileArchiveTracker create(Configuration conf)
       throws ZooKeeperConnectionException, IOException {
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "hfileArchiveCleaner", null);
+    ZKWatcher zkw = new ZKWatcher(conf, "hfileArchiveCleaner", null);
     return create(zkw, new HFileArchiveTableMonitor());
   }
 
@@ -247,12 +247,12 @@ public class TableHFileArchiveTracker extends ZooKeeperListener {
    * @return ZooKeeper tracker to monitor for this server if this server should archive hfiles for a
    *         given table
    */
-  private static TableHFileArchiveTracker create(ZooKeeperWatcher zkw,
+  private static TableHFileArchiveTracker create(ZKWatcher zkw,
       HFileArchiveTableMonitor monitor) {
     return new TableHFileArchiveTracker(zkw, monitor);
   }
 
-  public ZooKeeperWatcher getZooKeeperWatcher() {
+  public ZKWatcher getZooKeeperWatcher() {
     return this.watcher;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
index 9a7903a..39d9543 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java
@@ -19,13 +19,13 @@ package org.apache.hadoop.hbase.backup.example;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -148,7 +148,7 @@ public class ZKTableArchiveClient extends Configured {
    * @param zooKeeper zookeeper to used for building the full path
    * @return get the znode for long-term archival of a table for
    */
-  public static String getArchiveZNode(Configuration conf, ZooKeeperWatcher zooKeeper) {
+  public static String getArchiveZNode(Configuration conf, ZKWatcher zooKeeper) {
     return ZNodePaths.joinZNode(zooKeeper.znodePaths.baseZNode, conf.get(
       ZOOKEEPER_ZNODE_HFILE_ARCHIVE_KEY, TableHFileArchiveTracker.HFILE_ARCHIVE_ZNODE_PARENT));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
index c19c3a5..8a07b4b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
@@ -43,12 +43,12 @@ import org.apache.hadoop.hbase.master.SplitLogManager.Task;
 import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.hadoop.hbase.zookeeper.ZKMetadata;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 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.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.AsyncCallback;
@@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe
  * {@link SplitLogManagerCoordination}
  */
 @InterfaceAudience.Private
-public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
+public class ZKSplitLogManagerCoordination extends ZKListener implements
     SplitLogManagerCoordination {
 
   public static final int DEFAULT_TIMEOUT = 120000;
@@ -85,7 +85,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
 
   public boolean ignoreZKDeleteForTesting = false;
 
-  public ZKSplitLogManagerCoordination(Configuration conf, ZooKeeperWatcher watcher) {
+  public ZKSplitLogManagerCoordination(Configuration conf, ZKWatcher watcher) {
     super(watcher);
     this.conf = conf;
     taskFinisher = new TaskFinisher() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
index 10e2642..91b617f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
@@ -19,17 +19,15 @@ package org.apache.hadoop.hbase.coordination;
 
 import java.io.IOException;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
 import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
 import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
 import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -37,7 +35,7 @@ import org.apache.zookeeper.KeeperException;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
 public class ZkCoordinatedStateManager implements CoordinatedStateManager {
-  protected ZooKeeperWatcher watcher;
+  protected ZKWatcher watcher;
   protected SplitLogWorkerCoordination splitLogWorkerCoordination;
   protected SplitLogManagerCoordination splitLogManagerCoordination;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
index ef87498..e64907c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
@@ -42,12 +42,12 @@ import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.hadoop.hbase.zookeeper.ZKMetadata;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 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.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.AsyncCallback;
@@ -60,7 +60,7 @@ import org.apache.zookeeper.data.Stat;
  *
  */
 @InterfaceAudience.Private
-public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
+public class ZkSplitLogWorkerCoordination extends ZKListener implements
     SplitLogWorkerCoordination {
 
   private static final Log LOG = LogFactory.getLog(ZkSplitLogWorkerCoordination.class);
@@ -85,13 +85,13 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
 
   private final ServerName serverName;
 
-  public ZkSplitLogWorkerCoordination(ServerName serverName, ZooKeeperWatcher watcher) {
+  public ZkSplitLogWorkerCoordination(ServerName serverName, ZKWatcher watcher) {
     super(watcher);
     this.serverName = serverName;
   }
 
   /**
-   * Override handler from {@link ZooKeeperListener}
+   * Override handler from {@link ZKListener}
    */
   @Override
   public void nodeChildrenChanged(String path) {
@@ -107,7 +107,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
   }
 
   /**
-   * Override handler from {@link ZooKeeperListener}
+   * Override handler from {@link ZKListener}
    */
   @Override
   public void nodeDataChanged(String path) {
@@ -353,7 +353,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
    * @param taskZKVersion version of the task in zk
    * @return non-negative integer value when task can be owned by current region server otherwise -1
    */
-  protected static int attemptToOwnTask(boolean isFirstTime, ZooKeeperWatcher zkw,
+  protected static int attemptToOwnTask(boolean isFirstTime, ZKWatcher zkw,
       ServerName server, String task, int taskZKVersion) {
     int latestZKVersion = FAILED_TO_OWN_TASK;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
index f154347..ce9f290 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
@@ -23,6 +23,10 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
@@ -30,11 +34,7 @@ import org.apache.hadoop.hbase.ZNodeClearer;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-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.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -51,7 +51,7 @@ import org.apache.zookeeper.KeeperException;
  * the active master of the cluster.
  */
 @InterfaceAudience.Private
-public class ActiveMasterManager extends ZooKeeperListener {
+public class ActiveMasterManager extends ZKListener {
   private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
 
   final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
@@ -66,7 +66,7 @@ public class ActiveMasterManager extends ZooKeeperListener {
    * @param sn ServerName
    * @param master In an instance of a Master.
    */
-  ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
+  ActiveMasterManager(ZKWatcher watcher, ServerName sn, Server master) {
     super(watcher);
     watcher.registerListener(this);
     this.sn = sn;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java
new file mode 100644
index 0000000..81a8b55
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java
@@ -0,0 +1,135 @@
+/**
+ * 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.master;
+
+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.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.ServerName;
+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 ZKListener {
+  private static final Log LOG = LogFactory.getLog(DrainingServerTracker.class);
+
+  private ServerManager serverManager;
+  private final NavigableSet<ServerName> drainingServers = new TreeSet<>();
+  private Abortable abortable;
+
+  public DrainingServerTracker(ZKWatcher 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/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index cad77e5..1967296 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import com.google.common.base.Enums;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
@@ -175,17 +174,14 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.util.ZKDataMigrator;
-import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterMaintenanceModeTracker;
 import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
-import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
-import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.eclipse.jetty.server.Server;
@@ -2637,7 +2633,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ZooKeeperWatcher getZooKeeper() {
+  public ZKWatcher getZooKeeper() {
     return zooKeeper;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
index 677a2a0..7abf02c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 
@@ -85,7 +85,7 @@ public class MasterMetaBootstrap {
   }
 
   private void unassignExcessMetaReplica(int numMetaReplicasConfigured) {
-    final ZooKeeperWatcher zooKeeper = master.getZooKeeper();
+    final ZKWatcher zooKeeper = master.getZooKeeper();
     // unassign the unneeded replicas (for e.g., if the previous master was configured
     // with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
index d13fb76..6a9b572 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
 /**
  * Impl for exposing HMaster Information through JMX
@@ -70,7 +70,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
 
   @Override
   public String getZookeeperQuorum() {
-    ZooKeeperWatcher zk = master.getZooKeeper();
+    ZKWatcher zk = master.getZooKeeper();
     if (zk == null) {
       return "";
     }
@@ -100,7 +100,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
     }
     return StringUtils.join(serverManager.getOnlineServers().keySet(), ";");
   }
-  
+
   @Override
   public int getNumRegionServers() {
     ServerManager serverManager = this.master.getServerManager();
@@ -119,7 +119,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
     return StringUtils.join(serverManager.getDeadServers().copyServerNames(), ";");
   }
 
-  
+
   @Override
   public int getNumDeadRegionServers() {
     ServerManager serverManager = this.master.getServerManager();

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
new file mode 100644
index 0000000..0e9351d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
@@ -0,0 +1,164 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+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.hadoop.hbase.zookeeper.ZKListener;
+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.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+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 ZKListener {
+  private static final Log LOG = LogFactory.getLog(RegionServerTracker.class);
+  private NavigableMap<ServerName, RegionServerInfo> regionServers = new TreeMap<>();
+  private ServerManager serverManager;
+  private MasterServices server;
+
+  public RegionServerTracker(ZKWatcher 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/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index f19995f..79ffc8a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClockOutOfSyncException;
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionLoad;
@@ -59,7 +58,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 
@@ -507,7 +506,7 @@ public class ServerManager {
   void letRegionServersShutdown() {
     long previousLogTime = 0;
     ServerName sn = master.getServerName();
-    ZooKeeperWatcher zkw = master.getZooKeeper();
+    ZKWatcher zkw = master.getZooKeeper();
     int onlineServersCt;
     while ((onlineServersCt = onlineServers.size()) > 0){
 
@@ -554,7 +553,7 @@ public class ServerManager {
     }
   }
 
-  private List<String> getRegionServersInZK(final ZooKeeperWatcher zkw)
+  private List<String> getRegionServersInZK(final ZKWatcher zkw)
   throws KeeperException {
     return ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java
new file mode 100644
index 0000000..55287d2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java
@@ -0,0 +1,153 @@
+/**
+ * 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.master;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+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.yetus.audience.InterfaceAudience;
+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(ZKWatcher 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 ZKNodeTracker {
+
+    public SwitchStateTracker(ZKWatcher 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/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
index 97d3080..3f7bd74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
@@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
@@ -39,7 +40,6 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -48,12 +48,12 @@ import org.apache.zookeeper.KeeperException;
 @InterfaceAudience.Private
 public class ReplicationZKNodeCleaner {
   private static final Log LOG = LogFactory.getLog(ReplicationZKNodeCleaner.class);
-  private final ZooKeeperWatcher zkw;
+  private final ZKWatcher zkw;
   private final ReplicationQueuesClient queuesClient;
   private final ReplicationPeers replicationPeers;
   private final ReplicationQueueDeletor queueDeletor;
 
-  public ReplicationZKNodeCleaner(Configuration conf, ZooKeeperWatcher zkw, Abortable abortable)
+  public ReplicationZKNodeCleaner(Configuration conf, ZKWatcher zkw, Abortable abortable)
       throws IOException {
     try {
       this.zkw = zkw;
@@ -130,7 +130,7 @@ public class ReplicationZKNodeCleaner {
 
   private class ReplicationQueueDeletor extends ReplicationStateZKBase {
 
-    public ReplicationQueueDeletor(ZooKeeperWatcher zk, Configuration conf, Abortable abortable) {
+    public ReplicationQueueDeletor(ZKWatcher zk, Configuration conf, Abortable abortable) {
       super(zk, conf, abortable);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
index df94ffe..7cfaefd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -39,7 +40,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 /**
  * Manages and performs all replication admin operations.
@@ -49,11 +49,11 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 public class ReplicationManager {
 
   private final Configuration conf;
-  private final ZooKeeperWatcher zkw;
+  private final ZKWatcher zkw;
   private final ReplicationQueuesClient replicationQueuesClient;
   private final ReplicationPeers replicationPeers;
 
-  public ReplicationManager(Configuration conf, ZooKeeperWatcher zkw, Abortable abortable)
+  public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable)
       throws IOException {
     this.conf = conf;
     this.zkw = zkw;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
index 6985591..609ce8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
@@ -24,12 +24,12 @@ import java.util.List;
 
 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.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -41,7 +41,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
   private ZKProcedureUtil zkProc = null;
   protected ProcedureCoordinator coordinator = null;  // if started this should be non-null
 
-  ZooKeeperWatcher watcher;
+  ZKWatcher watcher;
   String procedureType;
   String coordName;
 
@@ -52,7 +52,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
    * @param coordName name of the node running the coordinator
    * @throws KeeperException if an unexpected zk error occurs
    */
-  public ZKProcedureCoordinator(ZooKeeperWatcher watcher,
+  public ZKProcedureCoordinator(ZKWatcher watcher,
       String procedureClass, String coordName) {
     this.watcher = watcher;
     this.procedureType = procedureClass;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
index 36f4f44..45e6760 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
@@ -23,13 +23,13 @@ import java.util.List;
 
 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.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -61,12 +61,12 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
 
   /**
    * Must call {@link #start(String, ProcedureMember)} before this can be used.
-   * @param watcher {@link ZooKeeperWatcher} to be owned by <tt>this</tt>. Closed via
+   * @param watcher {@link ZKWatcher} to be owned by <tt>this</tt>. Closed via
    *          {@link #close()}.
    * @param procType name of the znode describing the procedure type
    * @throws KeeperException if we can't reach zookeeper
    */
-  public ZKProcedureMemberRpcs(final ZooKeeperWatcher watcher, final String procType)
+  public ZKProcedureMemberRpcs(final ZKWatcher watcher, final String procType)
       throws KeeperException {
     this.zkController = new ZKProcedureUtil(watcher, procType) {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
index 24693cd..0349290 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
@@ -23,11 +23,11 @@ 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.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;
 
 /**
@@ -50,7 +50,7 @@ import org.apache.zookeeper.KeeperException;
  */
 @InterfaceAudience.Private
 public abstract class ZKProcedureUtil
-    extends ZooKeeperListener implements Closeable {
+    extends ZKListener implements Closeable {
 
   private static final Log LOG = LogFactory.getLog(ZKProcedureUtil.class);
 
@@ -73,7 +73,7 @@ public abstract class ZKProcedureUtil
    * @param procDescription name of the znode describing the procedure to run
    * @throws KeeperException when the procedure znodes cannot be created
    */
-  public ZKProcedureUtil(ZooKeeperWatcher watcher, String procDescription)
+  public ZKProcedureUtil(ZKWatcher watcher, String procDescription)
       throws KeeperException {
     super(watcher);
     // make sure we are listening for events
@@ -157,7 +157,7 @@ public abstract class ZKProcedureUtil
     return ZNodePaths.joinZNode(controller.abortZnode, opInstanceName);
   }
 
-  public ZooKeeperWatcher getWatcher() {
+  public ZKWatcher getWatcher() {
     return watcher;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
index ea34714..d328561 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
@@ -48,9 +48,8 @@ import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
 import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -320,7 +319,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
   @Override
   public void initialize(RegionServerServices rss) throws KeeperException {
     this.rss = rss;
-    ZooKeeperWatcher zkw = rss.getZooKeeper();
+    ZKWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
       MasterFlushTableProcedureManager.FLUSH_TABLE_PROCEDURE_SIGNATURE);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 4c34fe0..03657c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -162,10 +162,10 @@ import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
 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.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -380,7 +380,7 @@ public class HRegionServer extends HasThread implements
   final AtomicBoolean online = new AtomicBoolean(false);
 
   // zookeeper connection and watcher
-  protected ZooKeeperWatcher zooKeeper;
+  protected ZKWatcher zooKeeper;
 
   // master address tracker
   private MasterAddressTracker masterAddressTracker;
@@ -616,7 +616,7 @@ public class HRegionServer extends HasThread implements
       // Some unit tests don't need a cluster, so no zookeeper at all
       if (!conf.getBoolean("hbase.testing.nocluster", false)) {
         // Open connection to zookeeper and set primary watcher
-        zooKeeper = new ZooKeeperWatcher(conf, getProcessName() + ":" +
+        zooKeeper = new ZKWatcher(conf, getProcessName() + ":" +
           rpcServices.isa.getPort(), this, canCreateBaseZNode());
 
         // If no master in cluster, skip trying to track one or look for a cluster status.
@@ -905,7 +905,7 @@ public class HRegionServer extends HasThread implements
    * @throws IOException any IO exception, plus if the RS is stopped
    * @throws InterruptedException
    */
-  private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
+  private void blockAndCheckIfStopped(ZKNodeTracker tracker)
       throws IOException, InterruptedException {
     while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
       if (this.stopped) {
@@ -2886,7 +2886,7 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public ZooKeeperWatcher getZooKeeper() {
+  public ZKWatcher getZooKeeper() {
     return zooKeeper;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index 7d7833b..75d8e56 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WALProvider;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hdfs.DFSHedgedReadMetrics;
 import org.apache.hadoop.metrics2.MetricsExecutor;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -187,7 +187,7 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public String getZookeeperQuorum() {
-    ZooKeeperWatcher zk = regionServer.getZooKeeper();
+    ZKWatcher zk = regionServer.getZooKeeper();
     if (zk == null) {
       return "";
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
index a4b4387..6a7d83e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.DaemonThreadFactory;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -53,11 +54,9 @@ import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
 import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -394,7 +393,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
   @Override
   public void initialize(RegionServerServices rss) throws KeeperException {
     this.rss = rss;
-    ZooKeeperWatcher zkw = rss.getZooKeeper();
+    ZKWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
         SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
index a34bedd..4985b82 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
@@ -26,13 +26,13 @@ import java.util.UUID;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.AuthFailedException;
 import org.apache.zookeeper.KeeperException.ConnectionLossException;
@@ -50,7 +50,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
 
   private static final Log LOG = LogFactory.getLog(HBaseReplicationEndpoint.class);
 
-  private ZooKeeperWatcher zkw = null; // FindBugs: MT_CORRECTNESS
+  private ZKWatcher zkw = null; // FindBugs: MT_CORRECTNESS
 
   private List<ServerName> regionServers = new ArrayList<>(0);
   private long lastRegionServerUpdate;
@@ -123,7 +123,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
    * Get the ZK connection to this peer
    * @return zk connection
    */
-  protected ZooKeeperWatcher getZkw() {
+  protected ZKWatcher getZkw() {
     return zkw;
   }
 
@@ -133,7 +133,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
    */
   void reloadZkWatcher() throws IOException {
     if (zkw != null) zkw.close();
-    zkw = new ZooKeeperWatcher(ctx.getConfiguration(),
+    zkw = new ZKWatcher(ctx.getConfiguration(),
         "connection to cluster: " + ctx.getPeerId(), this);
     getZkw().registerListener(new PeerRegionServerListener(this));
   }
@@ -155,7 +155,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
    * @param zkw zk connection to use
    * @return list of region server addresses or an empty list if the slave is unavailable
    */
-  protected static List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
+  protected static List<ServerName> fetchSlavesAddresses(ZKWatcher zkw)
       throws KeeperException {
     List<String> children = ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.znodePaths.rsZNode);
     if (children == null) {
@@ -210,7 +210,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
   /**
    * Tracks changes to the list of region servers in a peer's cluster.
    */
-  public static class PeerRegionServerListener extends ZooKeeperListener {
+  public static class PeerRegionServerListener extends ZKListener {
 
     private final HBaseReplicationEndpoint replicationEndpoint;
     private final String regionServerListNode;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
index a14bd01..5972734 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
@@ -28,14 +28,13 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -45,7 +44,7 @@ import org.apache.zookeeper.KeeperException;
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
 public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
   private static final Log LOG = LogFactory.getLog(ReplicationHFileCleaner.class);
-  private ZooKeeperWatcher zkw;
+  private ZKWatcher zkw;
   private ReplicationQueuesClient rqc;
   private boolean stopped = false;
 
@@ -130,14 +129,14 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     // I can close myself when time comes.
     Configuration conf = new Configuration(config);
     try {
-      setConf(conf, new ZooKeeperWatcher(conf, "replicationHFileCleaner", null));
+      setConf(conf, new ZKWatcher(conf, "replicationHFileCleaner", null));
     } catch (IOException e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
 
   @VisibleForTesting
-  public void setConf(Configuration conf, ZooKeeperWatcher zk) {
+  public void setConf(Configuration conf, ZKWatcher zk) {
     super.setConf(conf);
     try {
       initReplicationQueuesClient(conf, zk);
@@ -146,7 +145,7 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     }
   }
 
-  private void initReplicationQueuesClient(Configuration conf, ZooKeeperWatcher zk)
+  private void initReplicationQueuesClient(Configuration conf, ZKWatcher zk)
       throws Exception {
     this.zkw = zk;
     this.rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index 3dcb332..57ed842 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.replication.master;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 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.fs.FileStatus;
@@ -30,7 +31,6 @@ import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -48,7 +48,7 @@ import org.apache.zookeeper.KeeperException;
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
 public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
   private static final Log LOG = LogFactory.getLog(ReplicationLogCleaner.class);
-  private ZooKeeperWatcher zkw;
+  private ZKWatcher zkw;
   private ReplicationQueuesClient replicationQueues;
   private boolean stopped = false;
   private Set<String> wals;
@@ -101,14 +101,14 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     // I can close myself when comes time.
     Configuration conf = new Configuration(config);
     try {
-      setConf(conf, new ZooKeeperWatcher(conf, "replicationLogCleaner", null));
+      setConf(conf, new ZKWatcher(conf, "replicationLogCleaner", null));
     } catch (IOException e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
 
   @VisibleForTesting
-  public void setConf(Configuration conf, ZooKeeperWatcher zk) {
+  public void setConf(Configuration conf, ZKWatcher zk) {
     super.setConf(conf);
     try {
       this.zkw = zk;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
index d094d1c..f442495 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.apache.zookeeper.KeeperException;
@@ -47,7 +47,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase {
 
   private static final Log LOG = LogFactory.getLog(TableCFsUpdater.class);
 
-  public TableCFsUpdater(ZooKeeperWatcher zookeeper,
+  public TableCFsUpdater(ZKWatcher zookeeper,
                          Configuration conf, Abortable abortable) {
     super(zookeeper, conf, abortable);
   }
@@ -137,7 +137,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase {
       printUsageAndExit();
     } else if (args[0].equals("update")) {
       Configuration conf = HBaseConfiguration.create();
-      ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "TableCFsUpdater", null);
+      ZKWatcher zkw = new ZKWatcher(conf, "TableCFsUpdater", null);
       try {
         TableCFsUpdater tableCFsUpdater = new TableCFsUpdater(zkw, conf, null);
         tableCFsUpdater.update();

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index 9d38026..fdb5559 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 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.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.zookeeper.KeeperException;
@@ -211,7 +211,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
     ClusterConnection connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
     Admin admin = connection.getAdmin();
 
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
+    ZKWatcher zkw = new ZKWatcher(conf, "DumpReplicationQueues" + System.currentTimeMillis(),
         new WarnOnlyAbortable(), true);
 
     try {
@@ -302,8 +302,8 @@ public class DumpReplicationQueues extends Configured implements Tool {
     return sb.toString();
   }
 
-  public String dumpQueues(ClusterConnection connection, ZooKeeperWatcher zkw, Set<String> peerIds,
-      boolean hdfs) throws Exception {
+  public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set<String> peerIds,
+                           boolean hdfs) throws Exception {
     ReplicationQueuesClient queuesClient;
     ReplicationPeers replicationPeers;
     ReplicationQueues replicationQueues;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 280289c..233159e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -77,7 +77,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
     ReplicationSourceManager manager;
     FileSystem fs;
     Path oldLogDir, logDir, walRootDir;
-    ZooKeeperWatcher zkw;
+    ZKWatcher zkw;
 
     Abortable abortable = new Abortable() {
       @Override
@@ -91,7 +91,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
     };
 
     zkw =
-        new ZooKeeperWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable,
+        new ZKWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable,
             true);
 
     walRootDir = FSUtils.getWALRootDir(conf);
@@ -123,9 +123,9 @@ public class ReplicationSyncUp extends Configured implements Tool {
 
   static class DummyServer implements Server {
     String hostname;
-    ZooKeeperWatcher zkw;
+    ZKWatcher zkw;
 
-    DummyServer(ZooKeeperWatcher zkw) {
+    DummyServer(ZKWatcher zkw) {
       // an unique name in case the first run fails
       hostname = System.currentTimeMillis() + ".SyncUpTool.replication.org";
       this.zkw = zkw;
@@ -141,7 +141,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return zkw;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 6ee9f60..05f9195 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -130,7 +130,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
 import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
@@ -953,7 +953,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
           + " accordingly.");
     }
 
-    ZooKeeperWatcher zk = null;
+    ZKWatcher zk = null;
     if (env instanceof MasterCoprocessorEnvironment) {
       // if running on HMaster
       MasterCoprocessorEnvironment mEnv = (MasterCoprocessorEnvironment)env;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
index 0db05c2..2ba4ac5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
@@ -39,13 +39,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -112,7 +112,7 @@ public class TableAuthManager implements Closeable {
   private ZKPermissionWatcher zkperms;
   private final AtomicLong mtime = new AtomicLong(0L);
 
-  private TableAuthManager(ZooKeeperWatcher watcher, Configuration conf)
+  private TableAuthManager(ZKWatcher watcher, Configuration conf)
       throws IOException {
     this.conf = conf;
 
@@ -734,14 +734,14 @@ public class TableAuthManager implements Closeable {
     return mtime.get();
   }
 
-  private static Map<ZooKeeperWatcher,TableAuthManager> managerMap = new HashMap<>();
+  private static Map<ZKWatcher,TableAuthManager> managerMap = new HashMap<>();
 
   private static Map<TableAuthManager, Integer> refCount = new HashMap<>();
 
   /** Returns a TableAuthManager from the cache. If not cached, constructs a new one. Returned
    * instance should be released back by calling {@link #release(TableAuthManager)}. */
   public synchronized static TableAuthManager getOrCreate(
-      ZooKeeperWatcher watcher, Configuration conf) throws IOException {
+          ZKWatcher watcher, Configuration conf) throws IOException {
     TableAuthManager instance = managerMap.get(watcher);
     if (instance == null) {
       instance = new TableAuthManager(watcher, conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
index 447ba51..d45b5b5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
@@ -20,15 +20,15 @@ package org.apache.hadoop.hbase.security.access;
 
 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.DaemonThreadFactory;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
 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.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 
 import java.io.Closeable;
@@ -41,7 +41,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * Handles synchronization of access control list entries and updates
@@ -53,7 +52,7 @@ import java.util.concurrent.atomic.AtomicReference;
  * trigger updates in the {@link TableAuthManager} permission cache.
  */
 @InterfaceAudience.Private
-public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable {
+public class ZKPermissionWatcher extends ZKListener implements Closeable {
   private static final Log LOG = LogFactory.getLog(ZKPermissionWatcher.class);
   // parent node for permissions lists
   static final String ACL_NODE = "acl";
@@ -63,7 +62,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable
   private final ExecutorService executor;
   private Future<?> childrenChangedFuture;
 
-  public ZKPermissionWatcher(ZooKeeperWatcher watcher,
+  public ZKPermissionWatcher(ZKWatcher watcher,
       TableAuthManager authManager, Configuration conf) {
     super(watcher);
     this.authManager = authManager;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
index 27ea509..aa6b1e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 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.Stoppable;
@@ -35,7 +36,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKLeaderManager;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
@@ -89,8 +89,8 @@ public class AuthenticationTokenSecretManager
    * org.apache.hadoop.hbase.ipc.SecureServer so public access is needed.
    */
   public AuthenticationTokenSecretManager(Configuration conf,
-      ZooKeeperWatcher zk, String serverName,
-      long keyUpdateInterval, long tokenMaxLifetime) {
+                                          ZKWatcher zk, String serverName,
+                                          long keyUpdateInterval, long tokenMaxLifetime) {
     this.zkWatcher = new ZKSecretWatcher(conf, zk, this);
     this.keyUpdateInterval = keyUpdateInterval;
     this.tokenMaxLifetime = tokenMaxLifetime;
@@ -144,9 +144,9 @@ public class AuthenticationTokenSecretManager
     AuthenticationKey masterKey = allKeys.get(identifier.getKeyId());
     if(masterKey == null) {
       if(zkWatcher.getWatcher().isAborted()) {
-        LOG.error("ZooKeeperWatcher is abort");
+        LOG.error("ZKWatcher is abort");
         throw new InvalidToken("Token keys could not be sync from zookeeper"
-            + " because of ZooKeeperWatcher abort");
+            + " because of ZKWatcher abort");
       }
       synchronized (this) {
         if (!leaderElector.isAlive() || leaderElector.isStopped()) {
@@ -254,7 +254,7 @@ public class AuthenticationTokenSecretManager
       }
     }
   }
-  
+
   synchronized boolean isCurrentKeyRolled() {
     return currentKey != null;
   }
@@ -297,7 +297,7 @@ public class AuthenticationTokenSecretManager
     private boolean isMaster = false;
     private ZKLeaderManager zkLeader;
 
-    public LeaderElector(ZooKeeperWatcher watcher, String serverName) {
+    public LeaderElector(ZKWatcher watcher, String serverName) {
       setDaemon(true);
       setName("ZKSecretWatcher-leaderElector");
       zkLeader = new ZKLeaderManager(watcher,


[10/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
deleted file mode 100644
index a5b084b..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
+++ /dev/null
@@ -1,251 +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.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the availability and value of a single ZooKeeper node.
- *
- * <p>Utilizes the {@link ZooKeeperListener} interface to get the necessary
- * ZooKeeper events related to the node.
- *
- * <p>This is the base class used by trackers in both the Master and
- * RegionServers.
- */
-@InterfaceAudience.Private
-public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
-  // LOG is being used in subclasses, hence keeping it protected
-  protected static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);
-  /** Path of node being tracked */
-  protected final String node;
-
-  /** Data of the node being tracked */
-  private byte [] data;
-
-  /** Used to abort if a fatal error occurs */
-  protected final Abortable abortable;
-
-  private boolean stopped = false;
-
-  /**
-   * Constructs a new ZK node tracker.
-   *
-   * <p>After construction, use {@link #start} to kick off tracking.
-   *
-   * @param watcher
-   * @param node
-   * @param abortable
-   */
-  public ZooKeeperNodeTracker(ZooKeeperWatcher watcher, String node,
-      Abortable abortable) {
-    super(watcher);
-    this.node = node;
-    this.abortable = abortable;
-    this.data = null;
-  }
-
-  /**
-   * Starts the tracking of the node in ZooKeeper.
-   *
-   * <p>Use {@link #blockUntilAvailable()} to block until the node is available
-   * or {@link #getData(boolean)} to get the data of the node if it is available.
-   */
-  public synchronized void start() {
-    this.watcher.registerListener(this);
-    try {
-      if(ZKUtil.watchAndCheckExists(watcher, node)) {
-        byte [] data = ZKUtil.getDataAndWatch(watcher, node);
-        if(data != null) {
-          this.data = data;
-        } else {
-          // It existed but now does not, try again to ensure a watch is set
-          LOG.debug("Try starting again because there is no data from " + node);
-          start();
-        }
-      }
-    } catch (KeeperException e) {
-      abortable.abort("Unexpected exception during initialization, aborting", e);
-    }
-  }
-
-  public synchronized void stop() {
-    this.stopped = true;
-    notifyAll();
-  }
-
-  /**
-   * Gets the data of the node, blocking until the node is available.
-   *
-   * @return data of the node
-   * @throws InterruptedException if the waiting thread is interrupted
-   */
-  public synchronized byte [] blockUntilAvailable()
-  throws InterruptedException {
-    return blockUntilAvailable(0, false);
-  }
-
-  /**
-   * Gets the data of the node, blocking until the node is available or the
-   * specified timeout has elapsed.
-   *
-   * @param timeout maximum time to wait for the node data to be available,
-   * n milliseconds.  Pass 0 for no timeout.
-   * @return data of the node
-   * @throws InterruptedException if the waiting thread is interrupted
-   */
-  public synchronized byte [] blockUntilAvailable(long timeout, boolean refresh)
-  throws InterruptedException {
-    if (timeout < 0) throw new IllegalArgumentException();
-    boolean notimeout = timeout == 0;
-    long startTime = System.currentTimeMillis();
-    long remaining = timeout;
-    if (refresh) {
-      try {
-        // This does not create a watch if the node does not exists
-        this.data = ZKUtil.getDataAndWatch(watcher, node);
-      } catch(KeeperException e) {
-        // We use to abort here, but in some cases the abort is ignored (
-        //  (empty Abortable), so it's better to log...
-        LOG.warn("Unexpected exception handling blockUntilAvailable", e);
-        abortable.abort("Unexpected exception handling blockUntilAvailable", e);
-      }
-    }
-    boolean nodeExistsChecked = (!refresh ||data!=null);
-    while (!this.stopped && (notimeout || remaining > 0) && this.data == null) {
-      if (!nodeExistsChecked) {
-        try {
-          nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1);
-        } catch (KeeperException e) {
-          LOG.warn(
-            "Got exception while trying to check existence in  ZooKeeper" +
-            " of the node: "+node+", retrying if timeout not reached",e );
-        }
-
-        // It did not exists, and now it does.
-        if (nodeExistsChecked){
-          LOG.debug("Node " + node + " now exists, resetting a watcher");
-          try {
-            // This does not create a watch if the node does not exists
-            this.data = ZKUtil.getDataAndWatch(watcher, node);
-          } catch (KeeperException e) {
-            LOG.warn("Unexpected exception handling blockUntilAvailable", e);
-            abortable.abort("Unexpected exception handling blockUntilAvailable", e);
-          }
-        }
-      }
-      // We expect a notification; but we wait with a
-      //  a timeout to lower the impact of a race condition if any
-      wait(100);
-      remaining = timeout - (System.currentTimeMillis() - startTime);
-    }
-    return this.data;
-  }
-
-  /**
-   * Gets the data of the node.
-   *
-   * <p>If the node is currently available, the most up-to-date known version of
-   * the data is returned.  If the node is not currently available, null is
-   * returned.
-   * @param refresh whether to refresh the data by calling ZK directly.
-   * @return data of the node, null if unavailable
-   */
-  public synchronized byte [] getData(boolean refresh) {
-    if (refresh) {
-      try {
-        this.data = ZKUtil.getDataAndWatch(watcher, node);
-      } catch(KeeperException e) {
-        abortable.abort("Unexpected exception handling getData", e);
-      }
-    }
-    return this.data;
-  }
-
-  public String getNode() {
-    return this.node;
-  }
-
-  @Override
-  public synchronized void nodeCreated(String path) {
-    if (!path.equals(node)) return;
-    try {
-      byte [] data = ZKUtil.getDataAndWatch(watcher, node);
-      if (data != null) {
-        this.data = data;
-        notifyAll();
-      } else {
-        nodeDeleted(path);
-      }
-    } catch(KeeperException e) {
-      abortable.abort("Unexpected exception handling nodeCreated event", e);
-    }
-  }
-
-  @Override
-  public synchronized void nodeDeleted(String path) {
-    if(path.equals(node)) {
-      try {
-        if(ZKUtil.watchAndCheckExists(watcher, node)) {
-          nodeCreated(path);
-        } else {
-          this.data = null;
-        }
-      } catch(KeeperException e) {
-        abortable.abort("Unexpected exception handling nodeDeleted event", e);
-      }
-    }
-  }
-
-  @Override
-  public synchronized void nodeDataChanged(String path) {
-    if(path.equals(node)) {
-      nodeCreated(path);
-    }
-  }
-  
-  /**
-   * Checks if the baseznode set as per the property 'zookeeper.znode.parent'
-   * exists.
-   * @return true if baseznode exists.
-   *         false if doesnot exists.
-   */
-  public boolean checkIfBaseNodeAvailable() {
-    try {
-      if (ZKUtil.checkExists(watcher, watcher.znodePaths.baseZNode) == -1) {
-        return false;
-      }
-    } catch (KeeperException e) {
-      abortable.abort("Exception while checking if basenode (" + watcher.znodePaths.baseZNode
-          + ") exists in ZooKeeper.",
-        e);
-    }
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    return "ZooKeeperNodeTracker{" +
-        "node='" + node + ", stopped=" + stopped + '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
deleted file mode 100644
index 5d10cdf..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ /dev/null
@@ -1,638 +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.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.CountDownLatch;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.security.Superusers;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooDefs.Perms;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.data.Stat;
-
-/**
- * Acts as the single ZooKeeper Watcher.  One instance of this is instantiated
- * for each Master, RegionServer, and client process.
- *
- * <p>This is the only class that implements {@link Watcher}.  Other internal
- * classes which need to be notified of ZooKeeper events must register with
- * the local instance of this watcher via {@link #registerListener}.
- *
- * <p>This class also holds and manages the connection to ZooKeeper.  Code to
- * deal with connection related events and exceptions are handled here.
- */
-@InterfaceAudience.Private
-public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
-  private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);
-
-  // Identifier for this watcher (for logging only).  It is made of the prefix
-  // passed on construction and the zookeeper sessionid.
-  private String prefix;
-  private String identifier;
-
-  // zookeeper quorum
-  private String quorum;
-
-  // zookeeper connection
-  private final RecoverableZooKeeper recoverableZooKeeper;
-
-  // abortable in case of zk failure
-  protected Abortable abortable;
-  // Used if abortable is null
-  private boolean aborted = false;
-
-  public final ZNodePaths znodePaths;
-
-  // listeners to be notified
-  private final List<ZooKeeperListener> listeners = new CopyOnWriteArrayList<>();
-
-  // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
-  // negotiation to complete
-  public CountDownLatch saslLatch = new CountDownLatch(1);
-
-
-
-  private final Configuration conf;
-
-  /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */
-  private static final Pattern NAME_PATTERN = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
-
-  /**
-   * Instantiate a ZooKeeper connection and watcher.
-   * @param identifier string that is passed to RecoverableZookeeper to be used as
-   * identifier for this instance. Use null for default.
-   * @throws IOException
-   * @throws ZooKeeperConnectionException
-   */
-  public ZooKeeperWatcher(Configuration conf, String identifier,
-      Abortable abortable) throws ZooKeeperConnectionException, IOException {
-    this(conf, identifier, abortable, false);
-  }
-
-  /**
-   * Instantiate a ZooKeeper connection and watcher.
-   * @param conf
-   * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
-   *          this instance. Use null for default.
-   * @param abortable Can be null if there is on error there is no host to abort: e.g. client
-   *          context.
-   * @param canCreateBaseZNode
-   * @throws IOException
-   * @throws ZooKeeperConnectionException
-   */
-  public ZooKeeperWatcher(Configuration conf, String identifier,
-      Abortable abortable, boolean canCreateBaseZNode)
-  throws IOException, ZooKeeperConnectionException {
-    this.conf = conf;
-    this.quorum = ZKConfig.getZKQuorumServersString(conf);
-    this.prefix = identifier;
-    // Identifier will get the sessionid appended later below down when we
-    // handle the syncconnect event.
-    this.identifier = identifier + "0x0";
-    this.abortable = abortable;
-    this.znodePaths = new ZNodePaths(conf);
-    PendingWatcher pendingWatcher = new PendingWatcher();
-    this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier);
-    pendingWatcher.prepare(this);
-    if (canCreateBaseZNode) {
-      try {
-        createBaseZNodes();
-      } catch (ZooKeeperConnectionException zce) {
-        try {
-          this.recoverableZooKeeper.close();
-        } catch (InterruptedException ie) {
-          LOG.debug("Encountered InterruptedException when closing " + this.recoverableZooKeeper);
-          Thread.currentThread().interrupt();
-        }
-        throw zce;
-      }
-    }
-  }
-
-  private void createBaseZNodes() throws ZooKeeperConnectionException {
-    try {
-      // Create all the necessary "directories" of znodes
-      ZKUtil.createWithParents(this, znodePaths.baseZNode);
-      ZKUtil.createAndFailSilent(this, znodePaths.rsZNode);
-      ZKUtil.createAndFailSilent(this, znodePaths.drainingZNode);
-      ZKUtil.createAndFailSilent(this, znodePaths.tableZNode);
-      ZKUtil.createAndFailSilent(this, znodePaths.splitLogZNode);
-      ZKUtil.createAndFailSilent(this, znodePaths.backupMasterAddressesZNode);
-      ZKUtil.createAndFailSilent(this, znodePaths.tableLockZNode);
-      ZKUtil.createAndFailSilent(this, znodePaths.masterMaintZNode);
-    } catch (KeeperException e) {
-      throw new ZooKeeperConnectionException(
-          prefix("Unexpected KeeperException creating base node"), e);
-    }
-  }
-
-  /** Returns whether the znode is supposed to be readable by the client
-   * and DOES NOT contain sensitive information (world readable).*/
-  public boolean isClientReadable(String node) {
-    // Developer notice: These znodes are world readable. DO NOT add more znodes here UNLESS
-    // all clients need to access this data to work. Using zk for sharing data to clients (other
-    // than service lookup case is not a recommended design pattern.
-    return
-        node.equals(znodePaths.baseZNode) ||
-        znodePaths.isAnyMetaReplicaZNode(node) ||
-        node.equals(znodePaths.masterAddressZNode) ||
-        node.equals(znodePaths.clusterIdZNode)||
-        node.equals(znodePaths.rsZNode) ||
-        // /hbase/table and /hbase/table/foo is allowed, /hbase/table-lock is not
-        node.equals(znodePaths.tableZNode) ||
-        node.startsWith(znodePaths.tableZNode + "/");
-  }
-
-  /**
-   * On master start, we check the znode ACLs under the root directory and set the ACLs properly
-   * if needed. If the cluster goes from an unsecure setup to a secure setup, this step is needed
-   * so that the existing znodes created with open permissions are now changed with restrictive
-   * perms.
-   */
-  public void checkAndSetZNodeAcls() {
-    if (!ZKUtil.isSecureZooKeeper(getConfiguration())) {
-      LOG.info("not a secure deployment, proceeding");
-      return;
-    }
-
-    // Check the base znodes permission first. Only do the recursion if base znode's perms are not
-    // correct.
-    try {
-      List<ACL> actualAcls = recoverableZooKeeper.getAcl(znodePaths.baseZNode, new Stat());
-
-      if (!isBaseZnodeAclSetup(actualAcls)) {
-        LOG.info("setting znode ACLs");
-        setZnodeAclsRecursive(znodePaths.baseZNode);
-      }
-    } catch(KeeperException.NoNodeException nne) {
-      return;
-    } catch(InterruptedException ie) {
-      interruptedExceptionNoThrow(ie, false);
-    } catch (IOException|KeeperException e) {
-      LOG.warn("Received exception while checking and setting zookeeper ACLs", e);
-    }
-  }
-
-  /**
-   * Set the znode perms recursively. This will do post-order recursion, so that baseZnode ACLs
-   * will be set last in case the master fails in between.
-   * @param znode
-   */
-  private void setZnodeAclsRecursive(String znode) throws KeeperException, InterruptedException {
-    List<String> children = recoverableZooKeeper.getChildren(znode, false);
-
-    for (String child : children) {
-      setZnodeAclsRecursive(ZNodePaths.joinZNode(znode, child));
-    }
-    List<ACL> acls = ZKUtil.createACL(this, znode, true);
-    LOG.info("Setting ACLs for znode:" + znode + " , acl:" + acls);
-    recoverableZooKeeper.setAcl(znode, acls, -1);
-  }
-
-  /**
-   * Checks whether the ACLs returned from the base znode (/hbase) is set for secure setup.
-   * @param acls acls from zookeeper
-   * @return whether ACLs are set for the base znode
-   * @throws IOException
-   */
-  private boolean isBaseZnodeAclSetup(List<ACL> acls) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Checking znode ACLs");
-    }
-    String[] superUsers = conf.getStrings(Superusers.SUPERUSER_CONF_KEY);
-    // Check whether ACL set for all superusers
-    if (superUsers != null && !checkACLForSuperUsers(superUsers, acls)) {
-      return false;
-    }
-
-    // this assumes that current authenticated user is the same as zookeeper client user
-    // configured via JAAS
-    String hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
-
-    if (acls.isEmpty()) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("ACL is empty");
-      }
-      return false;
-    }
-
-    for (ACL acl : acls) {
-      int perms = acl.getPerms();
-      Id id = acl.getId();
-      // We should only set at most 3 possible ACLs for 3 Ids. One for everyone, one for superuser
-      // and one for the hbase user
-      if (Ids.ANYONE_ID_UNSAFE.equals(id)) {
-        if (perms != Perms.READ) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
-              id, perms, Perms.READ));
-          }
-          return false;
-        }
-      } else if (superUsers != null && isSuperUserId(superUsers, id)) {
-        if (perms != Perms.ALL) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
-              id, perms, Perms.ALL));
-          }
-          return false;
-        }
-      } else if ("sasl".equals(id.getScheme())) {
-        String name = id.getId();
-        // If ZooKeeper recorded the Kerberos full name in the ACL, use only the shortname
-        Matcher match = NAME_PATTERN.matcher(name);
-        if (match.matches()) {
-          name = match.group(1);
-        }
-        if (name.equals(hbaseUser)) {
-          if (perms != Perms.ALL) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
-                id, perms, Perms.ALL));
-            }
-            return false;
-          }
-        } else {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Unexpected shortname in SASL ACL: " + id);
-          }
-          return false;
-        }
-      } else {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("unexpected ACL id '" + id + "'");
-        }
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /*
-   * Validate whether ACL set for all superusers.
-   */
-  private boolean checkACLForSuperUsers(String[] superUsers, List<ACL> acls) {
-    for (String user : superUsers) {
-      boolean hasAccess = false;
-      // TODO: Validate super group members also when ZK supports setting node ACL for groups.
-      if (!AuthUtil.isGroupPrincipal(user)) {
-        for (ACL acl : acls) {
-          if (user.equals(acl.getId().getId())) {
-            if (acl.getPerms() == Perms.ALL) {
-              hasAccess = true;
-            } else {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug(String.format(
-                  "superuser '%s' does not have correct permissions: have 0x%x, want 0x%x",
-                  acl.getId().getId(), acl.getPerms(), Perms.ALL));
-              }
-            }
-            break;
-          }
-        }
-        if (!hasAccess) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  /*
-   * Validate whether ACL ID is superuser.
-   */
-  public static boolean isSuperUserId(String[] superUsers, Id id) {
-    for (String user : superUsers) {
-      // TODO: Validate super group members also when ZK supports setting node ACL for groups.
-      if (!AuthUtil.isGroupPrincipal(user) && new Id("sasl", user).equals(id)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return this.identifier + ", quorum=" + quorum + ", baseZNode=" + znodePaths.baseZNode;
-  }
-
-  /**
-   * Adds this instance's identifier as a prefix to the passed <code>str</code>
-   * @param str String to amend.
-   * @return A new string with this instance's identifier as prefix: e.g.
-   * if passed 'hello world', the returned string could be
-   */
-  public String prefix(final String str) {
-    return this.toString() + " " + str;
-  }
-
-  /**
-   * Get the znodes corresponding to the meta replicas from ZK
-   * @return list of znodes
-   * @throws KeeperException
-   */
-  public List<String> getMetaReplicaNodes() throws KeeperException {
-    List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
-    List<String> metaReplicaNodes = new ArrayList<>(2);
-    if (childrenOfBaseNode != null) {
-      String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
-      for (String child : childrenOfBaseNode) {
-        if (child.startsWith(pattern)) metaReplicaNodes.add(child);
-      }
-    }
-    return metaReplicaNodes;
-  }
-
-  /**
-   * Register the specified listener to receive ZooKeeper events.
-   * @param listener
-   */
-  public void registerListener(ZooKeeperListener listener) {
-    listeners.add(listener);
-  }
-
-  /**
-   * Register the specified listener to receive ZooKeeper events and add it as
-   * the first in the list of current listeners.
-   * @param listener
-   */
-  public void registerListenerFirst(ZooKeeperListener listener) {
-    listeners.add(0, listener);
-  }
-
-  public void unregisterListener(ZooKeeperListener listener) {
-    listeners.remove(listener);
-  }
-
-  /**
-   * Clean all existing listeners
-   */
-  public void unregisterAllListeners() {
-    listeners.clear();
-  }
-
-  /**
-   * Get a copy of current registered listeners
-   */
-  public List<ZooKeeperListener> getListeners() {
-    return new ArrayList<>(listeners);
-  }
-
-  /**
-   * @return The number of currently registered listeners
-   */
-  public int getNumberOfListeners() {
-    return listeners.size();
-  }
-
-  /**
-   * Get the connection to ZooKeeper.
-   * @return connection reference to zookeeper
-   */
-  public RecoverableZooKeeper getRecoverableZooKeeper() {
-    return recoverableZooKeeper;
-  }
-
-  public void reconnectAfterExpiration() throws IOException, KeeperException, InterruptedException {
-    recoverableZooKeeper.reconnectAfterExpiration();
-  }
-
-  /**
-   * Get the quorum address of this instance.
-   * @return quorum string of this zookeeper connection instance
-   */
-  public String getQuorum() {
-    return quorum;
-  }
-
-  /**
-   * Get the znodePaths.
-   * <p>
-   * Mainly used for mocking as mockito can not mock a field access.
-   */
-  public ZNodePaths getZNodePaths() {
-    return znodePaths;
-  }
-
-  /**
-   * Method called from ZooKeeper for events and connection status.
-   * <p>
-   * Valid events are passed along to listeners.  Connection status changes
-   * are dealt with locally.
-   */
-  @Override
-  public void process(WatchedEvent event) {
-    LOG.debug(prefix("Received ZooKeeper Event, " +
-        "type=" + event.getType() + ", " +
-        "state=" + event.getState() + ", " +
-        "path=" + event.getPath()));
-
-    switch(event.getType()) {
-
-      // If event type is NONE, this is a connection status change
-      case None: {
-        connectionEvent(event);
-        break;
-      }
-
-      // Otherwise pass along to the listeners
-
-      case NodeCreated: {
-        for(ZooKeeperListener listener : listeners) {
-          listener.nodeCreated(event.getPath());
-        }
-        break;
-      }
-
-      case NodeDeleted: {
-        for(ZooKeeperListener listener : listeners) {
-          listener.nodeDeleted(event.getPath());
-        }
-        break;
-      }
-
-      case NodeDataChanged: {
-        for(ZooKeeperListener listener : listeners) {
-          listener.nodeDataChanged(event.getPath());
-        }
-        break;
-      }
-
-      case NodeChildrenChanged: {
-        for(ZooKeeperListener listener : listeners) {
-          listener.nodeChildrenChanged(event.getPath());
-        }
-        break;
-      }
-    }
-  }
-
-  // Connection management
-
-  /**
-   * Called when there is a connection-related event via the Watcher callback.
-   * <p>
-   * If Disconnected or Expired, this should shutdown the cluster. But, since
-   * we send a KeeperException.SessionExpiredException along with the abort
-   * call, it's possible for the Abortable to catch it and try to create a new
-   * session with ZooKeeper. This is what the client does in HCM.
-   * <p>
-   * @param event
-   */
-  private void connectionEvent(WatchedEvent event) {
-    switch(event.getState()) {
-      case SyncConnected:
-        this.identifier = this.prefix + "-0x" +
-          Long.toHexString(this.recoverableZooKeeper.getSessionId());
-        // Update our identifier.  Otherwise ignore.
-        LOG.debug(this.identifier + " connected");
-        break;
-
-      // Abort the server if Disconnected or Expired
-      case Disconnected:
-        LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
-        break;
-
-      case Expired:
-        String msg = prefix(this.identifier + " received expired from " +
-          "ZooKeeper, aborting");
-        // TODO: One thought is to add call to ZooKeeperListener so say,
-        // ZooKeeperNodeTracker can zero out its data values.
-        if (this.abortable != null) {
-          this.abortable.abort(msg, new KeeperException.SessionExpiredException());
-        }
-        break;
-
-      case ConnectedReadOnly:
-      case SaslAuthenticated:
-      case AuthFailed:
-        break;
-
-      default:
-        throw new IllegalStateException("Received event is not valid: " + event.getState());
-    }
-  }
-
-  /**
-   * Forces a synchronization of this ZooKeeper client connection.
-   * <p>
-   * Executing this method before running other methods will ensure that the
-   * subsequent operations are up-to-date and consistent as of the time that
-   * the sync is complete.
-   * <p>
-   * This is used for compareAndSwap type operations where we need to read the
-   * data of an existing node and delete or transition that node, utilizing the
-   * previously read version and data.  We want to ensure that the version read
-   * is up-to-date from when we begin the operation.
-   */
-  public void sync(String path) throws KeeperException {
-    this.recoverableZooKeeper.sync(path, null, null);
-  }
-
-  /**
-   * Handles KeeperExceptions in client calls.
-   * <p>
-   * This may be temporary but for now this gives one place to deal with these.
-   * <p>
-   * TODO: Currently this method rethrows the exception to let the caller handle
-   * <p>
-   * @param ke
-   * @throws KeeperException
-   */
-  public void keeperException(KeeperException ke)
-  throws KeeperException {
-    LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
-    throw ke;
-  }
-
-  /**
-   * Handles InterruptedExceptions in client calls.
-   * @param ie the InterruptedException instance thrown
-   * @throws KeeperException the exception to throw, transformed from the InterruptedException
-   */
-  public void interruptedException(InterruptedException ie) throws KeeperException {
-    interruptedExceptionNoThrow(ie, true);
-    // Throw a system error exception to let upper level handle it
-    throw new KeeperException.SystemErrorException();
-  }
-
-  /**
-   * Log the InterruptedException and interrupt current thread
-   * @param ie The IterruptedException to log
-   * @param throwLater Whether we will throw the exception latter
-   */
-  public void interruptedExceptionNoThrow(InterruptedException ie, boolean throwLater) {
-    LOG.debug(prefix("Received InterruptedException, will interrupt current thread"
-        + (throwLater ? " and rethrow a SystemErrorException" : "")),
-      ie);
-    // At least preserve interrupt.
-    Thread.currentThread().interrupt();
-  }
-
-  /**
-   * Close the connection to ZooKeeper.
-   *
-   */
-  @Override
-  public void close() {
-    try {
-      recoverableZooKeeper.close();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-    }
-  }
-
-  public Configuration getConfiguration() {
-    return conf;
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    if (this.abortable != null) this.abortable.abort(why, e);
-    else this.aborted = true;
-  }
-
-  @Override
-  public boolean isAborted() {
-    return this.abortable == null? this.aborted: this.abortable.isAborted();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
deleted file mode 100644
index e67c9fd..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java
+++ /dev/null
@@ -1,49 +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.concurrent.atomic.AtomicReference;
-
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(SmallTests.class)
-public class TestInstancePending {
-  @Test(timeout = 1000)
-  public void test() throws Exception {
-    final InstancePending<String> pending = new InstancePending<>();
-    final AtomicReference<String> getResultRef = new AtomicReference<>();
-
-    new Thread() {
-      @Override
-      public void run() {
-        getResultRef.set(pending.get());
-      }
-    }.start();
-
-    Thread.sleep(100);
-    Assert.assertNull(getResultRef.get());
-
-    pending.prepare("abc");
-    Thread.sleep(100);
-    Assert.assertEquals("abc", getResultRef.get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/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 a987bec..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);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
deleted file mode 100644
index 6b1e1f0..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
+++ /dev/null
@@ -1,113 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.security.Superusers;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooDefs.Perms;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-/**
- *
- */
-@Category({SmallTests.class})
-public class TestZKUtil {
-
-  @Test
-  public void testUnsecure() throws ZooKeeperConnectionException, IOException {
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(Superusers.SUPERUSER_CONF_KEY, "user1");
-    String node = "/hbase/testUnsecure";
-    ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false);
-    List<ACL> aclList = ZKUtil.createACL(watcher, node, false);
-    Assert.assertEquals(aclList.size(), 1);
-    Assert.assertTrue(aclList.contains(Ids.OPEN_ACL_UNSAFE.iterator().next()));
-  }
-
-  @Test
-  public void testSecuritySingleSuperuser() throws ZooKeeperConnectionException, IOException {
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(Superusers.SUPERUSER_CONF_KEY, "user1");
-    String node = "/hbase/testSecuritySingleSuperuser";
-    ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false);
-    List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
-    Assert.assertEquals(aclList.size(), 2); // 1+1, since ACL will be set for the creator by default
-    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
-    Assert.assertTrue(aclList.contains(Ids.CREATOR_ALL_ACL.iterator().next()));
-  }
-
-  @Test
-  public void testCreateACL() throws ZooKeeperConnectionException, IOException {
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(Superusers.SUPERUSER_CONF_KEY, "user1,@group1,user2,@group2,user3");
-    String node = "/hbase/testCreateACL";
-    ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false);
-    List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
-    Assert.assertEquals(aclList.size(), 4); // 3+1, since ACL will be set for the creator by default
-    Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
-    Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group2"))));
-    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
-    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user2"))));
-    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user3"))));
-  }
-
-  @Test
-  public void testCreateACLWithSameUser() throws ZooKeeperConnectionException, IOException {
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(Superusers.SUPERUSER_CONF_KEY, "user4,@group1,user5,user6");
-    UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("user4"));
-    String node = "/hbase/testCreateACL";
-    ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false);
-    List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
-    Assert.assertEquals(aclList.size(), 3); // 3, since service user the same as one of superuser
-    Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
-    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("auth", ""))));
-    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user5"))));
-    Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user6"))));
-  }
-
-  @Test(expected = KeeperException.SystemErrorException.class)
-  public void testInterruptedDuringAction()
-      throws ZooKeeperConnectionException, IOException, KeeperException, InterruptedException {
-    final RecoverableZooKeeper recoverableZk = Mockito.mock(RecoverableZooKeeper.class);
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest", null) {
-      @Override
-      public RecoverableZooKeeper getRecoverableZooKeeper() {
-        return recoverableZk;
-      }
-    };
-    Mockito.doThrow(new InterruptedException()).when(recoverableZk)
-        .getChildren(zkw.znodePaths.baseZNode, null);
-    ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.baseZNode);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java
deleted file mode 100644
index 3c99175..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java
+++ /dev/null
@@ -1,57 +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.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ SmallTests.class })
-public class TestZooKeeperWatcher {
-
-  @Test
-  public void testIsClientReadable() throws IOException {
-    ZooKeeperWatcher watcher =
-      new ZooKeeperWatcher(HBaseConfiguration.create(), "testIsClientReadable", null, false);
-
-    assertTrue(watcher.isClientReadable(watcher.znodePaths.baseZNode));
-    assertTrue(watcher.isClientReadable(watcher.znodePaths.getZNodeForReplica(0)));
-    assertTrue(watcher.isClientReadable(watcher.znodePaths.masterAddressZNode));
-    assertTrue(watcher.isClientReadable(watcher.znodePaths.clusterIdZNode));
-    assertTrue(watcher.isClientReadable(watcher.znodePaths.tableZNode));
-    assertTrue(
-      watcher.isClientReadable(ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, "foo")));
-    assertTrue(watcher.isClientReadable(watcher.znodePaths.rsZNode));
-
-    assertFalse(watcher.isClientReadable(watcher.znodePaths.tableLockZNode));
-    assertFalse(watcher.isClientReadable(watcher.znodePaths.balancerZNode));
-    assertFalse(watcher.isClientReadable(watcher.znodePaths.regionNormalizerZNode));
-    assertFalse(watcher.isClientReadable(watcher.znodePaths.clusterStateZNode));
-    assertFalse(watcher.isClientReadable(watcher.znodePaths.drainingZNode));
-    assertFalse(watcher.isClientReadable(watcher.znodePaths.splitLogZNode));
-    assertFalse(watcher.isClientReadable(watcher.znodePaths.backupMasterAddressesZNode));
-
-    watcher.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
index eba8518..c44be52 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 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.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -37,7 +37,7 @@ import org.junit.experimental.categories.Category;
 /**
  * An integration test that starts the cluster with three replicas for the meta
  * It then creates a table, flushes the meta, kills the server holding the primary.
- * After that a client issues put/get requests on the created table - the other 
+ * After that a client issues put/get requests on the created table - the other
  * replicas of the meta would be used to get the location of the region of the created
  * table.
  */
@@ -60,7 +60,7 @@ public class IntegrationTestMetaReplicas {
         StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
     // Make sure there are three servers.
     util.initializeCluster(3);
-    ZooKeeperWatcher zkw = util.getZooKeeperWatcher();
+    ZKWatcher zkw = util.getZooKeeperWatcher();
     Configuration conf = util.getConfiguration();
     String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
         HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
index 92bad7f..09cbda3 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 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.util.ToolRunner;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
@@ -139,7 +139,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
 
   private void testZNodeACLs() throws IOException, KeeperException, InterruptedException {
 
-    ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, "IntegrationTestZnodeACLs", null);
+    ZKWatcher watcher = new ZKWatcher(conf, "IntegrationTestZnodeACLs", null);
     RecoverableZooKeeper zk = ZKUtil.connect(this.conf, watcher);
 
     String baseZNode = watcher.znodePaths.baseZNode;
@@ -155,7 +155,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
     LOG.info("Checking ZK permissions: SUCCESS");
   }
 
-  private void checkZnodePermsRecursive(ZooKeeperWatcher watcher,
+  private void checkZnodePermsRecursive(ZKWatcher watcher,
       RecoverableZooKeeper zk, String znode) throws KeeperException, InterruptedException {
 
     boolean expectedWorldReadable = watcher.isClientReadable(znode);
@@ -201,7 +201,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
         assertTrue(expectedWorldReadable);
         // assert that anyone can only read
         assertEquals(perms, Perms.READ);
-      } else if (superUsers != null && ZooKeeperWatcher.isSuperUserId(superUsers, id)) {
+      } else if (superUsers != null && ZKWatcher.isSuperUserId(superUsers, id)) {
         // assert that super user has all the permissions
         assertEquals(perms, Perms.ALL);
       } else if (new Id("sasl", masterPrincipal).equals(id)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index d25d497..bf0081b 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
@@ -64,7 +65,6 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.MapReduceCell;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
@@ -406,10 +406,10 @@ public class Import extends Configured implements Tool {
         LOG.info("setting WAL durability to default.");
       }
       // TODO: This is kind of ugly doing setup of ZKW just to read the clusterid.
-      ZooKeeperWatcher zkw = null;
+      ZKWatcher zkw = null;
       Exception ex = null;
       try {
-        zkw = new ZooKeeperWatcher(conf, context.getTaskAttemptID().toString(), null);
+        zkw = new ZKWatcher(conf, context.getTaskAttemptID().toString(), null);
         clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
       } catch (ZooKeeperConnectionException e) {
         ex = e;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index acf6ff8..b8de9ec 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -330,10 +330,10 @@ public class VerifyReplication extends Configured implements Tool {
 
   private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
       final Configuration conf, String peerId) throws IOException {
-    ZooKeeperWatcher localZKW = null;
+    ZKWatcher localZKW = null;
     ReplicationPeerZKImpl peer = null;
     try {
-      localZKW = new ZooKeeperWatcher(conf, "VerifyReplication",
+      localZKW = new ZKWatcher(conf, "VerifyReplication",
           new Abortable() {
             @Override public void abort(String why, Throwable e) {}
             @Override public boolean isAborted() {return false;}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index 942fd8c..039f8ac 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -130,6 +130,10 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-client</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-zookeeper</artifactId>
+    </dependency>
     <!-- General dependencies -->
     <dependency>
       <groupId>org.apache.commons</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 9d12211..3ff6914 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -23,7 +23,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
 /**
  * A factory class for instantiating replication objects that deal with replication state.
@@ -48,17 +48,17 @@ public class ReplicationFactory {
     return (ReplicationQueuesClient) ConstructorUtils.invokeConstructor(classToBuild, args);
   }
 
-  public static ReplicationPeers getReplicationPeers(final ZooKeeperWatcher zk, Configuration conf,
-      Abortable abortable) {
+  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
+                                                     Abortable abortable) {
     return getReplicationPeers(zk, conf, null, abortable);
   }
 
-  public static ReplicationPeers getReplicationPeers(final ZooKeeperWatcher zk, Configuration conf,
-      final ReplicationQueuesClient queuesClient, Abortable abortable) {
+  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
+                                                     final ReplicationQueuesClient queuesClient, Abortable abortable) {
     return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable);
   }
 
-  public static ReplicationTracker getReplicationTracker(ZooKeeperWatcher zookeeper,
+  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,
       final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
       Stoppable stopper) {
     return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, abortable, stopper);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index 8f09479..214a313 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -34,9 +34,9 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
@@ -62,7 +62,7 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerZKImpl(ZooKeeperWatcher zkWatcher, Configuration conf,
+  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf,
                                String id, ReplicationPeerConfig peerConfig,
                                Abortable abortable)
       throws ReplicationException {
@@ -258,9 +258,9 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
   /**
    * Tracker for state of this peer
    */
-  public class PeerStateTracker extends ZooKeeperNodeTracker {
+  public class PeerStateTracker extends ZKNodeTracker {
 
-    public PeerStateTracker(String peerStateZNode, ZooKeeperWatcher watcher,
+    public PeerStateTracker(String peerStateZNode, ZKWatcher watcher,
         Abortable abortable) {
       super(watcher, peerStateZNode, abortable);
     }
@@ -281,11 +281,11 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
   /**
    * Tracker for PeerConfigNode of this peer
    */
-  public class PeerConfigTracker extends ZooKeeperNodeTracker {
+  public class PeerConfigTracker extends ZKNodeTracker {
 
     ReplicationPeerConfigListener listener;
 
-    public PeerConfigTracker(String peerConfigNode, ZooKeeperWatcher watcher,
+    public PeerConfigTracker(String peerConfigNode, ZKWatcher watcher,
         Abortable abortable) {
       super(watcher, peerConfigNode, abortable);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index cc84c1d..9c4e3fe 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
+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;
 
@@ -84,8 +84,8 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
 
   private static final Log LOG = LogFactory.getLog(ReplicationPeersZKImpl.class);
 
-  public ReplicationPeersZKImpl(final ZooKeeperWatcher zk, final Configuration conf,
-      final ReplicationQueuesClient queuesClient, Abortable abortable) {
+  public ReplicationPeersZKImpl(final ZKWatcher zk, final Configuration conf,
+                                final ReplicationQueuesClient queuesClient, Abortable abortable) {
     super(zk, conf, abortable);
     this.abortable = abortable;
     this.peerClusters = new ConcurrentHashMap<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
index c8328bd..c2a5df3 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.replication;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 /**
  * Wrapper around common arguments used to construct ReplicationQueues. Used to construct various
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 @InterfaceAudience.Private
 public class ReplicationQueuesArguments {
 
-  private ZooKeeperWatcher zk;
+  private ZKWatcher zk;
   private Configuration conf;
   private Abortable abort;
 
@@ -39,16 +39,16 @@ public class ReplicationQueuesArguments {
     this.abort = abort;
   }
 
-  public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZooKeeperWatcher zk) {
+  public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZKWatcher zk) {
     this(conf, abort);
     setZk(zk);
   }
 
-  public ZooKeeperWatcher getZk() {
+  public ZKWatcher getZk() {
     return zk;
   }
 
-  public void setZk(ZooKeeperWatcher zk) {
+  public void setZk(ZKWatcher zk) {
     this.zk = zk;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
index 67258c7..9b79294 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.replication;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 /**
  * Wrapper around common arguments used to construct ReplicationQueuesClient. Used to construct
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 @InterfaceAudience.Private
 public class ReplicationQueuesClientArguments extends ReplicationQueuesArguments {
   public ReplicationQueuesClientArguments(Configuration conf, Abortable abort,
-     ZooKeeperWatcher zk) {
+     ZKWatcher zk) {
     super(conf, abort, zk);
   }
   public ReplicationQueuesClientArguments(Configuration conf, Abortable abort) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index 49e55ef..b998f15 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -21,16 +21,16 @@ package org.apache.hadoop.hbase.replication;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 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.Abortable;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 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;
 import org.apache.zookeeper.data.Stat;
 
@@ -44,8 +44,8 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
     this(args.getZk(), args.getConf(), args.getAbortable());
   }
 
-  public ReplicationQueuesClientZKImpl(final ZooKeeperWatcher zk, Configuration conf,
-      Abortable abortable) {
+  public ReplicationQueuesClientZKImpl(final ZKWatcher zk, Configuration conf,
+                                       Abortable abortable) {
     super(zk, conf, abortable);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 7b1d5c2..95fd294 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -25,20 +25,18 @@ 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.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HConstants;
 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.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
+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;
 
 /**
@@ -75,8 +73,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
     this(args.getZk(), args.getConf(), args.getAbortable());
   }
 
-  public ReplicationQueuesZKImpl(final ZooKeeperWatcher zk, Configuration conf,
-      Abortable abortable) {
+  public ReplicationQueuesZKImpl(final ZKWatcher zk, Configuration conf,
+                                 Abortable abortable) {
     super(zk, conf, abortable);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index e96401e..ad970c6 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -22,21 +22,19 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 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;
 
-
 /**
  * This is a base class for maintaining replication state in zookeeper.
  */
@@ -61,7 +59,7 @@ public abstract class ReplicationStateZKBase {
   /** The name of the znode that contains tableCFs */
   protected final String tableCFsNodeName;
 
-  protected final ZooKeeperWatcher zookeeper;
+  protected final ZKWatcher zookeeper;
   protected final Configuration conf;
   protected final Abortable abortable;
 
@@ -74,8 +72,8 @@ public abstract class ReplicationStateZKBase {
       "zookeeper.znode.replication.hfile.refs";
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
 
-  public ReplicationStateZKBase(ZooKeeperWatcher zookeeper, Configuration conf,
-      Abortable abortable) {
+  public ReplicationStateZKBase(ZKWatcher zookeeper, Configuration conf,
+                                Abortable abortable) {
     this.zookeeper = zookeeper;
     this.conf = conf;
     this.abortable = abortable;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
index ade1c4d..aa72fc5 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
@@ -24,13 +24,13 @@ import java.util.concurrent.CopyOnWriteArrayList;
 
 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.Abortable;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -50,7 +50,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   private final ArrayList<String> otherRegionServers = new ArrayList<>();
   private final ReplicationPeers replicationPeers;
 
-  public ReplicationTrackerZKImpl(ZooKeeperWatcher zookeeper,
+  public ReplicationTrackerZKImpl(ZKWatcher zookeeper,
       final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
       Stoppable stopper) {
     super(zookeeper, conf, abortable);
@@ -88,12 +88,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    * Watcher used to be notified of the other region server's death in the local cluster. It
    * initiates the process to transfer the queues if it is able to grab the lock.
    */
-  public class OtherRegionServerWatcher extends ZooKeeperListener {
+  public class OtherRegionServerWatcher extends ZKListener {
 
     /**
      * Construct a ZooKeeper event listener.
      */
-    public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
+    public OtherRegionServerWatcher(ZKWatcher watcher) {
       super(watcher);
     }
 
@@ -145,12 +145,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   /**
    * Watcher used to follow the creation and deletion of peer clusters.
    */
-  public class PeersWatcher extends ZooKeeperListener {
+  public class PeersWatcher extends ZKListener {
 
     /**
      * Construct a ZooKeeper event listener.
      */
-    public PeersWatcher(ZooKeeperWatcher watcher) {
+    public PeersWatcher(ZKWatcher watcher) {
       super(watcher);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
index cf1ff20..1f239f8 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
@@ -37,11 +37,10 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 
 import java.io.IOException;
@@ -77,7 +76,7 @@ public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
     this(args.getConf(), args.getAbortable(), args.getZk());
   }
 
-  public TableBasedReplicationQueuesImpl(Configuration conf, Abortable abort, ZooKeeperWatcher zkw)
+  public TableBasedReplicationQueuesImpl(Configuration conf, Abortable abort, ZKWatcher zkw)
     throws IOException {
     super(conf, abort);
     replicationState = new ReplicationStateZKBase(zkw, conf, abort) {};

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 3d152bb..5fca659 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -77,8 +77,8 @@ import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
 import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.util.Bytes;
 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;
 
@@ -141,7 +141,7 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
   private final MasterServices masterServices;
   private Table rsGroupTable;
   private final ClusterConnection conn;
-  private final ZooKeeperWatcher watcher;
+  private final ZKWatcher watcher;
   private final RSGroupStartupWorker rsGroupStartupWorker = new RSGroupStartupWorker();
   // contains list of groups that were last flushed to persistent store
   private Set<String> prevRSGroups = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
index 9077f15..ba3534d 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
@@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Assert;
 
@@ -45,14 +45,14 @@ import java.util.Set;
 @InterfaceAudience.Private
 public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
   private Table table;
-  private ZooKeeperWatcher zkw;
+  private ZKWatcher zkw;
   private RSGroupAdmin wrapped;
 
   public VerifyingRSGroupAdminClient(RSGroupAdmin RSGroupAdmin, Configuration conf)
       throws IOException {
     wrapped = RSGroupAdmin;
     table = ConnectionFactory.createConnection(conf).getTable(RSGroupInfoManager.RSGROUP_TABLE_NAME);
-    zkw = new ZooKeeperWatcher(conf, this.getClass().getSimpleName(), null);
+    zkw = new ZKWatcher(conf, this.getClass().getSimpleName(), null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 1e5a1f3..daa34f8 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -390,6 +390,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-zookeeper</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-replication</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index a6b39f3..0d9eba8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 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;
 
 import java.io.IOException;
@@ -43,7 +43,7 @@ public interface Server extends Abortable, Stoppable {
   /**
    * Gets the ZooKeeper instance for this server.
    */
-  ZooKeeperWatcher getZooKeeper();
+  ZKWatcher getZooKeeper();
 
   /**
    * Returns a reference to the servers' connection.

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
index c4fad44..160d058 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 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.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 
 import java.io.IOException;
@@ -48,12 +48,12 @@ import java.util.concurrent.ConcurrentSkipListMap;
  *
  */
 @InterfaceAudience.Private
-public class ZKNamespaceManager extends ZooKeeperListener {
+public class ZKNamespaceManager extends ZKListener {
   private static final Log LOG = LogFactory.getLog(ZKNamespaceManager.class);
   private final String nsZNode;
   private volatile NavigableMap<String,NamespaceDescriptor> cache;
 
-  public ZKNamespaceManager(ZooKeeperWatcher zkw) throws IOException {
+  public ZKNamespaceManager(ZKWatcher zkw) throws IOException {
     super(zkw);
     nsZNode = zkw.znodePaths.namespaceZNode;
     cache = new ConcurrentSkipListMap<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
index d7fdeb7..cda5aff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
@@ -32,8 +32,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 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.zookeeper.KeeperException;
 
 /**
@@ -132,7 +132,7 @@ public class ZNodeClearer {
    * @param rsZnodePath from HBASE_ZNODE_FILE
    * @return String representation of ServerName or null if fails
    */
-  
+
   public static String parseMasterServerName(String rsZnodePath) {
     String masterServerName = null;
     try {
@@ -141,12 +141,12 @@ public class ZNodeClearer {
     } catch (IndexOutOfBoundsException e) {
       LOG.warn("String " + rsZnodePath + " has wrong format", e);
     }
-    return masterServerName; 
+    return masterServerName;
   }
-  
+
   /**
-   * 
-   * @return true if cluster is configured with master-rs collocation 
+   *
+   * @return true if cluster is configured with master-rs collocation
    */
   private static boolean tablesOnMaster(Configuration conf) {
     boolean tablesOnMaster = true;
@@ -167,9 +167,9 @@ public class ZNodeClearer {
     Configuration tempConf = new Configuration(conf);
     tempConf.setInt("zookeeper.recovery.retry", 0);
 
-    ZooKeeperWatcher zkw;
+    ZKWatcher zkw;
     try {
-      zkw = new ZooKeeperWatcher(tempConf, "clean znode for master",
+      zkw = new ZKWatcher(tempConf, "clean znode for master",
           new Abortable() {
             @Override public void abort(String why, Throwable e) {}
             @Override public boolean isAborted() { return false; }


[03/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
new file mode 100644
index 0000000..2f2b036
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java
@@ -0,0 +1,99 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.UUID;
+
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.ClusterId;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Publishes and synchronizes a unique identifier specific to a given HBase
+ * cluster.  The stored identifier is read from the file system by the active
+ * master on startup, and is subsequently available to all watchers (including
+ * clients).
+ */
+@InterfaceAudience.Private
+public class ZKClusterId {
+  private ZKWatcher watcher;
+  private Abortable abortable;
+  private String id;
+
+  public ZKClusterId(ZKWatcher watcher, Abortable abortable) {
+    this.watcher = watcher;
+    this.abortable = abortable;
+  }
+
+  public boolean hasId() {
+    return getId() != null;
+  }
+
+  public String getId() {
+    try {
+      if (id == null) {
+        id = readClusterIdZNode(watcher);
+      }
+    } catch (KeeperException ke) {
+      abortable.abort("Unexpected exception from ZooKeeper reading cluster ID",
+          ke);
+    }
+    return id;
+  }
+
+  public static String readClusterIdZNode(ZKWatcher watcher)
+  throws KeeperException {
+    if (ZKUtil.checkExists(watcher, watcher.znodePaths.clusterIdZNode) != -1) {
+      byte [] data;
+      try {
+        data = ZKUtil.getData(watcher, watcher.znodePaths.clusterIdZNode);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return null;
+      }
+      if (data != null) {
+        try {
+          return ClusterId.parseFrom(data).toString();
+        } catch (DeserializationException e) {
+          throw ZKUtil.convert(e);
+        }
+      }
+    }
+    return null;
+  }
+
+  public static void setClusterId(ZKWatcher watcher, ClusterId id)
+      throws KeeperException {
+    ZKUtil.createSetData(watcher, watcher.znodePaths.clusterIdZNode, id.toByteArray());
+  }
+
+  /**
+   * Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions
+   * @param zkw watcher connected to an ensemble
+   * @return the UUID read from zookeeper
+   * @throws KeeperException
+   */
+  public static UUID getUUIDForCluster(ZKWatcher zkw) throws KeeperException {
+    String uuid = readClusterIdZNode(zkw);
+    return uuid == null ? null : UUID.fromString(uuid);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
new file mode 100644
index 0000000..edd2ccd
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Handles coordination of a single "leader" instance among many possible
+ * candidates.  The first {@link ZKLeaderManager} to successfully create
+ * the given znode becomes the leader, allowing the instance to continue
+ * with whatever processing must be protected.  Other {@link ZKLeaderManager}
+ * instances will wait to be notified of changes to the leader znode.
+ * If the current master instance fails, the ephemeral leader znode will
+ * be removed, and all waiting instances will be notified, with the race
+ * to claim the leader znode beginning all over again.
+ * @deprecated Not used
+ */
+@Deprecated
+@InterfaceAudience.Private
+public class ZKLeaderManager extends ZKListener {
+  private static final Log LOG = LogFactory.getLog(ZKLeaderManager.class);
+
+  private final AtomicBoolean leaderExists = new AtomicBoolean();
+  private String leaderZNode;
+  private byte[] nodeId;
+  private Stoppable candidate;
+
+  public ZKLeaderManager(ZKWatcher watcher, String leaderZNode,
+                         byte[] identifier, Stoppable candidate) {
+    super(watcher);
+    this.leaderZNode = leaderZNode;
+    this.nodeId = identifier;
+    this.candidate = candidate;
+  }
+
+  public void start() {
+    try {
+      watcher.registerListener(this);
+      String parent = ZKUtil.getParent(leaderZNode);
+      if (ZKUtil.checkExists(watcher, parent) < 0) {
+        ZKUtil.createWithParents(watcher, parent);
+      }
+    } catch (KeeperException ke) {
+      watcher.abort("Unhandled zk exception when starting", ke);
+      candidate.stop("Unhandled zk exception starting up: "+ke.getMessage());
+    }
+  }
+
+  @Override
+  public void nodeCreated(String path) {
+    if (leaderZNode.equals(path) && !candidate.isStopped()) {
+      handleLeaderChange();
+    }
+  }
+
+  @Override
+  public void nodeDeleted(String path) {
+    if (leaderZNode.equals(path) && !candidate.isStopped()) {
+      handleLeaderChange();
+    }
+  }
+
+  private void handleLeaderChange() {
+    try {
+      synchronized(leaderExists) {
+        if (ZKUtil.watchAndCheckExists(watcher, leaderZNode)) {
+          LOG.info("Found new leader for znode: "+leaderZNode);
+          leaderExists.set(true);
+        } else {
+          LOG.info("Leader change, but no new leader found");
+          leaderExists.set(false);
+          leaderExists.notifyAll();
+        }
+      }
+    } catch (KeeperException ke) {
+      watcher.abort("ZooKeeper error checking for leader znode", ke);
+      candidate.stop("ZooKeeper error checking for leader: "+ke.getMessage());
+    }
+  }
+
+  /**
+   * Blocks until this instance has claimed the leader ZNode in ZooKeeper
+   */
+  public void waitToBecomeLeader() {
+    while (!candidate.isStopped()) {
+      try {
+        if (ZKUtil.createEphemeralNodeAndWatch(watcher, leaderZNode, nodeId)) {
+          // claimed the leader znode
+          leaderExists.set(true);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Claimed the leader znode as '"+
+                Bytes.toStringBinary(nodeId)+"'");
+          }
+          return;
+        }
+
+        // if claiming the node failed, there should be another existing node
+        byte[] currentId = ZKUtil.getDataAndWatch(watcher, leaderZNode);
+        if (currentId != null && Bytes.equals(currentId, nodeId)) {
+          // claimed with our ID, but we didn't grab it, possibly restarted?
+          LOG.info("Found existing leader with our ID ("+
+              Bytes.toStringBinary(nodeId)+"), removing");
+          ZKUtil.deleteNode(watcher, leaderZNode);
+          leaderExists.set(false);
+        } else {
+          LOG.info("Found existing leader with ID: "+Bytes.toStringBinary(nodeId));
+          leaderExists.set(true);
+        }
+      } catch (KeeperException ke) {
+        watcher.abort("Unexpected error from ZK, stopping candidate", ke);
+        candidate.stop("Unexpected error from ZK: "+ke.getMessage());
+        return;
+      }
+
+      // wait for next chance
+      synchronized(leaderExists) {
+        while (leaderExists.get() && !candidate.isStopped()) {
+          try {
+            leaderExists.wait();
+          } catch (InterruptedException ie) {
+            LOG.debug("Interrupted waiting on leader", ie);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Removes the leader znode, if it is currently claimed by this instance.
+   */
+  public void stepDownAsLeader() {
+    try {
+      synchronized(leaderExists) {
+        if (!leaderExists.get()) {
+          return;
+        }
+        byte[] leaderId = ZKUtil.getData(watcher, leaderZNode);
+        if (leaderId != null && Bytes.equals(nodeId, leaderId)) {
+          LOG.info("Stepping down as leader");
+          ZKUtil.deleteNodeFailSilent(watcher, leaderZNode);
+          leaderExists.set(false);
+        } else {
+          LOG.info("Not current leader, no need to step down");
+        }
+      }
+    } catch (KeeperException ke) {
+      watcher.abort("Unhandled zookeeper exception removing leader node", ke);
+      candidate.stop("Unhandled zookeeper exception removing leader node: "
+          + ke.getMessage());
+    } catch (InterruptedException e) {
+      watcher.abort("Unhandled zookeeper exception removing leader node", e);
+      candidate.stop("Unhandled zookeeper exception removing leader node: "
+          + e.getMessage());
+    }
+  }
+
+  public boolean hasLeader() {
+    return leaderExists.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java
new file mode 100644
index 0000000..595e713
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKListener.java
@@ -0,0 +1,85 @@
+/*
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * Base class for internal listeners of ZooKeeper events.
+ *
+ * The {@link ZKWatcher} for a process will execute the appropriate
+ * methods of implementations of this class.  In order to receive events from
+ * the watcher, every listener must register itself via {@link ZKWatcher#registerListener}.
+ *
+ * Subclasses need only override those methods in which they are interested.
+ *
+ * Note that the watcher will be blocked when invoking methods in listeners so
+ * they must not be long-running.
+ */
+@InterfaceAudience.Private
+public abstract class ZKListener {
+
+  // Reference to the zk watcher which also contains configuration and constants
+  protected ZKWatcher watcher;
+
+  /**
+   * Construct a ZooKeeper event listener.
+   */
+  public ZKListener(ZKWatcher watcher) {
+    this.watcher = watcher;
+  }
+
+  /**
+   * Called when a new node has been created.
+   * @param path full path of the new node
+   */
+  public void nodeCreated(String path) {
+    // no-op
+  }
+
+  /**
+   * Called when a node has been deleted
+   * @param path full path of the deleted node
+   */
+  public void nodeDeleted(String path) {
+    // no-op
+  }
+
+  /**
+   * Called when an existing node has changed data.
+   * @param path full path of the updated node
+   */
+  public void nodeDataChanged(String path) {
+    // no-op
+  }
+
+  /**
+   * Called when an existing node has a child node added or removed.
+   * @param path full path of the node whose children have changed
+   */
+  public void nodeChildrenChanged(String path) {
+    // no-op
+  }
+
+  /**
+   * @return The watcher associated with this listener
+   */
+  public ZKWatcher getWatcher() {
+    return this.watcher;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java
new file mode 100644
index 0000000..9cb0e7d
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java
@@ -0,0 +1,126 @@
+/*
+ *
+ * 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 ZKMainServer {
+  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 ZKMainServer().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/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java
new file mode 100644
index 0000000..20d4a55
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java
@@ -0,0 +1,108 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * 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 ZKMetrics implements ZKMetricsListener {
+  private final MetricsZooKeeperSource source;
+
+  public ZKMetrics() {
+    this(CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class));
+  }
+
+  @VisibleForTesting
+  public ZKMetrics(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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java
new file mode 100644
index 0000000..f17925e
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.java
@@ -0,0 +1,91 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface ZKMetricsListener {
+
+  /**
+   * 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/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java
new file mode 100644
index 0000000..8ce41e3
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKNodeTracker.java
@@ -0,0 +1,251 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Tracks the availability and value of a single ZooKeeper node.
+ *
+ * <p>Utilizes the {@link ZKListener} interface to get the necessary
+ * ZooKeeper events related to the node.
+ *
+ * <p>This is the base class used by trackers in both the Master and
+ * RegionServers.
+ */
+@InterfaceAudience.Private
+public abstract class ZKNodeTracker extends ZKListener {
+  // LOG is being used in subclasses, hence keeping it protected
+  protected static final Log LOG = LogFactory.getLog(ZKNodeTracker.class);
+  /** Path of node being tracked */
+  protected final String node;
+
+  /** Data of the node being tracked */
+  private byte [] data;
+
+  /** Used to abort if a fatal error occurs */
+  protected final Abortable abortable;
+
+  private boolean stopped = false;
+
+  /**
+   * Constructs a new ZK node tracker.
+   *
+   * <p>After construction, use {@link #start} to kick off tracking.
+   *
+   * @param watcher
+   * @param node
+   * @param abortable
+   */
+  public ZKNodeTracker(ZKWatcher watcher, String node,
+                       Abortable abortable) {
+    super(watcher);
+    this.node = node;
+    this.abortable = abortable;
+    this.data = null;
+  }
+
+  /**
+   * Starts the tracking of the node in ZooKeeper.
+   *
+   * <p>Use {@link #blockUntilAvailable()} to block until the node is available
+   * or {@link #getData(boolean)} to get the data of the node if it is available.
+   */
+  public synchronized void start() {
+    this.watcher.registerListener(this);
+    try {
+      if(ZKUtil.watchAndCheckExists(watcher, node)) {
+        byte [] data = ZKUtil.getDataAndWatch(watcher, node);
+        if(data != null) {
+          this.data = data;
+        } else {
+          // It existed but now does not, try again to ensure a watch is set
+          LOG.debug("Try starting again because there is no data from " + node);
+          start();
+        }
+      }
+    } catch (KeeperException e) {
+      abortable.abort("Unexpected exception during initialization, aborting", e);
+    }
+  }
+
+  public synchronized void stop() {
+    this.stopped = true;
+    notifyAll();
+  }
+
+  /**
+   * Gets the data of the node, blocking until the node is available.
+   *
+   * @return data of the node
+   * @throws InterruptedException if the waiting thread is interrupted
+   */
+  public synchronized byte [] blockUntilAvailable()
+  throws InterruptedException {
+    return blockUntilAvailable(0, false);
+  }
+
+  /**
+   * Gets the data of the node, blocking until the node is available or the
+   * specified timeout has elapsed.
+   *
+   * @param timeout maximum time to wait for the node data to be available,
+   * n milliseconds.  Pass 0 for no timeout.
+   * @return data of the node
+   * @throws InterruptedException if the waiting thread is interrupted
+   */
+  public synchronized byte [] blockUntilAvailable(long timeout, boolean refresh)
+  throws InterruptedException {
+    if (timeout < 0) throw new IllegalArgumentException();
+    boolean notimeout = timeout == 0;
+    long startTime = System.currentTimeMillis();
+    long remaining = timeout;
+    if (refresh) {
+      try {
+        // This does not create a watch if the node does not exists
+        this.data = ZKUtil.getDataAndWatch(watcher, node);
+      } catch(KeeperException e) {
+        // We use to abort here, but in some cases the abort is ignored (
+        //  (empty Abortable), so it's better to log...
+        LOG.warn("Unexpected exception handling blockUntilAvailable", e);
+        abortable.abort("Unexpected exception handling blockUntilAvailable", e);
+      }
+    }
+    boolean nodeExistsChecked = (!refresh ||data!=null);
+    while (!this.stopped && (notimeout || remaining > 0) && this.data == null) {
+      if (!nodeExistsChecked) {
+        try {
+          nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1);
+        } catch (KeeperException e) {
+          LOG.warn(
+            "Got exception while trying to check existence in  ZooKeeper" +
+            " of the node: "+node+", retrying if timeout not reached",e );
+        }
+
+        // It did not exists, and now it does.
+        if (nodeExistsChecked){
+          LOG.debug("Node " + node + " now exists, resetting a watcher");
+          try {
+            // This does not create a watch if the node does not exists
+            this.data = ZKUtil.getDataAndWatch(watcher, node);
+          } catch (KeeperException e) {
+            LOG.warn("Unexpected exception handling blockUntilAvailable", e);
+            abortable.abort("Unexpected exception handling blockUntilAvailable", e);
+          }
+        }
+      }
+      // We expect a notification; but we wait with a
+      //  a timeout to lower the impact of a race condition if any
+      wait(100);
+      remaining = timeout - (System.currentTimeMillis() - startTime);
+    }
+    return this.data;
+  }
+
+  /**
+   * Gets the data of the node.
+   *
+   * <p>If the node is currently available, the most up-to-date known version of
+   * the data is returned.  If the node is not currently available, null is
+   * returned.
+   * @param refresh whether to refresh the data by calling ZK directly.
+   * @return data of the node, null if unavailable
+   */
+  public synchronized byte [] getData(boolean refresh) {
+    if (refresh) {
+      try {
+        this.data = ZKUtil.getDataAndWatch(watcher, node);
+      } catch(KeeperException e) {
+        abortable.abort("Unexpected exception handling getData", e);
+      }
+    }
+    return this.data;
+  }
+
+  public String getNode() {
+    return this.node;
+  }
+
+  @Override
+  public synchronized void nodeCreated(String path) {
+    if (!path.equals(node)) return;
+    try {
+      byte [] data = ZKUtil.getDataAndWatch(watcher, node);
+      if (data != null) {
+        this.data = data;
+        notifyAll();
+      } else {
+        nodeDeleted(path);
+      }
+    } catch(KeeperException e) {
+      abortable.abort("Unexpected exception handling nodeCreated event", e);
+    }
+  }
+
+  @Override
+  public synchronized void nodeDeleted(String path) {
+    if(path.equals(node)) {
+      try {
+        if(ZKUtil.watchAndCheckExists(watcher, node)) {
+          nodeCreated(path);
+        } else {
+          this.data = null;
+        }
+      } catch(KeeperException e) {
+        abortable.abort("Unexpected exception handling nodeDeleted event", e);
+      }
+    }
+  }
+
+  @Override
+  public synchronized void nodeDataChanged(String path) {
+    if(path.equals(node)) {
+      nodeCreated(path);
+    }
+  }
+
+  /**
+   * Checks if the baseznode set as per the property 'zookeeper.znode.parent'
+   * exists.
+   * @return true if baseznode exists.
+   *         false if doesnot exists.
+   */
+  public boolean checkIfBaseNodeAvailable() {
+    try {
+      if (ZKUtil.checkExists(watcher, watcher.znodePaths.baseZNode) == -1) {
+        return false;
+      }
+    } catch (KeeperException e) {
+      abortable.abort("Exception while checking if basenode (" + watcher.znodePaths.baseZNode
+          + ") exists in ZooKeeper.",
+        e);
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "ZKNodeTracker{" +
+        "node='" + node + ", stopped=" + stopped + '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
new file mode 100644
index 0000000..8116c23
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
@@ -0,0 +1,65 @@
+/*
+ *
+ * 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/330b0d05/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
new file mode 100644
index 0000000..33cc43e
--- /dev/null
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
@@ -0,0 +1,122 @@
+/*
+  * 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 SplitLogManager and 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(ZKWatcher 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(ZKWatcher 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(ZKWatcher 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;
+  }
+}


[06/13] hbase git commit: HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
index a9896ce..2e555a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
@@ -28,7 +28,6 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.List;
 import java.util.concurrent.CountDownLatch;
 
 import org.apache.commons.logging.Log;
@@ -39,20 +38,18 @@ import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -104,7 +101,7 @@ public class TestMultiSlaveReplication {
     utility1.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = utility1.getZkCluster();
     utility1.setZkCluster(miniZK);
-    new ZooKeeperWatcher(conf1, "cluster1", null, true);
+    new ZKWatcher(conf1, "cluster1", null, true);
 
     conf2 = new Configuration(conf1);
     conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
@@ -114,11 +111,11 @@ public class TestMultiSlaveReplication {
 
     utility2 = new HBaseTestingUtility(conf2);
     utility2.setZkCluster(miniZK);
-    new ZooKeeperWatcher(conf2, "cluster2", null, true);
+    new ZKWatcher(conf2, "cluster2", null, true);
 
     utility3 = new HBaseTestingUtility(conf3);
     utility3.setZkCluster(miniZK);
-    new ZooKeeperWatcher(conf3, "cluster3", null, true);
+    new ZKWatcher(conf3, "cluster3", null, true);
 
     table = new HTableDescriptor(tableName);
     HColumnDescriptor fam = new HColumnDescriptor(famName);
@@ -190,7 +187,7 @@ public class TestMultiSlaveReplication {
     // Even if the log was rolled in the middle of the replication
     // "row" is still replication.
     checkRow(row, 1, htable2);
-    // Replication thread of cluster 2 may be sleeping, and since row2 is not there in it, 
+    // Replication thread of cluster 2 may be sleeping, and since row2 is not there in it,
     // we should wait before checking.
     checkWithWait(row, 1, htable3);
 
@@ -244,7 +241,7 @@ public class TestMultiSlaveReplication {
     region.getWAL().unregisterWALActionsListener(listener);
   }
 
- 
+
   private void checkWithWait(byte[] row, int count, Table table) throws Exception {
     Get get = new Get(row);
     for (int i = 0; i < NB_RETRIES; i++) {
@@ -267,7 +264,7 @@ public class TestMultiSlaveReplication {
       }
     }
   }
-  
+
   private void checkRow(byte[] row, int count, Table... tables) throws IOException {
     Get get = new Get(row);
     for (Table table : tables) {
@@ -299,7 +296,7 @@ public class TestMultiSlaveReplication {
       if (removedFromAll) {
         break;
       } else {
-        Thread.sleep(SLEEP_TIME);        
+        Thread.sleep(SLEEP_TIME);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
index 6572404..84ce9a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -114,7 +114,7 @@ public class TestPerTableCFReplication {
     utility1 = new HBaseTestingUtility(conf1);
     utility1.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = utility1.getZkCluster();
-    new ZooKeeperWatcher(conf1, "cluster1", null, true);
+    new ZKWatcher(conf1, "cluster1", null, true);
 
     conf2 = new Configuration(conf1);
     conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
@@ -124,11 +124,11 @@ public class TestPerTableCFReplication {
 
     utility2 = new HBaseTestingUtility(conf2);
     utility2.setZkCluster(miniZK);
-    new ZooKeeperWatcher(conf2, "cluster3", null, true);
+    new ZKWatcher(conf2, "cluster3", null, true);
 
     utility3 = new HBaseTestingUtility(conf3);
     utility3.setZkCluster(miniZK);
-    new ZooKeeperWatcher(conf3, "cluster3", null, true);
+    new ZKWatcher(conf3, "cluster3", null, true);
 
     table = new HTableDescriptor(tableName);
     HColumnDescriptor fam = new HColumnDescriptor(famName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index 58b97b9..f7d1009 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.runners.Parameterized.Parameter;
@@ -64,8 +64,8 @@ public class TestReplicationBase {
   protected static Configuration conf2;
   protected static Configuration CONF_WITH_LOCALFS;
 
-  protected static ZooKeeperWatcher zkw1;
-  protected static ZooKeeperWatcher zkw2;
+  protected static ZKWatcher zkw1;
+  protected static ZKWatcher zkw2;
 
   protected static ReplicationAdmin admin;
   protected static Admin hbaseAdmin;
@@ -127,8 +127,8 @@ public class TestReplicationBase {
     MiniZooKeeperCluster miniZK = utility1.getZkCluster();
     // Have to reget conf1 in case zk cluster location different
     // than default
-    conf1 = utility1.getConfiguration();  
-    zkw1 = new ZooKeeperWatcher(conf1, "cluster1", null, true);
+    conf1 = utility1.getConfiguration();
+    zkw1 = new ZKWatcher(conf1, "cluster1", null, true);
     admin = new ReplicationAdmin(conf1);
     LOG.info("Setup first Zk");
 
@@ -140,7 +140,7 @@ public class TestReplicationBase {
 
     utility2 = new HBaseTestingUtility(conf2);
     utility2.setZkCluster(miniZK);
-    zkw2 = new ZooKeeperWatcher(conf2, "cluster2", null, true);
+    zkw2 = new ZKWatcher(conf2, "cluster2", null, true);
     LOG.info("Setup second Zk");
 
     CONF_WITH_LOCALFS = HBaseConfiguration.create(conf1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
index cc09751..97daa63 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 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.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -54,7 +54,7 @@ public class TestReplicationStateHBaseImpl {
 
   private static Configuration conf;
   private static HBaseTestingUtility utility;
-  private static ZooKeeperWatcher zkw;
+  private static ZKWatcher zkw;
   private static String replicationZNode;
 
   private static ReplicationQueues rq1;
@@ -413,7 +413,7 @@ public class TestReplicationStateHBaseImpl {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return null;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index dde7868..bb9f665 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 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.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -58,7 +58,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
 
   private static Configuration conf;
   private static HBaseTestingUtility utility;
-  private static ZooKeeperWatcher zkw;
+  private static ZKWatcher zkw;
   private static String replicationZNode;
   private ReplicationQueuesZKImpl rqZK;
 
@@ -80,7 +80,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
     // Add a dummy region server and set up the cluster id
     Configuration testConf = new Configuration(conf);
     testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode);
-    ZooKeeperWatcher zkw1 = new ZooKeeperWatcher(testConf, "test1", null);
+    ZKWatcher zkw1 = new ZKWatcher(testConf, "test1", null);
     String fakeRs = ZNodePaths.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234");
     ZKUtil.createWithParents(zkw1, fakeRs);
     ZKClusterId.setClusterId(zkw1, new ClusterId());
@@ -152,7 +152,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return zkw;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
index 83fdad7..665eedb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -43,7 +43,7 @@ public class TestReplicationTableBase {
   private static long TIME_OUT_MILLIS = 3000;
   private static Configuration conf;
   private static HBaseTestingUtility utility;
-  private static ZooKeeperWatcher zkw;
+  private static ZKWatcher zkw;
   private static ReplicationTableBase rb;
   private static ReplicationQueues rq;
   private static ReplicationQueuesClient rqc;

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index f6e7b39..2db4f76 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 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.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -67,7 +67,7 @@ public class TestReplicationTrackerZKImpl {
   private static HBaseTestingUtility utility;
 
   // Each one of the below variables are reinitialized before every test case
-  private ZooKeeperWatcher zkw;
+  private ZKWatcher zkw;
   private ReplicationPeers rp;
   private ReplicationTracker rt;
   private AtomicInteger rsRemovedCount;
@@ -82,7 +82,7 @@ public class TestReplicationTrackerZKImpl {
     utility = new HBaseTestingUtility();
     utility.startMiniZKCluster();
     conf = utility.getConfiguration();
-    ZooKeeperWatcher zk = HBaseTestingUtility.getZooKeeperWatcher(utility);
+    ZKWatcher zk = HBaseTestingUtility.getZooKeeperWatcher(utility);
     ZKUtil.createWithParents(zk, zk.znodePaths.rsZNode);
   }
 
@@ -193,7 +193,7 @@ public class TestReplicationTrackerZKImpl {
     int exists = 0;
     int hyphen = 0;
     rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    
+
     try{
       rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
     }catch(IllegalArgumentException e){
@@ -207,11 +207,11 @@ public class TestReplicationTrackerZKImpl {
     }
     assertEquals(1, exists);
     assertEquals(1, hyphen);
-    
+
     // clean up
     rp.unregisterPeer("6");
   }
-  
+
   private class DummyReplicationListener implements ReplicationListener {
 
     @Override
@@ -252,7 +252,7 @@ public class TestReplicationTrackerZKImpl {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return zkw;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index 7a8e639..1c5aa71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -102,14 +102,14 @@ public class TestSerialReplication {
     utility1 = new HBaseTestingUtility(conf1);
     utility1.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = utility1.getZkCluster();
-    new ZooKeeperWatcher(conf1, "cluster1", null, true);
+    new ZKWatcher(conf1, "cluster1", null, true);
 
     conf2 = new Configuration(conf1);
     conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
 
     utility2 = new HBaseTestingUtility(conf2);
     utility2.setZkCluster(miniZK);
-    new ZooKeeperWatcher(conf2, "cluster2", null, true);
+    new ZKWatcher(conf2, "cluster2", null, true);
 
     utility1.startMiniCluster(1, 10);
     utility2.startMiniCluster(1, 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
index 1a02317..cb895ca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -55,7 +55,7 @@ public class TestTableCFsUpdater extends TableCFsUpdater {
   private static final Log LOG = LogFactory.getLog(TestTableCFsUpdater.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  private static ZooKeeperWatcher zkw = null;
+  private static ZKWatcher zkw = null;
   private static Abortable abortable = null;
 
   @Rule
@@ -80,7 +80,7 @@ public class TestTableCFsUpdater extends TableCFsUpdater {
         return false;
       }
     };
-    zkw = new ZooKeeperWatcher(conf, "TableCFs", abortable, true);
+    zkw = new ZKWatcher(conf, "TableCFs", abortable, true);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java
index 2469c7c..9b1648f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestGlobalThrottler.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -86,14 +86,14 @@ public class TestGlobalThrottler {
     utility1 = new HBaseTestingUtility(conf1);
     utility1.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = utility1.getZkCluster();
-    new ZooKeeperWatcher(conf1, "cluster1", null, true);
+    new ZKWatcher(conf1, "cluster1", null, true);
 
     conf2 = new Configuration(conf1);
     conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
 
     utility2 = new HBaseTestingUtility(conf2);
     utility2.setZkCluster(miniZK);
-    new ZooKeeperWatcher(conf2, "cluster2", null, true);
+    new ZKWatcher(conf2, "cluster2", null, true);
 
     ReplicationAdmin admin1 = new ReplicationAdmin(conf1);
     ReplicationPeerConfig rpc = new ReplicationPeerConfig();

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index dd1e19a..7730100 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -83,7 +83,7 @@ import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -116,7 +116,7 @@ public abstract class TestReplicationSourceManager {
 
   protected static ReplicationSourceManager manager;
 
-  protected static ZooKeeperWatcher zkw;
+  protected static ZKWatcher zkw;
 
   protected static HTableDescriptor htd;
 
@@ -149,7 +149,7 @@ public abstract class TestReplicationSourceManager {
   protected static void setupZkAndReplication() throws Exception {
     // The implementing class should set up the conf
     assertNotNull(conf);
-    zkw = new ZooKeeperWatcher(conf, "test", null);
+    zkw = new ZKWatcher(conf, "test", null);
     ZKUtil.createWithParents(zkw, "/hbase/replication");
     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1");
     ZKUtil.setData(zkw, "/hbase/replication/peers/1",
@@ -668,7 +668,7 @@ public abstract class TestReplicationSourceManager {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return zkw;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
index ad8cb14..f9d0b1c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.TestTableName;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -509,7 +509,7 @@ public class TestAccessController2 extends SecureTestUtil {
 
     // Namespace needs this, as they follow the lazy creation of ACL znode.
     grantOnNamespace(TEST_UTIL, TESTGROUP1_USER1.getShortName(), ns, Action.ADMIN);
-    ZooKeeperWatcher zkw = TEST_UTIL.getMiniHBaseCluster().getMaster().getZooKeeper();
+    ZKWatcher zkw = TEST_UTIL.getMiniHBaseCluster().getMaster().getZooKeeper();
     assertTrue("The acl znode for table should exist",  ZKUtil.checkExists(zkw, baseAclZNode +
         table.getNameAsString()) != -1);
     assertTrue("The acl znode for namespace should exist", ZKUtil.checkExists(zkw, baseAclZNode +

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
index 30e8396..607ea8c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
 public class TestTablePermissions {
   private static final Log LOG = LogFactory.getLog(TestTablePermissions.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static ZooKeeperWatcher ZKW;
+  private static ZKWatcher ZKW;
   private final static Abortable ABORTABLE = new Abortable() {
     private final AtomicBoolean abort = new AtomicBoolean(false);
 
@@ -97,7 +97,7 @@ public class TestTablePermissions {
     // Wait for the ACL table to become available
     UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
 
-    ZKW = new ZooKeeperWatcher(UTIL.getConfiguration(),
+    ZKW = new ZKWatcher(UTIL.getConfiguration(),
       "TestTablePermissions", ABORTABLE);
 
     UTIL.createTable(TEST_TABLE, TEST_FAMILY);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
index 76de0c6..18fb15f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -75,9 +75,9 @@ public class TestZKPermissionWatcher {
 
     // start minicluster
     UTIL.startMiniCluster();
-    AUTH_A = TableAuthManager.getOrCreate(new ZooKeeperWatcher(conf,
+    AUTH_A = TableAuthManager.getOrCreate(new ZKWatcher(conf,
       "TestZKPermissionsWatcher_1", ABORTABLE), conf);
-    AUTH_B = TableAuthManager.getOrCreate(new ZooKeeperWatcher(conf,
+    AUTH_B = TableAuthManager.getOrCreate(new ZKWatcher(conf,
       "TestZKPermissionsWatcher_2", ABORTABLE), conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index 5a0b6bd..3e0bddc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -76,7 +76,7 @@ import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.Service;
@@ -127,7 +127,7 @@ public class TestTokenAuthentication {
     private HBaseTestingUtility TEST_UTIL;
     private RpcServerInterface rpcServer;
     private InetSocketAddress isa;
-    private ZooKeeperWatcher zookeeper;
+    private ZKWatcher zookeeper;
     private Sleeper sleeper;
     private boolean started = false;
     private boolean aborted = false;
@@ -220,7 +220,7 @@ public class TestTokenAuthentication {
     }
 
     @Override
-    public ZooKeeperWatcher getZooKeeper() {
+    public ZKWatcher getZooKeeper() {
       return zookeeper;
     }
 
@@ -261,7 +261,7 @@ public class TestTokenAuthentication {
       // ZK configuration must _not_ have hbase.security.authentication or it will require SASL auth
       Configuration zkConf = new Configuration(conf);
       zkConf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
-      this.zookeeper = new ZooKeeperWatcher(zkConf, TokenServer.class.getSimpleName(),
+      this.zookeeper = new ZKWatcher(zkConf, TokenServer.class.getSimpleName(),
           this, true);
       this.rpcServer.start();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
index 857cdd0..ea07338 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -71,13 +71,13 @@ public class TestZKSecretWatcher {
   private static class AuthenticationTokenSecretManagerForTest
   extends AuthenticationTokenSecretManager {
     private CountDownLatch latch = new CountDownLatch(1);
-    
+
     public AuthenticationTokenSecretManagerForTest(Configuration conf,
-        ZooKeeperWatcher zk, String serverName,
-        long keyUpdateInterval, long tokenMaxLifetime) {
+                                                   ZKWatcher zk, String serverName,
+                                                   long keyUpdateInterval, long tokenMaxLifetime) {
       super(conf, zk, serverName, keyUpdateInterval, tokenMaxLifetime);
     }
-    
+
     @Override
     synchronized boolean removeKey(Integer keyId) {
       boolean b = super.removeKey(keyId);
@@ -86,19 +86,19 @@ public class TestZKSecretWatcher {
       }
       return b;
     }
-    
+
     CountDownLatch getLatch() {
       return latch;
     }
   }
-  
+
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
     TEST_UTIL.startMiniZKCluster();
     Configuration conf = TEST_UTIL.getConfiguration();
 
-    ZooKeeperWatcher zk = newZK(conf, "server1", new MockAbortable());
+    ZKWatcher zk = newZK(conf, "server1", new MockAbortable());
     AuthenticationTokenSecretManagerForTest[] tmp = new AuthenticationTokenSecretManagerForTest[2];
     tmp[0] = new AuthenticationTokenSecretManagerForTest(
         conf, zk, "server1", 60*60*1000, 60*1000);
@@ -179,7 +179,7 @@ public class TestZKSecretWatcher {
 
     // bring up a new slave
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZooKeeperWatcher zk = newZK(conf, "server3", new MockAbortable());
+    ZKWatcher zk = newZK(conf, "server3", new MockAbortable());
     KEY_SLAVE2 = new AuthenticationTokenSecretManager(
         conf, zk, "server3", 60*60*1000, 60*1000);
     KEY_SLAVE2.start();
@@ -233,7 +233,7 @@ public class TestZKSecretWatcher {
     assertTrue(newCurrent.getKeyId() > current.getKeyId());
 
     // add another slave
-    ZooKeeperWatcher zk3 = newZK(conf, "server4", new MockAbortable());
+    ZKWatcher zk3 = newZK(conf, "server4", new MockAbortable());
     KEY_SLAVE3 = new AuthenticationTokenSecretManager(
         conf, zk3, "server4", 60*60*1000, 60*1000);
     KEY_SLAVE3.start();
@@ -275,10 +275,10 @@ public class TestZKSecretWatcher {
     assertTrue(newCurrent2.getKeyId() > current2.getKeyId());
   }
 
-  private static ZooKeeperWatcher newZK(Configuration conf, String name,
-      Abortable abort) throws Exception {
+  private static ZKWatcher newZK(Configuration conf, String name,
+                                 Abortable abort) throws Exception {
     Configuration copy = HBaseConfiguration.create(conf);
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(copy, name, abort);
+    ZKWatcher zk = new ZKWatcher(copy, name, abort);
     return zk;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
index fcadc9d..1757ddd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Writables;
 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.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -55,7 +55,7 @@ public class TestZKSecretWatcherRefreshKeys {
       return abort;
     }
   }
-  
+
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
@@ -67,19 +67,19 @@ public class TestZKSecretWatcherRefreshKeys {
     TEST_UTIL.shutdownMiniZKCluster();
   }
 
-  private static ZooKeeperWatcher newZK(Configuration conf, String name,
-      Abortable abort) throws Exception {
+  private static ZKWatcher newZK(Configuration conf, String name,
+                                 Abortable abort) throws Exception {
     Configuration copy = HBaseConfiguration.create(conf);
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(copy, name, abort);
+    ZKWatcher zk = new ZKWatcher(copy, name, abort);
     return zk;
   }
 
   @Test
   public void testRefreshKeys() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZooKeeperWatcher zk = newZK(conf, "127.0.0.1", new MockAbortable());
-    AuthenticationTokenSecretManager keyManager = 
-        new AuthenticationTokenSecretManager(conf, zk, "127.0.0.1", 
+    ZKWatcher zk = newZK(conf, "127.0.0.1", new MockAbortable());
+    AuthenticationTokenSecretManager keyManager =
+        new AuthenticationTokenSecretManager(conf, zk, "127.0.0.1",
             60 * 60 * 1000, 60 * 1000);
     ZKSecretWatcher watcher = new ZKSecretWatcher(conf, zk, keyManager);
     ZKUtil.deleteChildrenRecursively(zk, watcher.getKeysParentZNode());

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
index e5b1c39..c2da3ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.Before;
 import org.junit.experimental.categories.Category;
 
@@ -109,7 +109,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
     TEST_UTIL = new HBaseTestingUtility(conf);
     TEST_UTIL.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
-    zkw1 = new ZooKeeperWatcher(conf, "cluster1", null, true);
+    zkw1 = new ZKWatcher(conf, "cluster1", null, true);
     admin = TEST_UTIL.getAdmin();
 
     // Base conf2 on conf1 so it gets the right zk cluster.
@@ -124,7 +124,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
     setVisibilityLabelServiceImpl(conf1, ExpAsStringVisibilityLabelServiceImpl.class);
     TEST_UTIL1 = new HBaseTestingUtility(conf1);
     TEST_UTIL1.setZkCluster(miniZK);
-    zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true);
+    zkw2 = new ZKWatcher(conf1, "cluster2", null, true);
 
     TEST_UTIL.startMiniCluster(1);
     // Wait for the labels table to become available

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
index 513c765..678227b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
@@ -74,7 +74,7 @@ import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -109,8 +109,8 @@ public class TestVisibilityLabelsReplication {
   public final static byte[] fam = Bytes.toBytes("info");
   public final static byte[] qual = Bytes.toBytes("qual");
   public final static byte[] value = Bytes.toBytes("value");
-  protected static ZooKeeperWatcher zkw1;
-  protected static ZooKeeperWatcher zkw2;
+  protected static ZKWatcher zkw1;
+  protected static ZKWatcher zkw2;
   protected static int expected[] = { 4, 6, 4, 0, 3 };
   private static final String NON_VISIBILITY = "non-visibility";
   protected static String[] expectedVisString = {
@@ -160,7 +160,7 @@ public class TestVisibilityLabelsReplication {
     TEST_UTIL = new HBaseTestingUtility(conf);
     TEST_UTIL.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
-    zkw1 = new ZooKeeperWatcher(conf, "cluster1", null, true);
+    zkw1 = new ZKWatcher(conf, "cluster1", null, true);
     admin = TEST_UTIL.getAdmin();
 
     // Base conf2 on conf1 so it gets the right zk cluster.
@@ -176,7 +176,7 @@ public class TestVisibilityLabelsReplication {
     USER1 = User.createUserForTesting(conf1, "user1", new String[] {});
     TEST_UTIL1 = new HBaseTestingUtility(conf1);
     TEST_UTIL1.setZkCluster(miniZK);
-    zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true);
+    zkw2 = new ZKWatcher(conf1, "cluster2", null, true);
 
     TEST_UTIL.startMiniCluster(1);
     // Wait for the labels table to become available

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index db44219..4acec76 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
 /**
  * Basic mock Server for handler tests.
@@ -39,10 +39,10 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 public class MockServer implements Server {
   private static final Log LOG = LogFactory.getLog(MockServer.class);
   final static ServerName NAME = ServerName.valueOf("MockServer", 123, -1);
-  
+
   boolean stopped;
   boolean aborted;
-  final ZooKeeperWatcher zk;
+  final ZKWatcher zk;
   final HBaseTestingUtility htu;
 
   @SuppressWarnings("unused")
@@ -66,7 +66,7 @@ public class MockServer implements Server {
   throws ZooKeeperConnectionException, IOException {
     this.htu = htu;
     this.zk = zkw?
-      new ZooKeeperWatcher(htu.getConfiguration(), NAME.toString(), this, true):
+      new ZKWatcher(htu.getConfiguration(), NAME.toString(), this, true):
       null;
   }
 
@@ -94,7 +94,7 @@ public class MockServer implements Server {
   }
 
   @Override
-  public ZooKeeperWatcher getZooKeeper() {
+  public ZKWatcher getZooKeeper() {
     return this.zk;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index 7463da1..ba3475e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -86,7 +86,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
 import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -1493,7 +1493,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
     Assert.assertEquals(1, replicationAdmin.getPeersCount());
 
     // create replicator
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "Test Hbase Fsck", connection);
+    ZKWatcher zkw = new ZKWatcher(conf, "Test Hbase Fsck", connection);
     ReplicationQueues repQueues =
         ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, connection,
           zkw));

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
index e71210d..6fa99fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java
@@ -71,7 +71,7 @@ public class TestRecoverableZooKeeper {
   public void testSetDataVersionMismatchInLoop() throws Exception {
     String znode = "/hbase/splitWAL/9af7cfc9b15910a0b3d714bf40a3248f";
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testSetDataVersionMismatchInLoop",
+    ZKWatcher zkw = new ZKWatcher(conf, "testSetDataVersionMismatchInLoop",
         abortable, true);
     String ensemble = ZKConfig.getZKQuorumServersString(conf);
     RecoverableZooKeeper rzk = ZKUtil.connect(conf, ensemble, zkw);

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java
index 1faf8e5..69915a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java
@@ -61,12 +61,12 @@ public class TestZKLeaderManager {
 
   private static class MockLeader extends Thread implements Stoppable {
     private boolean stopped;
-    private ZooKeeperWatcher watcher;
+    private ZKWatcher watcher;
     private ZKLeaderManager zkLeader;
     private AtomicBoolean master = new AtomicBoolean(false);
     private int index;
 
-    public MockLeader(ZooKeeperWatcher watcher, int index) {
+    public MockLeader(ZKWatcher watcher, int index) {
       setDaemon(true);
       setName("TestZKLeaderManager-leader-" + index);
       this.index = index;
@@ -83,7 +83,7 @@ public class TestZKLeaderManager {
       return index;
     }
 
-    public ZooKeeperWatcher getWatcher() {
+    public ZKWatcher getWatcher() {
       return watcher;
     }
 
@@ -132,7 +132,7 @@ public class TestZKLeaderManager {
     MockAbortable abortable = new MockAbortable();
     CANDIDATES = new MockLeader[3];
     for (int i = 0; i < 3; i++) {
-      ZooKeeperWatcher watcher = newZK(conf, "server"+i, abortable);
+      ZKWatcher watcher = newZK(conf, "server"+i, abortable);
       CANDIDATES[i] = new MockLeader(watcher, i);
       CANDIDATES[i].start();
     }
@@ -225,10 +225,10 @@ public class TestZKLeaderManager {
     return currentLeader;
   }
 
-  private static ZooKeeperWatcher newZK(Configuration conf, String name,
-      Abortable abort) throws Exception {
+  private static ZKWatcher newZK(Configuration conf, String name,
+                                 Abortable abort) throws Exception {
     Configuration copy = HBaseConfiguration.create(conf);
-    ZooKeeperWatcher zk = new ZooKeeperWatcher(copy, name, abort);
+    ZKWatcher zk = new ZKWatcher(copy, name, abort);
     return zk;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java
new file mode 100644
index 0000000..c99bd05
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java
@@ -0,0 +1,118 @@
+/**
+ *
+ * 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.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.security.Permission;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestZKMainServer {
+  // ZKMS calls System.exit.  Catch the call and prevent exit using trick described up in
+  // http://stackoverflow.com/questions/309396/java-how-to-test-methods-that-call-system-exit
+  protected static class ExitException extends SecurityException {
+    private static final long serialVersionUID = 1L;
+    public final int status;
+    public ExitException(int status) {
+      super("There is no escape!");
+      this.status = status;
+    }
+  }
+
+  private static class NoExitSecurityManager extends SecurityManager {
+    @Override
+    public void checkPermission(Permission perm) {
+      // allow anything.
+    }
+
+    @Override
+    public void checkPermission(Permission perm, Object context) {
+      // allow anything.
+    }
+
+    @Override
+    public void checkExit(int status) {
+      super.checkExit(status);
+      throw new ExitException(status);
+    }
+  }
+
+  /**
+   * We need delete of a znode to work at least.
+   * @throws Exception
+   */
+  @Test
+  public void testCommandLineWorks() throws Exception {
+    System.setSecurityManager(new NoExitSecurityManager());
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    htu.getConfiguration().setInt(HConstants.ZK_SESSION_TIMEOUT, 1000);
+    htu.startMiniZKCluster();
+    try {
+      ZKWatcher zkw = htu.getZooKeeperWatcher();
+      String znode = "/testCommandLineWorks";
+      ZKUtil.createWithParents(zkw, znode, HConstants.EMPTY_BYTE_ARRAY);
+      ZKUtil.checkExists(zkw, znode);
+      boolean exception = false;
+      try {
+        ZKMainServer.main(new String [] {"-server",
+          "localhost:" + htu.getZkCluster().getClientPort(), "delete", znode});
+      } catch (ExitException ee) {
+        // ZKMS calls System.exit which should trigger this exception.
+        exception = true;
+      }
+      assertTrue(exception);
+      assertEquals(-1, ZKUtil.checkExists(zkw, znode));
+    } finally {
+      htu.shutdownMiniZKCluster();
+      System.setSecurityManager(null); // or save and restore original
+    }
+  }
+
+  @Test
+  public void testHostPortParse() {
+    ZKMainServer parser = new ZKMainServer();
+    Configuration c = HBaseConfiguration.create();
+    assertEquals("localhost:" + c.get(HConstants.ZOOKEEPER_CLIENT_PORT), parser.parse(c));
+    final String port = "1234";
+    c.set(HConstants.ZOOKEEPER_CLIENT_PORT, port);
+    c.set("hbase.zookeeper.quorum", "example.com");
+    assertEquals("example.com:" + port, parser.parse(c));
+    c.set("hbase.zookeeper.quorum", "example1.com,example2.com,example3.com");
+    String ensemble = parser.parse(c);
+    assertTrue(port, ensemble.matches("(example[1-3]\\.com:1234,){2}example[1-3]\\.com:" + port));
+
+    // multiple servers with its own port
+    c.set("hbase.zookeeper.quorum", "example1.com:5678,example2.com:9012,example3.com:3456");
+    ensemble = parser.parse(c);
+    assertEquals(ensemble, "example1.com:5678,example2.com:9012,example3.com:3456");
+
+    // some servers without its own port, which will be assigned the default client port
+    c.set("hbase.zookeeper.quorum", "example1.com:5678,example2.com:9012,example3.com");
+    ensemble = parser.parse(c);
+    assertEquals(ensemble, "example1.com:5678,example2.com:9012,example3.com:" + port);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
index 7971417..614d4be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
@@ -53,7 +53,7 @@ import org.junit.experimental.categories.Category;
 public class TestZKMulti {
   private static final Log LOG = LogFactory.getLog(TestZKMulti.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static ZooKeeperWatcher zkw = null;
+  private static ZKWatcher zkw = null;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -70,7 +70,7 @@ public class TestZKMulti {
         return false;
       }
     };
-    zkw = new ZooKeeperWatcher(conf,
+    zkw = new ZKWatcher(conf,
       "TestZKMulti", abortable, true);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java
new file mode 100644
index 0000000..ac5c11a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java
@@ -0,0 +1,351 @@
+/**
+ *
+ * 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.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.Semaphore;
+
+import junit.framework.Assert;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.master.TestActiveMasterManager.NodeDeletionListener;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, MediumTests.class})
+public class TestZKNodeTracker {
+  private static final Log LOG = LogFactory.getLog(TestZKNodeTracker.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private final static Random rand = new Random();
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniZKCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+
+  /**
+   * Test that we can interrupt a node that is blocked on a wait.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test public void testInterruptible() throws IOException, InterruptedException {
+    Abortable abortable = new StubAbortable();
+    ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
+      "testInterruptible", abortable);
+    final TestTracker tracker = new TestTracker(zk, "/xyz", abortable);
+    tracker.start();
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        try {
+          tracker.blockUntilAvailable();
+        } catch (InterruptedException e) {
+          throw new RuntimeException("Interrupted", e);
+        }
+      }
+    };
+    t.start();
+    while (!t.isAlive()) Threads.sleep(1);
+    tracker.stop();
+    t.join();
+    // If it wasn't interruptible, we'd never get to here.
+  }
+
+  @Test
+  public void testNodeTracker() throws Exception {
+    Abortable abortable = new StubAbortable();
+    ZKWatcher zk = new ZKWatcher(TEST_UTIL.getConfiguration(),
+        "testNodeTracker", abortable);
+    ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode);
+
+    final String node =
+      ZNodePaths.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString());
+
+    final byte [] dataOne = Bytes.toBytes("dataOne");
+    final byte [] dataTwo = Bytes.toBytes("dataTwo");
+
+    // Start a ZKNT with no node currently available
+    TestTracker localTracker = new TestTracker(zk, node, abortable);
+    localTracker.start();
+    zk.registerListener(localTracker);
+
+    // Make sure we don't have a node
+    assertNull(localTracker.getData(false));
+
+    // Spin up a thread with another ZKNT and have it block
+    WaitToGetDataThread thread = new WaitToGetDataThread(zk, node);
+    thread.start();
+
+    // Verify the thread doesn't have a node
+    assertFalse(thread.hasData);
+
+    // Now, start a new ZKNT with the node already available
+    TestTracker secondTracker = new TestTracker(zk, node, null);
+    secondTracker.start();
+    zk.registerListener(secondTracker);
+
+    // Put up an additional zk listener so we know when zk event is done
+    TestingZKListener zkListener = new TestingZKListener(zk, node);
+    zk.registerListener(zkListener);
+    assertEquals(0, zkListener.createdLock.availablePermits());
+
+    // Create a completely separate zk connection for test triggers and avoid
+    // any weird watcher interactions from the test
+    final ZooKeeper zkconn = new ZooKeeper(
+        ZKConfig.getZKQuorumServersString(TEST_UTIL.getConfiguration()), 60000,
+        new StubWatcher());
+
+    // Add the node with data one
+    zkconn.create(node, dataOne, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+    // Wait for the zk event to be processed
+    zkListener.waitForCreation();
+    thread.join();
+
+    // Both trackers should have the node available with data one
+    assertNotNull(localTracker.getData(false));
+    assertNotNull(localTracker.blockUntilAvailable());
+    assertTrue(Bytes.equals(localTracker.getData(false), dataOne));
+    assertTrue(thread.hasData);
+    assertTrue(Bytes.equals(thread.tracker.getData(false), dataOne));
+    LOG.info("Successfully got data one");
+
+    // Make sure it's available and with the expected data
+    assertNotNull(secondTracker.getData(false));
+    assertNotNull(secondTracker.blockUntilAvailable());
+    assertTrue(Bytes.equals(secondTracker.getData(false), dataOne));
+    LOG.info("Successfully got data one with the second tracker");
+
+    // Drop the node
+    zkconn.delete(node, -1);
+    zkListener.waitForDeletion();
+
+    // Create a new thread but with the existing thread's tracker to wait
+    TestTracker threadTracker = thread.tracker;
+    thread = new WaitToGetDataThread(zk, node, threadTracker);
+    thread.start();
+
+    // Verify other guys don't have data
+    assertFalse(thread.hasData);
+    assertNull(secondTracker.getData(false));
+    assertNull(localTracker.getData(false));
+    LOG.info("Successfully made unavailable");
+
+    // Create with second data
+    zkconn.create(node, dataTwo, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+    // Wait for the zk event to be processed
+    zkListener.waitForCreation();
+    thread.join();
+
+    // All trackers should have the node available with data two
+    assertNotNull(localTracker.getData(false));
+    assertNotNull(localTracker.blockUntilAvailable());
+    assertTrue(Bytes.equals(localTracker.getData(false), dataTwo));
+    assertNotNull(secondTracker.getData(false));
+    assertNotNull(secondTracker.blockUntilAvailable());
+    assertTrue(Bytes.equals(secondTracker.getData(false), dataTwo));
+    assertTrue(thread.hasData);
+    assertTrue(Bytes.equals(thread.tracker.getData(false), dataTwo));
+    LOG.info("Successfully got data two on all trackers and threads");
+
+    // Change the data back to data one
+    zkconn.setData(node, dataOne, -1);
+
+    // Wait for zk event to be processed
+    zkListener.waitForDataChange();
+
+    // All trackers should have the node available with data one
+    assertNotNull(localTracker.getData(false));
+    assertNotNull(localTracker.blockUntilAvailable());
+    assertTrue(Bytes.equals(localTracker.getData(false), dataOne));
+    assertNotNull(secondTracker.getData(false));
+    assertNotNull(secondTracker.blockUntilAvailable());
+    assertTrue(Bytes.equals(secondTracker.getData(false), dataOne));
+    assertTrue(thread.hasData);
+    assertTrue(Bytes.equals(thread.tracker.getData(false), dataOne));
+    LOG.info("Successfully got data one following a data change on all trackers and threads");
+  }
+
+  public static class WaitToGetDataThread extends Thread {
+
+    TestTracker tracker;
+    boolean hasData;
+
+    public WaitToGetDataThread(ZKWatcher zk, String node) {
+      tracker = new TestTracker(zk, node, null);
+      tracker.start();
+      zk.registerListener(tracker);
+      hasData = false;
+    }
+
+    public WaitToGetDataThread(ZKWatcher zk, String node,
+                               TestTracker tracker) {
+      this.tracker = tracker;
+      hasData = false;
+    }
+
+    @Override
+    public void run() {
+      LOG.info("Waiting for data to be available in WaitToGetDataThread");
+      try {
+        tracker.blockUntilAvailable();
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      LOG.info("Data now available in tracker from WaitToGetDataThread");
+      hasData = true;
+    }
+  }
+
+  public static class TestTracker extends ZKNodeTracker {
+    public TestTracker(ZKWatcher watcher, String node,
+                       Abortable abortable) {
+      super(watcher, node, abortable);
+    }
+  }
+
+  public static class TestingZKListener extends ZKListener {
+    private static final Log LOG = LogFactory.getLog(NodeDeletionListener.class);
+
+    private Semaphore deletedLock;
+    private Semaphore createdLock;
+    private Semaphore changedLock;
+    private String node;
+
+    public TestingZKListener(ZKWatcher watcher, String node) {
+      super(watcher);
+      deletedLock = new Semaphore(0);
+      createdLock = new Semaphore(0);
+      changedLock = new Semaphore(0);
+      this.node = node;
+    }
+
+    @Override
+    public void nodeDeleted(String path) {
+      if(path.equals(node)) {
+        LOG.debug("nodeDeleted(" + path + ")");
+        deletedLock.release();
+      }
+    }
+
+    @Override
+    public void nodeCreated(String path) {
+      if(path.equals(node)) {
+        LOG.debug("nodeCreated(" + path + ")");
+        createdLock.release();
+      }
+    }
+
+    @Override
+    public void nodeDataChanged(String path) {
+      if(path.equals(node)) {
+        LOG.debug("nodeDataChanged(" + path + ")");
+        changedLock.release();
+      }
+    }
+
+    public void waitForDeletion() throws InterruptedException {
+      deletedLock.acquire();
+    }
+
+    public void waitForCreation() throws InterruptedException {
+      createdLock.acquire();
+    }
+
+    public void waitForDataChange() throws InterruptedException {
+      changedLock.acquire();
+    }
+  }
+
+  public static class StubAbortable implements Abortable {
+    @Override
+    public void abort(final String msg, final Throwable t) {}
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+  }
+
+  public static class StubWatcher implements Watcher {
+    @Override
+    public void process(WatchedEvent event) {}
+  }
+
+  @Test
+  public void testCleanZNode() throws Exception {
+    ZKWatcher zkw = new ZKWatcher(TEST_UTIL.getConfiguration(),
+        "testNodeTracker", new TestZKNodeTracker.StubAbortable());
+
+    final ServerName sn = ServerName.valueOf("127.0.0.1:52", 45L);
+
+    ZKUtil.createAndFailSilent(zkw,
+        TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT,
+            HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT));
+
+    final String nodeName =  zkw.znodePaths.masterAddressZNode;
+
+    // Check that we manage the case when there is no data
+    ZKUtil.createAndFailSilent(zkw, nodeName);
+    MasterAddressTracker.deleteIfEquals(zkw, sn.toString());
+    Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null);
+
+    // Check that we don't delete if we're not supposed to
+    ZKUtil.setData(zkw, nodeName, MasterAddressTracker.toByteArray(sn, 0));
+    MasterAddressTracker.deleteIfEquals(zkw, ServerName.valueOf("127.0.0.2:52", 45L).toString());
+    Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null);
+
+    // Check that we delete when we're supposed to
+    ZKUtil.setData(zkw, nodeName,MasterAddressTracker.toByteArray(sn, 0));
+    MasterAddressTracker.deleteIfEquals(zkw, sn.toString());
+    Assert.assertTrue( ZKUtil.getData(zkw, nodeName)== null );
+
+    // Check that we support the case when the znode does not exist
+    MasterAddressTracker.deleteIfEquals(zkw, sn.toString()); // must not throw an exception
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
index 05ad73e..c5bce00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
@@ -49,9 +49,9 @@ public class TestZooKeeperACL {
   private final static HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
-  private static ZooKeeperWatcher zkw;
+  private static ZKWatcher zkw;
   private static boolean secureZKAvailable;
-  
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     File saslConfFile = File.createTempFile("tmp", "jaas.conf");
@@ -76,7 +76,7 @@ public class TestZooKeeperACL {
     TEST_UTIL.getConfiguration().setInt("hbase.zookeeper.property.maxClientCnxns", 1000);
 
     // If Hadoop is missing HADOOP-7070 the cluster will fail to start due to
-    // the JAAS configuration required by ZK being clobbered by Hadoop 
+    // the JAAS configuration required by ZK being clobbered by Hadoop
     try {
       TEST_UTIL.startMiniCluster();
     } catch (IOException e) {
@@ -84,7 +84,7 @@ public class TestZooKeeperACL {
       secureZKAvailable = false;
       return;
     }
-    zkw = new ZooKeeperWatcher(
+    zkw = new ZKWatcher(
       new Configuration(TEST_UTIL.getConfiguration()),
         TestZooKeeper.class.getName(), null);
   }
@@ -112,7 +112,7 @@ public class TestZooKeeperACL {
   }
 
   /**
-   * Create a node and check its ACL. When authentication is enabled on 
+   * Create a node and check its ACL. When authentication is enabled on
    * ZooKeeper, all nodes (except /hbase/root-region-server, /hbase/master
    * and /hbase/hbaseid) should be created so that only the hbase server user
    * (master or region server user) that created them can access them, and
@@ -285,7 +285,7 @@ public class TestZooKeeperACL {
     assertEquals(testJaasConfig, false);
     saslConfFile.delete();
   }
-  
+
   /**
    * Check if Programmatic way of setting zookeeper security settings is valid.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/330b0d05/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java
deleted file mode 100644
index 368919c..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java
+++ /dev/null
@@ -1,118 +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.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.security.Permission;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestZooKeeperMainServer {
-  // ZKMS calls System.exit.  Catch the call and prevent exit using trick described up in
-  // http://stackoverflow.com/questions/309396/java-how-to-test-methods-that-call-system-exit
-  protected static class ExitException extends SecurityException {
-    private static final long serialVersionUID = 1L;
-    public final int status;
-    public ExitException(int status) {
-      super("There is no escape!");
-      this.status = status;
-    }
-  }
-
-  private static class NoExitSecurityManager extends SecurityManager {
-    @Override
-    public void checkPermission(Permission perm) {
-      // allow anything.
-    }
-
-    @Override
-    public void checkPermission(Permission perm, Object context) {
-      // allow anything.
-    }
-
-    @Override
-    public void checkExit(int status) {
-      super.checkExit(status);
-      throw new ExitException(status);
-    }
-  }
-
-  /**
-   * We need delete of a znode to work at least.
-   * @throws Exception
-   */
-  @Test
-  public void testCommandLineWorks() throws Exception {
-    System.setSecurityManager(new NoExitSecurityManager());
-    HBaseTestingUtility htu = new HBaseTestingUtility();
-    htu.getConfiguration().setInt(HConstants.ZK_SESSION_TIMEOUT, 1000);
-    htu.startMiniZKCluster();
-    try {
-      ZooKeeperWatcher zkw = htu.getZooKeeperWatcher();
-      String znode = "/testCommandLineWorks";
-      ZKUtil.createWithParents(zkw, znode, HConstants.EMPTY_BYTE_ARRAY);
-      ZKUtil.checkExists(zkw, znode);
-      boolean exception = false;
-      try {
-        ZooKeeperMainServer.main(new String [] {"-server",
-          "localhost:" + htu.getZkCluster().getClientPort(), "delete", znode});
-      } catch (ExitException ee) {
-        // ZKMS calls System.exit which should trigger this exception.
-        exception = true;
-      }
-      assertTrue(exception);
-      assertEquals(-1, ZKUtil.checkExists(zkw, znode));
-    } finally {
-      htu.shutdownMiniZKCluster();
-      System.setSecurityManager(null); // or save and restore original
-    }
-  }
-
-  @Test
-  public void testHostPortParse() {
-    ZooKeeperMainServer parser = new ZooKeeperMainServer();
-    Configuration c = HBaseConfiguration.create();
-    assertEquals("localhost:" + c.get(HConstants.ZOOKEEPER_CLIENT_PORT), parser.parse(c));
-    final String port = "1234";
-    c.set(HConstants.ZOOKEEPER_CLIENT_PORT, port);
-    c.set("hbase.zookeeper.quorum", "example.com");
-    assertEquals("example.com:" + port, parser.parse(c));
-    c.set("hbase.zookeeper.quorum", "example1.com,example2.com,example3.com");
-    String ensemble = parser.parse(c);
-    assertTrue(port, ensemble.matches("(example[1-3]\\.com:1234,){2}example[1-3]\\.com:" + port));
-
-    // multiple servers with its own port
-    c.set("hbase.zookeeper.quorum", "example1.com:5678,example2.com:9012,example3.com:3456");
-    ensemble = parser.parse(c);
-    assertEquals(ensemble, "example1.com:5678,example2.com:9012,example3.com:3456");
-
-    // some servers without its own port, which will be assigned the default client port
-    c.set("hbase.zookeeper.quorum", "example1.com:5678,example2.com:9012,example3.com");
-    ensemble = parser.parse(c);
-    assertEquals(ensemble, "example1.com:5678,example2.com:9012,example3.com:" + port);
-  }
-}