You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/01/15 10:33:27 UTC

[01/48] hbase git commit: HBASE-19771 restore_snapshot shell command gives wrong namespace if the namespace doesn't exist [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19397-branch-2 9251677ae -> 650a01da7 (forced update)


HBASE-19771 restore_snapshot shell command gives wrong namespace if the namespace doesn't exist

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: f7fbc168a1538ab4157db8aeeeee8fa9f7827774
Parents: 72702ee
Author: Janos Gub <jg...@hortonworks.com>
Authored: Fri Jan 12 10:16:13 2018 +0100
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jan 12 07:37:40 2018 -0800

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/shell/commands.rb | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f7fbc168/hbase-shell/src/main/ruby/shell/commands.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands.rb b/hbase-shell/src/main/ruby/shell/commands.rb
index f17a7f6..1b8de9e 100644
--- a/hbase-shell/src/main/ruby/shell/commands.rb
+++ b/hbase-shell/src/main/ruby/shell/commands.rb
@@ -117,7 +117,8 @@ module Shell
           raise "Unknown region #{args.first}!"
         end
         if cause.is_a?(org.apache.hadoop.hbase.NamespaceNotFoundException)
-          raise "Unknown namespace #{args.first.split(':')[0]}!"
+          s = /.*NamespaceNotFoundException: (?<namespace>[^\n]+).*/.match(cause.message)
+          raise "Unknown namespace #{s['namespace']}!"
         end
         if cause.is_a?(org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException)
           raise "Unknown snapshot #{args.first}!"


[33/48] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly

Posted by zh...@apache.org.
HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: e504f1de32a4eb411b9a8aea51bb3f5bd1275ab7
Parents: d798b81
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 27 22:03:51 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |   9 +-
 .../hbase/replication/ReplicationQueues.java    | 160 -------
 .../replication/ReplicationQueuesArguments.java |  70 ---
 .../replication/ReplicationQueuesZKImpl.java    | 407 -----------------
 .../hbase/replication/ReplicationTableBase.java | 442 -------------------
 .../replication/ReplicationTrackerZKImpl.java   |  21 +-
 .../replication/ZKReplicationQueueStorage.java  |  22 +
 .../replication/TestReplicationStateBasic.java  | 131 +++---
 .../replication/TestReplicationStateZKImpl.java |  41 +-
 .../regionserver/DumpReplicationQueues.java     |  15 +-
 .../RecoveredReplicationSource.java             |  17 +-
 .../RecoveredReplicationSourceShipper.java      |  22 +-
 .../replication/regionserver/Replication.java   |  41 +-
 .../regionserver/ReplicationSource.java         |  23 +-
 .../ReplicationSourceInterface.java             |  11 +-
 .../regionserver/ReplicationSourceManager.java  | 261 ++++++-----
 .../regionserver/ReplicationSyncUp.java         |  29 +-
 .../hbase/master/cleaner/TestLogsCleaner.java   |  12 +-
 .../cleaner/TestReplicationHFileCleaner.java    |  26 +-
 .../cleaner/TestReplicationZKNodeCleaner.java   |  22 +-
 .../replication/ReplicationSourceDummy.java     |   6 +-
 .../replication/TestReplicationSyncUpTool.java  |   6 +-
 .../TestReplicationSourceManager.java           | 104 ++---
 .../TestReplicationSourceManagerZkImpl.java     |  58 +--
 24 files changed, 385 insertions(+), 1571 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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 6c1c213..5e70e57 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
@@ -17,12 +17,11 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import org.apache.commons.lang3.reflect.ConstructorUtils;
-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.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A factory class for instantiating replication objects that deal with replication state.
@@ -30,12 +29,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args)
-      throws Exception {
-    return (ReplicationQueues) ConstructorUtils.invokeConstructor(ReplicationQueuesZKImpl.class,
-      args);
-  }
-
   public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
       Abortable abortable) {
     return getReplicationPeers(zk, conf, null, abortable);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
deleted file mode 100644
index 7f440b1..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ /dev/null
@@ -1,160 +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.replication;
-
-import java.util.List;
-import java.util.SortedSet;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * This provides an interface for maintaining a region server's replication queues. These queues
- * keep track of the WALs and HFile references (if hbase.replication.bulkload.enabled is enabled)
- * that still need to be replicated to remote clusters.
- */
-@InterfaceAudience.Private
-public interface ReplicationQueues {
-
-  /**
-   * Initialize the region server replication queue interface.
-   * @param serverName The server name of the region server that owns the replication queues this
-   *          interface manages.
-   */
-  void init(String serverName) throws ReplicationException;
-
-  /**
-   * Remove a replication queue.
-   * @param queueId a String that identifies the queue.
-   */
-  void removeQueue(String queueId);
-
-  /**
-   * Add a new WAL file to the given queue. If the queue does not exist it is created.
-   * @param queueId a String that identifies the queue.
-   * @param filename name of the WAL
-   */
-  void addLog(String queueId, String filename) throws ReplicationException;
-
-  /**
-   * Remove an WAL file from the given queue.
-   * @param queueId a String that identifies the queue.
-   * @param filename name of the WAL
-   */
-  void removeLog(String queueId, String filename);
-
-  /**
-   * Set the current position for a specific WAL in a given queue.
-   * @param queueId a String that identifies the queue
-   * @param filename name of the WAL
-   * @param position the current position in the file
-   */
-  void setLogPosition(String queueId, String filename, long position);
-
-  /**
-   * Get the current position for a specific WAL in a given queue.
-   * @param queueId a String that identifies the queue
-   * @param filename name of the WAL
-   * @return the current position in the file
-   */
-  long getLogPosition(String queueId, String filename) throws ReplicationException;
-
-  /**
-   * Remove all replication queues for this region server.
-   */
-  void removeAllQueues();
-
-  /**
-   * Get a list of all WALs in the given queue.
-   * @param queueId a String that identifies the queue
-   * @return a list of WALs, null if no such queue exists for this server
-   */
-  List<String> getLogsInQueue(String queueId);
-
-  /**
-   * Get a list of all queues for this region server.
-   * @return a list of queueIds, an empty list if this region server is dead and has no outstanding queues
-   */
-  List<String> getAllQueues();
-
-  /**
-   * Get queueIds from a dead region server, whose queues has not been claimed by other region
-   * servers.
-   * @return empty if the queue exists but no children, null if the queue does not exist.
-  */
-  List<String> getUnClaimedQueueIds(String regionserver);
-
-  /**
-   * Take ownership for the queue identified by queueId and belongs to a dead region server.
-   * @param regionserver the id of the dead region server
-   * @param queueId the id of the queue
-   * @return the new PeerId and A SortedSet of WALs in its queue, and null if no unclaimed queue.
-   */
-  Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId);
-
-  /**
-   * Remove the znode of region server if the queue is empty.
-   * @param regionserver
-   */
-  void removeReplicatorIfQueueIsEmpty(String regionserver);
-
-  /**
-   * Get a list of all region servers that have outstanding replication queues. These servers could
-   * be alive, dead or from a previous run of the cluster.
-   * @return a list of server names
-   */
-  List<String> getListOfReplicators();
-
-  /**
-   * Checks if the provided znode is the same as this region server's
-   * @param regionserver the id of the region server
-   * @return if this is this rs's znode
-   */
-  boolean isThisOurRegionServer(String regionserver);
-
-  /**
-   * Add a peer to hfile reference queue if peer does not exist.
-   * @param peerId peer cluster id to be added
-   * @throws ReplicationException if fails to add a peer id to hfile reference queue
-   */
-  void addPeerToHFileRefs(String peerId) throws ReplicationException;
-
-  /**
-   * Remove a peer from hfile reference queue.
-   * @param peerId peer cluster id to be removed
-   */
-  void removePeerFromHFileRefs(String peerId);
-
-  /**
-   * Add new hfile references to the queue.
-   * @param peerId peer cluster id to which the hfiles need to be replicated
-   * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which
-   *          will be added in the queue }
-   * @throws ReplicationException if fails to add a hfile reference
-   */
-  void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs) throws ReplicationException;
-
-  /**
-   * Remove hfile references from the queue.
-   * @param peerId peer cluster id from which this hfile references needs to be removed
-   * @param files list of hfile references to be removed
-   */
-  void removeHFileRefs(String peerId, List<String> files);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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
deleted file mode 100644
index c2a5df3..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
+++ /dev/null
@@ -1,70 +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.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;
-
-/**
- * Wrapper around common arguments used to construct ReplicationQueues. Used to construct various
- * ReplicationQueues Implementations with different constructor arguments by reflection.
- */
-@InterfaceAudience.Private
-public class ReplicationQueuesArguments {
-
-  private ZKWatcher zk;
-  private Configuration conf;
-  private Abortable abort;
-
-  public ReplicationQueuesArguments(Configuration conf, Abortable abort) {
-    this.conf = conf;
-    this.abort = abort;
-  }
-
-  public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZKWatcher zk) {
-    this(conf, abort);
-    setZk(zk);
-  }
-
-  public ZKWatcher getZk() {
-    return zk;
-  }
-
-  public void setZk(ZKWatcher zk) {
-    this.zk = zk;
-  }
-
-  public Configuration getConf() {
-    return conf;
-  }
-
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  public Abortable getAbortable() {
-    return abort;
-  }
-
-  public void setAbortable(Abortable abort) {
-    this.abort = abort;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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
deleted file mode 100644
index 7551cb7..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ /dev/null
@@ -1,407 +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.replication;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-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.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.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides an implementation of the
- * interface using ZooKeeper. The
- * base znode that this class works at is the myQueuesZnode. The myQueuesZnode contains a list of
- * all outstanding WAL files on this region server that need to be replicated. The myQueuesZnode is
- * the regionserver name (a concatenation of the region server’s hostname, client port and start
- * code). For example:
- *
- * /hbase/replication/rs/hostname.example.org,6020,1234
- *
- * Within this znode, the region server maintains a set of WAL replication queues. These queues are
- * represented by child znodes named using there give queue id. For example:
- *
- * /hbase/replication/rs/hostname.example.org,6020,1234/1
- * /hbase/replication/rs/hostname.example.org,6020,1234/2
- *
- * Each queue has one child znode for every WAL that still needs to be replicated. The value of
- * these WAL child znodes is the latest position that has been replicated. This position is updated
- * every time a WAL entry is replicated. For example:
- *
- * /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254]
- */
-@InterfaceAudience.Private
-public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements ReplicationQueues {
-
-  /** Znode containing all replication queues for this region server. */
-  private String myQueuesZnode;
-
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class);
-
-  public ReplicationQueuesZKImpl(ReplicationQueuesArguments args) {
-    this(args.getZk(), args.getConf(), args.getAbortable());
-  }
-
-  public ReplicationQueuesZKImpl(final ZKWatcher zk, Configuration conf,
-                                 Abortable abortable) {
-    super(zk, conf, abortable);
-  }
-
-  @Override
-  public void init(String serverName) throws ReplicationException {
-    this.myQueuesZnode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) {
-        ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Could not initialize replication queues.", e);
-    }
-    if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
-      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) {
-      try {
-        if (ZKUtil.checkExists(this.zookeeper, this.hfileRefsZNode) < 0) {
-          ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
-        }
-      } catch (KeeperException e) {
-        throw new ReplicationException("Could not initialize hfile references replication queue.",
-            e);
-      }
-    }
-  }
-
-  @Override
-  public void removeQueue(String queueId) {
-    try {
-      ZKUtil.deleteNodeRecursively(this.zookeeper,
-        ZNodePaths.joinZNode(this.myQueuesZnode, queueId));
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e);
-    }
-  }
-
-  @Override
-  public void addLog(String queueId, String filename) throws ReplicationException {
-    String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-    znode = ZNodePaths.joinZNode(znode, filename);
-    try {
-      ZKUtil.createWithParents(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      throw new ReplicationException(
-          "Could not add log because znode could not be created. queueId=" + queueId
-              + ", filename=" + filename);
-    }
-  }
-
-  @Override
-  public void removeLog(String queueId, String filename) {
-    try {
-      String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-      znode = ZNodePaths.joinZNode(znode, filename);
-      ZKUtil.deleteNode(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename="
-          + filename + ")", e);
-    }
-  }
-
-  @Override
-  public void setLogPosition(String queueId, String filename, long position) {
-    try {
-      String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-      znode = ZNodePaths.joinZNode(znode, filename);
-      // Why serialize String of Long and not Long as bytes?
-      ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to write replication wal position (filename=" + filename
-          + ", position=" + position + ")", e);
-    }
-  }
-
-  @Override
-  public long getLogPosition(String queueId, String filename) throws ReplicationException {
-    String clusterZnode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-    String znode = ZNodePaths.joinZNode(clusterZnode, filename);
-    byte[] bytes = null;
-    try {
-      bytes = ZKUtil.getData(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      throw new ReplicationException("Internal Error: could not get position in log for queueId="
-          + queueId + ", filename=" + filename, e);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      return 0;
-    }
-    try {
-      return ZKUtil.parseWALPositionFrom(bytes);
-    } catch (DeserializationException de) {
-      LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename
-          + " znode content, continuing.");
-    }
-    // if we can not parse the position, start at the beginning of the wal file
-    // again
-    return 0;
-  }
-
-  @Override
-  public boolean isThisOurRegionServer(String regionserver) {
-    return ZNodePaths.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode);
-  }
-
-  @Override
-  public List<String> getUnClaimedQueueIds(String regionserver) {
-    if (isThisOurRegionServer(regionserver)) {
-      return null;
-    }
-    String rsZnodePath = ZNodePaths.joinZNode(this.queuesZNode, regionserver);
-    List<String> queues = null;
-    try {
-      queues = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZnodePath);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to getUnClaimedQueueIds for RS" + regionserver, e);
-    }
-    return queues;
-  }
-
-  @Override
-  public Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId) {
-    LOG.info("Atomically moving " + regionserver + "/" + queueId + "'s WALs to my queue");
-    return moveQueueUsingMulti(regionserver, queueId);
-  }
-
-  @Override
-  public void removeReplicatorIfQueueIsEmpty(String regionserver) {
-    String rsPath = ZNodePaths.joinZNode(this.queuesZNode, regionserver);
-    try {
-      List<String> list = ZKUtil.listChildrenNoWatch(this.zookeeper, rsPath);
-      if (list != null && list.isEmpty()){
-        ZKUtil.deleteNode(this.zookeeper, rsPath);
-      }
-    } catch (KeeperException e) {
-      LOG.warn("Got error while removing replicator", e);
-    }
-  }
-
-  @Override
-  public void removeAllQueues() {
-    try {
-      ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode);
-    } catch (KeeperException e) {
-      // if the znode is already expired, don't bother going further
-      if (e instanceof KeeperException.SessionExpiredException) {
-        return;
-      }
-      this.abortable.abort("Failed to delete replication queues for region server: "
-          + this.myQueuesZnode, e);
-    }
-  }
-
-  @Override
-  public List<String> getLogsInQueue(String queueId) {
-    String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e);
-    }
-    return result;
-  }
-
-  @Override
-  public List<String> getAllQueues() {
-    List<String> listOfQueues = null;
-    try {
-      listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get a list of queues for region server: "
-          + this.myQueuesZnode, e);
-    }
-    return listOfQueues == null ? new ArrayList<>() : listOfQueues;
-  }
-
-  /**
-   * It "atomically" copies one peer's wals queue from another dead region server and returns them
-   * all sorted. The new peer id is equal to the old peer id appended with the dead server's znode.
-   * @param znode pertaining to the region server to copy the queues from
-   * @peerId peerId pertaining to the queue need to be copied
-   */
-  private Pair<String, SortedSet<String>> moveQueueUsingMulti(String znode, String peerId) {
-    try {
-      // hbase/replication/rs/deadrs
-      String deadRSZnodePath = ZNodePaths.joinZNode(this.queuesZNode, znode);
-      List<ZKUtilOp> listOfOps = new ArrayList<>();
-      ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
-
-      String newPeerId = peerId + "-" + znode;
-      String newPeerZnode = ZNodePaths.joinZNode(this.myQueuesZnode, newPeerId);
-      // check the logs queue for the old peer cluster
-      String oldClusterZnode = ZNodePaths.joinZNode(deadRSZnodePath, peerId);
-      List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
-
-      if (!peerExists(replicationQueueInfo.getPeerId())) {
-        LOG.warn("Peer " + replicationQueueInfo.getPeerId() +
-                " didn't exist, will move its queue to avoid the failure of multi op");
-        for (String wal : wals) {
-          String oldWalZnode = ZNodePaths.joinZNode(oldClusterZnode, wal);
-          listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
-        }
-        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
-        ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
-        return null;
-      }
-
-      SortedSet<String> logQueue = new TreeSet<>();
-      if (wals == null || wals.isEmpty()) {
-        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
-      } else {
-        // create the new cluster znode
-        ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
-        listOfOps.add(op);
-        // get the offset of the logs and set it to new znodes
-        for (String wal : wals) {
-          String oldWalZnode = ZNodePaths.joinZNode(oldClusterZnode, wal);
-          byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode);
-          LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
-          String newLogZnode = ZNodePaths.joinZNode(newPeerZnode, wal);
-          listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
-          listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
-          logQueue.add(wal);
-        }
-        // add delete op for peer
-        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
-
-        if (LOG.isTraceEnabled())
-          LOG.trace(" The multi list size is: " + listOfOps.size());
-      }
-      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
-
-      LOG.info("Atomically moved " + znode + "/" + peerId + "'s WALs to my queue");
-      return new Pair<>(newPeerId, logQueue);
-    } catch (KeeperException e) {
-      // Multi call failed; it looks like some other regionserver took away the logs.
-      LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
-    } catch (InterruptedException e) {
-      LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
-      Thread.currentThread().interrupt();
-    }
-    return null;
-  }
-
-  @Override
-  public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
-      throws ReplicationException {
-    String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    boolean debugEnabled = LOG.isDebugEnabled();
-    if (debugEnabled) {
-      LOG.debug("Adding hfile references " + pairs + " in queue " + peerZnode);
-    }
-
-    int size = pairs.size();
-    List<ZKUtilOp> listOfOps = new ArrayList<>(size);
-
-    for (int i = 0; i < size; i++) {
-      listOfOps.add(ZKUtilOp.createAndFailSilent(
-        ZNodePaths.joinZNode(peerZnode, pairs.get(i).getSecond().getName()),
-        HConstants.EMPTY_BYTE_ARRAY));
-    }
-    if (debugEnabled) {
-      LOG.debug(" The multi list size for adding hfile references in zk for node " + peerZnode
-          + " is " + listOfOps.size());
-    }
-    try {
-      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
-    } catch (KeeperException e) {
-      throw new ReplicationException("Failed to create hfile reference znode=" + e.getPath(), e);
-    }
-  }
-
-  @Override
-  public void removeHFileRefs(String peerId, List<String> files) {
-    String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    boolean debugEnabled = LOG.isDebugEnabled();
-    if (debugEnabled) {
-      LOG.debug("Removing hfile references " + files + " from queue " + peerZnode);
-    }
-
-    int size = files.size();
-    List<ZKUtilOp> listOfOps = new ArrayList<>(size);
-
-    for (int i = 0; i < size; i++) {
-      listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZNodePaths.joinZNode(peerZnode, files.get(i))));
-    }
-    if (debugEnabled) {
-      LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode
-          + " is " + listOfOps.size());
-    }
-    try {
-      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
-    } catch (KeeperException e) {
-      LOG.error("Failed to remove hfile reference znode=" + e.getPath(), e);
-    }
-  }
-
-  @Override
-  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
-    String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
-        LOG.info("Adding peer " + peerId + " to hfile reference queue.");
-        ZKUtil.createWithParents(this.zookeeper, peerZnode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
-          e);
-    }
-  }
-
-  @Override
-  public void removePeerFromHFileRefs(String peerId) {
-    final String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Peer " + peerZnode + " not found in hfile reference queue.");
-        }
-        return;
-      } else {
-        LOG.info("Removing peer " + peerZnode + " from hfile reference queue.");
-        ZKUtil.deleteNodeRecursively(this.zookeeper, peerZnode);
-      }
-    } catch (KeeperException e) {
-      LOG.error("Ignoring the exception to remove peer " + peerId + " from hfile reference queue.",
-        e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java
deleted file mode 100644
index 0d8427c..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java
+++ /dev/null
@@ -1,442 +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.replication;
-
-import org.apache.hadoop.hbase.CompareOperator;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.TableExistsException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-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.regionserver.BloomType;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.RetryCounterFactory;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executor;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-/*
- * Abstract class that provides an interface to the Replication Table. Which is currently
- * being used for WAL offset tracking.
- * The basic schema of this table will store each individual queue as a
- * seperate row. The row key will be a unique identifier of the creating server's name and the
- * queueId. Each queue must have the following two columns:
- *  COL_QUEUE_OWNER: tracks which server is currently responsible for tracking the queue
- *  COL_QUEUE_OWNER_HISTORY: a "|" delimited list of the previous server's that have owned this
- *    queue. The most recent previous owner is the leftmost entry.
- * They will also have columns mapping [WAL filename : offset]
- * The most flexible method of interacting with the Replication Table is by calling
- * getOrBlockOnReplicationTable() which will return a new copy of the Replication Table. It is up
- * to the caller to close the returned table.
- */
-@InterfaceAudience.Private
-abstract class ReplicationTableBase {
-
-  /** Name of the HBase Table used for tracking replication*/
-  public static final TableName REPLICATION_TABLE_NAME =
-    TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication");
-
-  // Column family and column names for Queues in the Replication Table
-  public static final byte[] CF_QUEUE = Bytes.toBytes("q");
-  public static final byte[] COL_QUEUE_OWNER = Bytes.toBytes("o");
-  public static final byte[] COL_QUEUE_OWNER_HISTORY = Bytes.toBytes("h");
-
-  // Column Descriptor for the Replication Table
-  private static final HColumnDescriptor REPLICATION_COL_DESCRIPTOR =
-    new HColumnDescriptor(CF_QUEUE).setMaxVersions(1)
-      .setInMemory(true)
-      .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-        // TODO: Figure out which bloom filter to use
-      .setBloomFilterType(BloomType.NONE);
-
-  // The value used to delimit the queueId and server name inside of a queue's row key. Currently a
-  // hyphen, because it is guaranteed that queueId (which is a cluster id) cannot contain hyphens.
-  // See HBASE-11394.
-  public static final String ROW_KEY_DELIMITER = "-";
-
-  // The value used to delimit server names in the queue history list
-  public static final String QUEUE_HISTORY_DELIMITER = "|";
-
-  /*
-  * Make sure that HBase table operations for replication have a high number of retries. This is
-  * because the server is aborted if any HBase table operation fails. Each RPC will be attempted
-  * 3600 times before exiting. This provides each operation with 2 hours of retries
-  * before the server is aborted.
-  */
-  private static final int CLIENT_RETRIES = 3600;
-  private static final int RPC_TIMEOUT = 2000;
-  private static final int OPERATION_TIMEOUT = CLIENT_RETRIES * RPC_TIMEOUT;
-
-  // We only need a single thread to initialize the Replication Table
-  private static final int NUM_INITIALIZE_WORKERS = 1;
-
-  protected final Configuration conf;
-  protected final Abortable abortable;
-  private final Connection connection;
-  private final Executor executor;
-  private volatile CountDownLatch replicationTableInitialized;
-
-  public ReplicationTableBase(Configuration conf, Abortable abort) throws IOException {
-    this.conf = new Configuration(conf);
-    this.abortable = abort;
-    decorateConf();
-    this.connection = ConnectionFactory.createConnection(this.conf);
-    this.executor = setUpExecutor();
-    this.replicationTableInitialized = new CountDownLatch(1);
-    createReplicationTableInBackground();
-  }
-
-  /**
-   * Modify the connection's config so that operations run on the Replication Table have longer and
-   * a larger number of retries
-   */
-  private void decorateConf() {
-    this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, CLIENT_RETRIES);
-  }
-
-  /**
-   * Sets up the thread pool executor used to build the Replication Table in the background
-   * @return the configured executor
-   */
-  private Executor setUpExecutor() {
-    ThreadPoolExecutor tempExecutor = new ThreadPoolExecutor(NUM_INITIALIZE_WORKERS,
-        NUM_INITIALIZE_WORKERS, 100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>());
-    ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
-    tfb.setNameFormat("ReplicationTableExecutor-%d");
-    tfb.setDaemon(true);
-    tempExecutor.setThreadFactory(tfb.build());
-    return tempExecutor;
-  }
-
-  /**
-   * Get whether the Replication Table has been successfully initialized yet
-   * @return whether the Replication Table is initialized
-   */
-  public boolean getInitializationStatus() {
-    return replicationTableInitialized.getCount() == 0;
-  }
-
-  /**
-   * Increases the RPC and operations timeouts for the Replication Table
-   */
-  private Table setReplicationTableTimeOuts(Table replicationTable) {
-    replicationTable.setRpcTimeout(RPC_TIMEOUT);
-    replicationTable.setOperationTimeout(OPERATION_TIMEOUT);
-    return replicationTable;
-  }
-
-  /**
-   * Build the row key for the given queueId. This will uniquely identify it from all other queues
-   * in the cluster.
-   * @param serverName The owner of the queue
-   * @param queueId String identifier of the queue
-   * @return String representation of the queue's row key
-   */
-  protected String buildQueueRowKey(String serverName, String queueId) {
-    return queueId + ROW_KEY_DELIMITER + serverName;
-  }
-
-  /**
-   * Parse the original queueId from a row key
-   * @param rowKey String representation of a queue's row key
-   * @return the original queueId
-   */
-  protected String getRawQueueIdFromRowKey(String rowKey) {
-    return rowKey.split(ROW_KEY_DELIMITER)[0];
-  }
-
-  /**
-   * Returns a queue's row key given either its raw or reclaimed queueId
-   *
-   * @param queueId queueId of the queue
-   * @return byte representation of the queue's row key
-   */
-  protected byte[] queueIdToRowKey(String serverName, String queueId) {
-    // Cluster id's are guaranteed to have no hyphens, so if the passed in queueId has no hyphen
-    // then this is not a reclaimed queue.
-    if (!queueId.contains(ROW_KEY_DELIMITER)) {
-      return Bytes.toBytes(buildQueueRowKey(serverName, queueId));
-      // If the queueId contained some hyphen it was reclaimed. In this case, the queueId is the
-      // queue's row key
-    } else {
-      return Bytes.toBytes(queueId);
-    }
-  }
-
-  /**
-   * Creates a "|" delimited record of the queue's past region server owners.
-   *
-   * @param originalHistory the queue's original owner history
-   * @param oldServer the name of the server that used to own the queue
-   * @return the queue's new owner history
-   */
-  protected String buildClaimedQueueHistory(String originalHistory, String oldServer) {
-    return oldServer + QUEUE_HISTORY_DELIMITER + originalHistory;
-  }
-
-  /**
-   * Get a list of all region servers that have outstanding replication queues. These servers could
-   * be alive, dead or from a previous run of the cluster.
-   * @return a list of server names
-   */
-  protected List<String> getListOfReplicators() {
-    // scan all of the queues and return a list of all unique OWNER values
-    Set<String> peerServers = new HashSet<>();
-    ResultScanner allQueuesInCluster = null;
-    try (Table replicationTable = getOrBlockOnReplicationTable()){
-      Scan scan = new Scan();
-      scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
-      allQueuesInCluster = replicationTable.getScanner(scan);
-      for (Result queue : allQueuesInCluster) {
-        peerServers.add(Bytes.toString(queue.getValue(CF_QUEUE, COL_QUEUE_OWNER)));
-      }
-    } catch (IOException e) {
-      String errMsg = "Failed getting list of replicators";
-      abortable.abort(errMsg, e);
-    } finally {
-      if (allQueuesInCluster != null) {
-        allQueuesInCluster.close();
-      }
-    }
-    return new ArrayList<>(peerServers);
-  }
-
-  protected List<String> getAllQueues(String serverName) {
-    List<String> allQueues = new ArrayList<>();
-    ResultScanner queueScanner = null;
-    try {
-      queueScanner = getQueuesBelongingToServer(serverName);
-      for (Result queue : queueScanner) {
-        String rowKey =  Bytes.toString(queue.getRow());
-        // If the queue does not have a Owner History, then we must be its original owner. So we
-        // want to return its queueId in raw form
-        if (Bytes.toString(queue.getValue(CF_QUEUE, COL_QUEUE_OWNER_HISTORY)).length() == 0) {
-          allQueues.add(getRawQueueIdFromRowKey(rowKey));
-        } else {
-          allQueues.add(rowKey);
-        }
-      }
-      return allQueues;
-    } catch (IOException e) {
-      String errMsg = "Failed getting list of all replication queues for serverName=" + serverName;
-      abortable.abort(errMsg, e);
-      return null;
-    } finally {
-      if (queueScanner != null) {
-        queueScanner.close();
-      }
-    }
-  }
-
-  protected List<String> getLogsInQueue(String serverName, String queueId) {
-    String rowKey = queueId;
-    if (!queueId.contains(ROW_KEY_DELIMITER)) {
-      rowKey = buildQueueRowKey(serverName, queueId);
-    }
-    return getLogsInQueue(Bytes.toBytes(rowKey));
-  }
-
-  protected List<String> getLogsInQueue(byte[] rowKey) {
-    String errMsg = "Failed getting logs in queue queueId=" + Bytes.toString(rowKey);
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      Get getQueue = new Get(rowKey);
-      Result queue = replicationTable.get(getQueue);
-      if (queue == null || queue.isEmpty()) {
-        abortable.abort(errMsg, new ReplicationException(errMsg));
-        return null;
-      }
-      return readWALsFromResult(queue);
-    } catch (IOException e) {
-      abortable.abort(errMsg, e);
-      return null;
-    }
-  }
-
-  /**
-   * Read all of the WAL's from a queue into a list
-   *
-   * @param queue HBase query result containing the queue
-   * @return a list of all the WAL filenames
-   */
-  protected List<String> readWALsFromResult(Result queue) {
-    List<String> wals = new ArrayList<>();
-    Map<byte[], byte[]> familyMap = queue.getFamilyMap(CF_QUEUE);
-    for (byte[] cQualifier : familyMap.keySet()) {
-      // Ignore the meta data fields of the queue
-      if (Arrays.equals(cQualifier, COL_QUEUE_OWNER) || Arrays.equals(cQualifier,
-          COL_QUEUE_OWNER_HISTORY)) {
-        continue;
-      }
-      wals.add(Bytes.toString(cQualifier));
-    }
-    return wals;
-  }
-
-  /**
-   * Get the queue id's and meta data (Owner and History) for the queues belonging to the named
-   * server
-   *
-   * @param server name of the server
-   * @return a ResultScanner over the QueueIds belonging to the server
-   * @throws IOException
-   */
-  protected ResultScanner getQueuesBelongingToServer(String server) throws IOException {
-    Scan scan = new Scan();
-    SingleColumnValueFilter filterMyQueues = new SingleColumnValueFilter(CF_QUEUE, COL_QUEUE_OWNER,
-    CompareOperator.EQUAL, Bytes.toBytes(server));
-    scan.setFilter(filterMyQueues);
-    scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
-    scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY);
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      ResultScanner results = replicationTable.getScanner(scan);
-      return results;
-    }
-  }
-
-  /**
-   * Attempts to acquire the Replication Table. This operation will block until it is assigned by
-   * the CreateReplicationWorker thread. It is up to the caller of this method to close the
-   * returned Table
-   * @return the Replication Table when it is created
-   * @throws IOException
-   */
-  protected Table getOrBlockOnReplicationTable() throws IOException {
-    // Sleep until the Replication Table becomes available
-    try {
-      replicationTableInitialized.await();
-    } catch (InterruptedException e) {
-      String errMsg = "Unable to acquire the Replication Table due to InterruptedException: " +
-          e.getMessage();
-      throw new InterruptedIOException(errMsg);
-    }
-    return getAndSetUpReplicationTable();
-  }
-
-  /**
-   * Creates a new copy of the Replication Table and sets up the proper Table time outs for it
-   *
-   * @return the Replication Table
-   * @throws IOException
-   */
-  private Table getAndSetUpReplicationTable() throws IOException {
-    Table replicationTable = connection.getTable(REPLICATION_TABLE_NAME);
-    setReplicationTableTimeOuts(replicationTable);
-    return replicationTable;
-  }
-
-  /**
-   * Builds the Replication Table in a background thread. Any method accessing the Replication Table
-   * should do so through getOrBlockOnReplicationTable()
-   *
-   * @return the Replication Table
-   * @throws IOException if the Replication Table takes too long to build
-   */
-  private void createReplicationTableInBackground() throws IOException {
-    executor.execute(new CreateReplicationTableWorker());
-  }
-
-  /**
-   * Attempts to build the Replication Table. Will continue blocking until we have a valid
-   * Table for the Replication Table.
-   */
-  private class CreateReplicationTableWorker implements Runnable {
-
-    private Admin admin;
-
-    @Override
-    public void run() {
-      try {
-        admin = connection.getAdmin();
-        if (!replicationTableExists()) {
-          createReplicationTable();
-        }
-        int maxRetries = conf.getInt("hbase.replication.queues.createtable.retries.number",
-            CLIENT_RETRIES);
-        RetryCounterFactory counterFactory = new RetryCounterFactory(maxRetries, RPC_TIMEOUT);
-        RetryCounter retryCounter = counterFactory.create();
-        while (!replicationTableExists()) {
-          retryCounter.sleepUntilNextRetry();
-          if (!retryCounter.shouldRetry()) {
-            throw new IOException("Unable to acquire the Replication Table");
-          }
-        }
-        replicationTableInitialized.countDown();
-      } catch (IOException | InterruptedException e) {
-        abortable.abort("Failed building Replication Table", e);
-      }
-    }
-
-    /**
-     * Create the replication table with the provided HColumnDescriptor REPLICATION_COL_DESCRIPTOR
-     * in TableBasedReplicationQueuesImpl
-     *
-     * @throws IOException
-     */
-    private void createReplicationTable() throws IOException {
-      HTableDescriptor replicationTableDescriptor = new HTableDescriptor(REPLICATION_TABLE_NAME);
-      replicationTableDescriptor.addFamily(REPLICATION_COL_DESCRIPTOR);
-      try {
-        admin.createTable(replicationTableDescriptor);
-      } catch (TableExistsException e) {
-        // In this case we can just continue as normal
-      }
-    }
-
-    /**
-     * Checks whether the Replication Table exists yet
-     *
-     * @return whether the Replication Table exists
-     * @throws IOException
-     */
-    private boolean replicationTableExists() {
-      try {
-        return admin.tableExists(REPLICATION_TABLE_NAME);
-      } catch (IOException e) {
-        return false;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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 2c522f6..5659e4b 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
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -54,6 +53,8 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
     super(zookeeper, conf, abortable);
     this.stopper = stopper;
     this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper));
+    // watch the changes
+    refreshOtherRegionServersList(true);
   }
 
   @Override
@@ -71,7 +72,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    */
   @Override
   public List<String> getListOfRegionServers() {
-    refreshOtherRegionServersList();
+    refreshOtherRegionServersList(false);
 
     List<String> list = null;
     synchronized (otherRegionServers) {
@@ -137,7 +138,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
       if (!path.startsWith(this.watcher.znodePaths.rsZNode)) {
         return false;
       }
-      return refreshOtherRegionServersList();
+      return refreshOtherRegionServersList(true);
     }
   }
 
@@ -157,8 +158,8 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    * @return true if the local list of the other region servers was updated with the ZK data (even
    *         if it was empty), false if the data was missing in ZK
    */
-  private boolean refreshOtherRegionServersList() {
-    List<String> newRsList = getRegisteredRegionServers();
+  private boolean refreshOtherRegionServersList(boolean watch) {
+    List<String> newRsList = getRegisteredRegionServers(watch);
     if (newRsList == null) {
       return false;
     } else {
@@ -174,10 +175,14 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    * Get a list of all the other region servers in this cluster and set a watch
    * @return a list of server nanes
    */
-  private List<String> getRegisteredRegionServers() {
+  private List<String> getRegisteredRegionServers(boolean watch) {
     List<String> result = null;
     try {
-      result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      if (watch) {
+        result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      } else {
+        result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      }
     } catch (KeeperException e) {
       this.abortable.abort("Get list of registered region servers", e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 0275d52..41f50d8 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -54,6 +54,28 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe
 
 /**
  * ZK based replication queue storage.
+ * <p>
+ * The base znode for each regionserver is the regionserver name. For example:
+ *
+ * <pre>
+ * /hbase/replication/rs/hostname.example.org,6020,1234
+ * </pre>
+ *
+ * Within this znode, the region server maintains a set of WAL replication queues. These queues are
+ * represented by child znodes named using there give queue id. For example:
+ *
+ * <pre>
+ * /hbase/replication/rs/hostname.example.org,6020,1234/1
+ * /hbase/replication/rs/hostname.example.org,6020,1234/2
+ * </pre>
+ *
+ * Each queue has one child znode for every WAL that still needs to be replicated. The value of
+ * these WAL child znodes is the latest position that has been replicated. This position is updated
+ * every time a WAL entry is replicated. For example:
+ *
+ * <pre>
+ * /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254]
+ * </pre>
  */
 @InterfaceAudience.Private
 class ZKReplicationQueueStorage extends ZKReplicationStorageBase

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 8905d43..4afda5d 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -42,9 +42,8 @@ import org.slf4j.LoggerFactory;
  */
 public abstract class TestReplicationStateBasic {
 
-  protected ReplicationQueues rq1;
-  protected ReplicationQueues rq2;
-  protected ReplicationQueues rq3;
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class);
+
   protected ReplicationQueueStorage rqs;
   protected ServerName server1 = ServerName.valueOf("hostname1.example.org", 1234, 12345);
   protected ServerName server2 = ServerName.valueOf("hostname2.example.org", 1234, 12345);
@@ -63,8 +62,6 @@ public abstract class TestReplicationStateBasic {
   protected static final int ZK_MAX_COUNT = 300;
   protected static final int ZK_SLEEP_INTERVAL = 100; // millis
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class);
-
   @Test
   public void testReplicationQueueStorage() throws ReplicationException {
     // Test methods with empty state
@@ -76,15 +73,13 @@ public abstract class TestReplicationStateBasic {
      * Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each --
      * server2: zero queues
      */
-    rq1.init(server1.getServerName());
-    rq2.init(server2.getServerName());
-    rq1.addLog("qId1", "trash");
-    rq1.removeLog("qId1", "trash");
-    rq1.addLog("qId2", "filename1");
-    rq1.addLog("qId3", "filename2");
-    rq1.addLog("qId3", "filename3");
-    rq2.addLog("trash", "trash");
-    rq2.removeQueue("trash");
+    rqs.addWAL(server1, "qId1", "trash");
+    rqs.removeWAL(server1, "qId1", "trash");
+    rqs.addWAL(server1,"qId2", "filename1");
+    rqs.addWAL(server1,"qId3", "filename2");
+    rqs.addWAL(server1,"qId3", "filename3");
+    rqs.addWAL(server2,"trash", "trash");
+    rqs.removeQueue(server2,"trash");
 
     List<ServerName> reps = rqs.getListOfReplicators();
     assertEquals(2, reps.size());
@@ -105,62 +100,55 @@ public abstract class TestReplicationStateBasic {
     assertTrue(list.contains("qId3"));
   }
 
+  private void removeAllQueues(ServerName serverName) throws ReplicationException {
+    for (String queue: rqs.getAllQueues(serverName)) {
+      rqs.removeQueue(serverName, queue);
+    }
+  }
   @Test
   public void testReplicationQueues() throws ReplicationException {
-    rq1.init(server1.getServerName());
-    rq2.init(server2.getServerName());
-    rq3.init(server3.getServerName());
     // Initialize ReplicationPeer so we can add peers (we don't transfer lone queues)
     rp.init();
 
-    // 3 replicators should exist
-    assertEquals(3, rq1.getListOfReplicators().size());
-    rq1.removeQueue("bogus");
-    rq1.removeLog("bogus", "bogus");
-    rq1.removeAllQueues();
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
-    assertNull(rq1.getLogsInQueue("bogus"));
-    assertNull(rq1.getUnClaimedQueueIds(ServerName.valueOf("bogus", 1234, -1L).toString()));
-
-    rq1.setLogPosition("bogus", "bogus", 5L);
+    rqs.removeQueue(server1, "bogus");
+    rqs.removeWAL(server1, "bogus", "bogus");
+    removeAllQueues(server1);
+    assertEquals(0, rqs.getAllQueues(server1).size());
+    assertEquals(0, rqs.getWALPosition(server1, "bogus", "bogus"));
+    assertTrue(rqs.getWALsInQueue(server1, "bogus").isEmpty());
+    assertTrue(rqs.getAllQueues(ServerName.valueOf("bogus", 1234, 12345)).isEmpty());
 
     populateQueues();
 
-    assertEquals(3, rq1.getListOfReplicators().size());
-    assertEquals(0, rq2.getLogsInQueue("qId1").size());
-    assertEquals(5, rq3.getLogsInQueue("qId5").size());
-    assertEquals(0, rq3.getLogPosition("qId1", "filename0"));
-    rq3.setLogPosition("qId5", "filename4", 354L);
-    assertEquals(354L, rq3.getLogPosition("qId5", "filename4"));
+    assertEquals(3, rqs.getListOfReplicators().size());
+    assertEquals(0, rqs.getWALsInQueue(server2, "qId1").size());
+    assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size());
+    assertEquals(0, rqs.getWALPosition(server3, "qId1", "filename0"));
+    rqs.setWALPosition(server3, "qId5", "filename4", 354L);
+    assertEquals(354L, rqs.getWALPosition(server3, "qId5", "filename4"));
 
-    assertEquals(5, rq3.getLogsInQueue("qId5").size());
-    assertEquals(0, rq2.getLogsInQueue("qId1").size());
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(1, rq2.getAllQueues().size());
-    assertEquals(5, rq3.getAllQueues().size());
+    assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size());
+    assertEquals(0, rqs.getWALsInQueue(server2, "qId1").size());
+    assertEquals(0, rqs.getAllQueues(server1).size());
+    assertEquals(1, rqs.getAllQueues(server2).size());
+    assertEquals(5, rqs.getAllQueues(server3).size());
 
-    assertEquals(0, rq3.getUnClaimedQueueIds(server1.getServerName()).size());
-    rq3.removeReplicatorIfQueueIsEmpty(server1.getServerName());
-    assertEquals(2, rq3.getListOfReplicators().size());
+    assertEquals(0, rqs.getAllQueues(server1).size());
+    rqs.removeReplicatorIfQueueIsEmpty(server1);
+    assertEquals(2, rqs.getListOfReplicators().size());
 
-    List<String> queues = rq2.getUnClaimedQueueIds(server3.getServerName());
+    List<String> queues = rqs.getAllQueues(server3);
     assertEquals(5, queues.size());
     for (String queue : queues) {
-      rq2.claimQueue(server3.getServerName(), queue);
+      rqs.claimQueue(server3, queue, server2);
     }
-    rq2.removeReplicatorIfQueueIsEmpty(server3.getServerName());
-    assertEquals(1, rq2.getListOfReplicators().size());
-
-    // Try to claim our own queues
-    assertNull(rq2.getUnClaimedQueueIds(server2.getServerName()));
-    rq2.removeReplicatorIfQueueIsEmpty(server2.getServerName());
-
-    assertEquals(6, rq2.getAllQueues().size());
+    rqs.removeReplicatorIfQueueIsEmpty(server3);
+    assertEquals(1, rqs.getListOfReplicators().size());
 
-    rq2.removeAllQueues();
-
-    assertEquals(0, rq2.getListOfReplicators().size());
+    assertEquals(6, rqs.getAllQueues(server2).size());
+    removeAllQueues(server2);
+    rqs.removeReplicatorIfQueueIsEmpty(server2);
+    assertEquals(0, rqs.getListOfReplicators().size());
   }
 
   @Test
@@ -197,7 +185,6 @@ public abstract class TestReplicationStateBasic {
   @Test
   public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
     rp.init();
-    rq1.init(server1.getServerName());
 
     List<Pair<Path, Path>> files1 = new ArrayList<>(3);
     files1.add(new Pair<>(null, new Path("file_1")));
@@ -206,8 +193,8 @@ public abstract class TestReplicationStateBasic {
     assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
     assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
     rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rq1.addPeerToHFileRefs(ID_ONE);
-    rq1.addHFileRefs(ID_ONE, files1);
+    rqs.addPeerToHFileRefs(ID_ONE);
+    rqs.addHFileRefs(ID_ONE, files1);
     assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
     assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
     List<String> hfiles2 = new ArrayList<>(files1.size());
@@ -215,43 +202,41 @@ public abstract class TestReplicationStateBasic {
       hfiles2.add(p.getSecond().getName());
     }
     String removedString = hfiles2.remove(0);
-    rq1.removeHFileRefs(ID_ONE, hfiles2);
+    rqs.removeHFileRefs(ID_ONE, hfiles2);
     assertEquals(1, rqs.getReplicableHFiles(ID_ONE).size());
     hfiles2 = new ArrayList<>(1);
     hfiles2.add(removedString);
-    rq1.removeHFileRefs(ID_ONE, hfiles2);
+    rqs.removeHFileRefs(ID_ONE, hfiles2);
     assertEquals(0, rqs.getReplicableHFiles(ID_ONE).size());
     rp.unregisterPeer(ID_ONE);
   }
 
   @Test
   public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
-    rq1.init(server1.getServerName());
-
     rp.init();
     rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rq1.addPeerToHFileRefs(ID_ONE);
+    rqs.addPeerToHFileRefs(ID_ONE);
     rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
-    rq1.addPeerToHFileRefs(ID_TWO);
+    rqs.addPeerToHFileRefs(ID_TWO);
 
     List<Pair<Path, Path>> files1 = new ArrayList<>(3);
     files1.add(new Pair<>(null, new Path("file_1")));
     files1.add(new Pair<>(null, new Path("file_2")));
     files1.add(new Pair<>(null, new Path("file_3")));
-    rq1.addHFileRefs(ID_ONE, files1);
-    rq1.addHFileRefs(ID_TWO, files1);
+    rqs.addHFileRefs(ID_ONE, files1);
+    rqs.addHFileRefs(ID_TWO, files1);
     assertEquals(2, rqs.getAllPeersFromHFileRefsQueue().size());
     assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
     assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
 
     rp.unregisterPeer(ID_ONE);
-    rq1.removePeerFromHFileRefs(ID_ONE);
+    rqs.removePeerFromHFileRefs(ID_ONE);
     assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
     assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
     assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
 
     rp.unregisterPeer(ID_TWO);
-    rq1.removePeerFromHFileRefs(ID_TWO);
+    rqs.removePeerFromHFileRefs(ID_TWO);
     assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
     assertTrue(rqs.getReplicableHFiles(ID_TWO).isEmpty());
   }
@@ -363,15 +348,15 @@ public abstract class TestReplicationStateBasic {
    * 3, 4, 5 log files respectively
    */
   protected void populateQueues() throws ReplicationException {
-    rq1.addLog("trash", "trash");
-    rq1.removeQueue("trash");
+    rqs.addWAL(server1, "trash", "trash");
+    rqs.removeQueue(server1, "trash");
 
-    rq2.addLog("qId1", "trash");
-    rq2.removeLog("qId1", "trash");
+    rqs.addWAL(server2, "qId1", "trash");
+    rqs.removeWAL(server2, "qId1", "trash");
 
     for (int i = 1; i < 6; i++) {
       for (int j = 0; j < i; j++) {
-        rq3.addLog("qId" + i, "filename" + j);
+        rqs.addWAL(server3, "qId" + i, "filename" + j);
       }
       // Add peers for the corresponding queues so they are not orphans
       rp.registerPeer("qId" + i,

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index 5fe7c55..ac869d9 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -17,10 +17,6 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -40,7 +36,6 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -54,7 +49,6 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
   private static HBaseZKTestingUtility utility;
   private static ZKWatcher zkw;
   private static String replicationZNode;
-  private ReplicationQueuesZKImpl rqZK;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -84,23 +78,9 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
   @Before
   public void setUp() {
     zkTimeoutCount = 0;
-    WarnOnlyAbortable abortable = new WarnOnlyAbortable();
-    try {
-      rq1 = ReplicationFactory
-          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
-      rq2 = ReplicationFactory
-          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
-      rq3 = ReplicationFactory
-          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
-      rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
-    } catch (Exception e) {
-      // This should not occur, because getReplicationQueues() only throws for
-      // TableBasedReplicationQueuesImpl
-      fail("ReplicationFactory.getReplicationQueues() threw an IO Exception");
-    }
-    rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
+    rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
+    rp = ReplicationFactory.getReplicationPeers(zkw, conf, new WarnOnlyAbortable());
     OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
-    rqZK = new ReplicationQueuesZKImpl(zkw, conf, abortable);
   }
 
   @After
@@ -113,23 +93,6 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
     utility.shutdownMiniZKCluster();
   }
 
-  @Test
-  public void testIsPeerPath_PathToParentOfPeerNode() {
-    assertFalse(rqZK.isPeerPath(rqZK.peersZNode));
-  }
-
-  @Test
-  public void testIsPeerPath_PathToChildOfPeerNode() {
-    String peerChild = ZNodePaths.joinZNode(ZNodePaths.joinZNode(rqZK.peersZNode, "1"), "child");
-    assertFalse(rqZK.isPeerPath(peerChild));
-  }
-
-  @Test
-  public void testIsPeerPath_ActualPeerPath() {
-    String peerPath = ZNodePaths.joinZNode(rqZK.peersZNode, "1");
-    assertTrue(rqZK.isPeerPath(peerPath));
-  }
-
   private static class WarnOnlyAbortable implements Abortable {
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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 d8f9625..73e600e 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
@@ -49,8 +49,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -307,14 +305,10 @@ public class DumpReplicationQueues extends Configured implements Tool {
       boolean hdfs) throws Exception {
     ReplicationQueueStorage queueStorage;
     ReplicationPeers replicationPeers;
-    ReplicationQueues replicationQueues;
     ReplicationTracker replicationTracker;
-    ReplicationQueuesArguments replicationArgs =
-        new ReplicationQueuesArguments(getConf(), new WarnOnlyAbortable(), zkw);
     StringBuilder sb = new StringBuilder();
 
     queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
-    replicationQueues = ReplicationFactory.getReplicationQueues(replicationArgs);
     replicationPeers =
         ReplicationFactory.getReplicationPeers(zkw, getConf(), queueStorage, connection);
     replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
@@ -328,7 +322,6 @@ public class DumpReplicationQueues extends Configured implements Tool {
     }
     for (ServerName regionserver : regionservers) {
       List<String> queueIds = queueStorage.getAllQueues(regionserver);
-      replicationQueues.init(regionserver.getServerName());
       if (!liveRegionServers.contains(regionserver.getServerName())) {
         deadRegionServers.add(regionserver.getServerName());
       }
@@ -338,17 +331,17 @@ public class DumpReplicationQueues extends Configured implements Tool {
         if (!peerIds.contains(queueInfo.getPeerId())) {
           deletedQueues.add(regionserver + "/" + queueId);
           sb.append(
-            formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, hdfs));
+            formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, true, hdfs));
         } else {
           sb.append(
-            formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, hdfs));
+            formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, false, hdfs));
         }
       }
     }
     return sb.toString();
   }
 
-  private String formatQueue(ServerName regionserver, ReplicationQueues replicationQueues,
+  private String formatQueue(ServerName regionserver, ReplicationQueueStorage queueStorage,
       ReplicationQueueInfo queueInfo, String queueId, List<String> wals, boolean isDeleted,
       boolean hdfs) throws Exception {
     StringBuilder sb = new StringBuilder();
@@ -370,7 +363,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
     peersQueueSize.addAndGet(queueInfo.getPeerId(), wals.size());
 
     for (String wal : wals) {
-      long position = replicationQueues.getLogPosition(queueInfo.getPeerId(), wal);
+      long position = queueStorage.getWALPosition(regionserver, queueInfo.getPeerId(), wal);
       sb.append("    Replication position for " + wal + ": " + (position > 0 ? position : "0"
           + " (not started or nothing to replicate)") + "\n");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index bd191e3..e0c45d5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -29,15 +28,15 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class that handles the recovered source of a replication stream, which is transfered from
@@ -52,10 +51,10 @@ public class RecoveredReplicationSource extends ReplicationSource {
 
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server,
+      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
       String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
-    super.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerClusterZnode,
+    super.init(conf, fs, manager, queueStorage, replicationPeers, server, peerClusterZnode,
       clusterId, replicationEndpoint, walFileLengthProvider, metrics);
     this.actualPeerId = this.replicationQueueInfo.getPeerId();
   }
@@ -64,7 +63,7 @@ public class RecoveredReplicationSource extends ReplicationSource {
   protected void tryStartNewShipper(String walGroupId, PriorityBlockingQueue<Path> queue) {
     final RecoveredReplicationSourceShipper worker =
         new RecoveredReplicationSourceShipper(conf, walGroupId, queue, this,
-            this.replicationQueues);
+            this.queueStorage);
     ReplicationSourceShipper extant = workerThreads.putIfAbsent(walGroupId, worker);
     if (extant != null) {
       LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
index 630b90b..fb365bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
@@ -23,13 +23,13 @@ import java.util.concurrent.PriorityBlockingQueue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.WALEntryBatch;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *  Used by a {@link RecoveredReplicationSource}.
@@ -40,14 +40,14 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
       LoggerFactory.getLogger(RecoveredReplicationSourceShipper.class);
 
   protected final RecoveredReplicationSource source;
-  private final ReplicationQueues replicationQueues;
+  private final ReplicationQueueStorage replicationQueues;
 
   public RecoveredReplicationSourceShipper(Configuration conf, String walGroupId,
       PriorityBlockingQueue<Path> queue, RecoveredReplicationSource source,
-      ReplicationQueues replicationQueues) {
+      ReplicationQueueStorage queueStorage) {
     super(conf, walGroupId, queue, source);
     this.source = source;
-    this.replicationQueues = replicationQueues;
+    this.replicationQueues = queueStorage;
   }
 
   @Override
@@ -116,11 +116,11 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
     long startPosition = 0;
     String peerClusterZnode = source.getPeerClusterZnode();
     try {
-      startPosition = this.replicationQueues.getLogPosition(peerClusterZnode,
-        this.queue.peek().getName());
+      startPosition = this.replicationQueues.getWALPosition(source.getServerWALsBelongTo(),
+        peerClusterZnode, this.queue.peek().getName());
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Recovered queue started with log " + this.queue.peek() + " at position "
-            + startPosition);
+        LOG.trace("Recovered queue started with log " + this.queue.peek() + " at position " +
+          startPosition);
       }
     } catch (ReplicationException e) {
       terminate("Couldn't get the position of this recovered queue " + peerClusterZnode, e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index ff45f53..72f0fe7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -28,12 +27,6 @@ import java.util.UUID;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,27 +36,33 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 /**
  * Gateway to Replication.  Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  */
@@ -74,7 +73,7 @@ public class Replication implements
       LoggerFactory.getLogger(Replication.class);
   private boolean replicationForBulkLoadData;
   private ReplicationSourceManager replicationManager;
-  private ReplicationQueues replicationQueues;
+  private ReplicationQueueStorage queueStorage;
   private ReplicationPeers replicationPeers;
   private ReplicationTracker replicationTracker;
   private Configuration conf;
@@ -127,10 +126,8 @@ public class Replication implements
     }
 
     try {
-      this.replicationQueues =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, this.server,
-            server.getZooKeeper()));
-      this.replicationQueues.init(this.server.getServerName().toString());
+      this.queueStorage =
+          ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
       this.replicationPeers =
           ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server);
       this.replicationPeers.init();
@@ -147,7 +144,7 @@ public class Replication implements
       throw new IOException("Could not read cluster id", ke);
     }
     this.replicationManager =
-        new ReplicationSourceManager(replicationQueues, replicationPeers, replicationTracker, conf,
+        new ReplicationSourceManager(queueStorage, replicationPeers, replicationTracker, conf,
             this.server, fs, logDir, oldLogDir, clusterId, walFileLengthProvider);
     this.statsThreadPeriod =
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 3744017..271eea7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -31,7 +30,6 @@ import java.util.concurrent.PriorityBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -41,9 +39,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
 import org.apache.hadoop.hbase.replication.ClusterMarkingEntryFilter;
@@ -52,7 +47,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -60,6 +55,10 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 
@@ -83,7 +82,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   private Map<String, PriorityBlockingQueue<Path>> queues = new HashMap<>();
   // per group queue size, keep no more than this number of logs in each wal group
   protected int queueSizePerGroup;
-  protected ReplicationQueues replicationQueues;
+  protected ReplicationQueueStorage queueStorage;
   private ReplicationPeers replicationPeers;
 
   protected Configuration conf;
@@ -148,7 +147,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
    */
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server,
+      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
       String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     this.server = server;
@@ -161,7 +160,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     this.maxRetriesMultiplier =
         this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per
     this.queueSizePerGroup = this.conf.getInt("hbase.regionserver.maxlogs", 32);
-    this.replicationQueues = replicationQueues;
+    this.queueStorage = queueStorage;
     this.replicationPeers = replicationPeers;
     this.manager = manager;
     this.fs = fs;
@@ -229,7 +228,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       List<String> tableCfs = tableCFMap.get(tableName);
       if (tableCFMap.containsKey(tableName)
           && (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) {
-        this.replicationQueues.addHFileRefs(peerId, pairs);
+        this.queueStorage.addHFileRefs(peerId, pairs);
         metrics.incrSizeOfHFileRefsQueue(pairs.size());
       } else {
         LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
@@ -238,7 +237,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     } else {
       // user has explicitly not defined any table cfs for replication, means replicate all the
       // data
-      this.replicationQueues.addHFileRefs(peerId, pairs);
+      this.queueStorage.addHFileRefs(peerId, pairs);
       metrics.incrSizeOfHFileRefsQueue(pairs.size());
     }
   }


[32/48] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index b6cf54d..4b9ed74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,9 +31,10 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Interface that defines a replication source
@@ -47,15 +47,10 @@ public interface ReplicationSourceInterface {
    * @param conf the configuration to use
    * @param fs the file system to use
    * @param manager the manager to use
-   * @param replicationQueues
-   * @param replicationPeers
    * @param server the server for this region server
-   * @param peerClusterZnode
-   * @param clusterId
-   * @throws IOException
    */
   void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server,
+      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
       String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index b1d82c8..853bafb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -34,19 +34,21 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-
+import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
@@ -60,7 +62,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -68,6 +70,7 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -95,7 +98,7 @@ public class ReplicationSourceManager implements ReplicationListener {
   private final List<ReplicationSourceInterface> sources;
   // List of all the sources we got from died RSs
   private final List<ReplicationSourceInterface> oldsources;
-  private final ReplicationQueues replicationQueues;
+  private final ReplicationQueueStorage queueStorage;
   private final ReplicationTracker replicationTracker;
   private final ReplicationPeers replicationPeers;
   // UUID for this cluster
@@ -130,7 +133,7 @@ public class ReplicationSourceManager implements ReplicationListener {
 
   /**
    * Creates a replication manager and sets the watch on all the other registered region servers
-   * @param replicationQueues the interface for manipulating replication queues
+   * @param queueStorage the interface for manipulating replication queues
    * @param replicationPeers
    * @param replicationTracker
    * @param conf the configuration to use
@@ -140,14 +143,14 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param oldLogDir the directory where old logs are archived
    * @param clusterId
    */
-  public ReplicationSourceManager(ReplicationQueues replicationQueues,
+  public ReplicationSourceManager(ReplicationQueueStorage queueStorage,
       ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf,
       Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId,
       WALFileLengthProvider walFileLengthProvider) throws IOException {
     //CopyOnWriteArrayList is thread-safe.
     //Generally, reading is more than modifying.
     this.sources = new CopyOnWriteArrayList<>();
-    this.replicationQueues = replicationQueues;
+    this.queueStorage = queueStorage;
     this.replicationPeers = replicationPeers;
     this.replicationTracker = replicationTracker;
     this.server = server;
@@ -184,6 +187,19 @@ public class ReplicationSourceManager implements ReplicationListener {
     connection = ConnectionFactory.createConnection(conf);
   }
 
+  @FunctionalInterface
+  private interface ReplicationQueueOperation {
+    void exec() throws ReplicationException;
+  }
+
+  private void abortWhenFail(ReplicationQueueOperation op) {
+    try {
+      op.exec();
+    } catch (ReplicationException e) {
+      server.abort("Failed to operate on replication queue", e);
+    }
+  }
+
   /**
    * Provide the id of the peer and a log key and this method will figure which
    * wal it belongs to and will log, for this region server, the current
@@ -195,12 +211,13 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param queueRecovered indicates if this queue comes from another region server
    * @param holdLogInZK if true then the log is retained in ZK
    */
-  public void logPositionAndCleanOldLogs(Path log, String id, long position,
-      boolean queueRecovered, boolean holdLogInZK) {
+  public void logPositionAndCleanOldLogs(Path log, String id, long position, boolean queueRecovered,
+      boolean holdLogInZK) {
     String fileName = log.getName();
-    this.replicationQueues.setLogPosition(id, fileName, position);
+    abortWhenFail(
+      () -> this.queueStorage.setWALPosition(server.getServerName(), id, fileName, position));
     if (holdLogInZK) {
-     return;
+      return;
     }
     cleanOldLogs(fileName, id, queueRecovered);
   }
@@ -227,36 +244,59 @@ public class ReplicationSourceManager implements ReplicationListener {
         }
       }
     }
- }
+  }
 
   private void cleanOldLogs(SortedSet<String> wals, String key, String id) {
     SortedSet<String> walSet = wals.headSet(key);
-    LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
+    }
     for (String wal : walSet) {
-      this.replicationQueues.removeLog(id, wal);
+      abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), id, wal));
     }
     walSet.clear();
   }
 
+  private void adoptAbandonedQueues() {
+    List<ServerName> currentReplicators = null;
+    try {
+      currentReplicators = queueStorage.getListOfReplicators();
+    } catch (ReplicationException e) {
+      server.abort("Failed to get all replicators", e);
+      return;
+    }
+    if (currentReplicators == null || currentReplicators.isEmpty()) {
+      return;
+    }
+    List<ServerName> otherRegionServers = replicationTracker.getListOfRegionServers().stream()
+        .map(ServerName::valueOf).collect(Collectors.toList());
+    LOG.info(
+      "Current list of replicators: " + currentReplicators + " other RSs: " + otherRegionServers);
+
+    // Look if there's anything to process after a restart
+    for (ServerName rs : currentReplicators) {
+      if (!otherRegionServers.contains(rs)) {
+        transferQueues(rs);
+      }
+    }
+  }
+
   /**
-   * Adds a normal source per registered peer cluster and tries to process all
-   * old region server wal queues
+   * Adds a normal source per registered peer cluster and tries to process all old region server wal
+   * queues
+   * <p>
+   * The returned future is for adoptAbandonedQueues task.
    */
-  void init() throws IOException, ReplicationException {
+  Future<?> init() throws IOException, ReplicationException {
     for (String id : this.replicationPeers.getConnectedPeerIds()) {
       addSource(id);
       if (replicationForBulkLoadDataEnabled) {
         // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
         // when a peer was added before replication for bulk loaded data was enabled.
-        this.replicationQueues.addPeerToHFileRefs(id);
+        this.queueStorage.addPeerToHFileRefs(id);
       }
     }
-    AdoptAbandonedQueuesWorker adoptionWorker = new AdoptAbandonedQueuesWorker();
-    try {
-      this.executor.execute(adoptionWorker);
-    } catch (RejectedExecutionException ex) {
-      LOG.info("Cancelling the adoption of abandoned queues because of " + ex.getMessage());
-    }
+    return this.executor.submit(this::adoptAbandonedQueues);
   }
 
   /**
@@ -264,15 +304,12 @@ public class ReplicationSourceManager implements ReplicationListener {
    * need to enqueue the latest log of each wal group and do replication
    * @param id the id of the peer cluster
    * @return the source that was created
-   * @throws IOException
    */
   @VisibleForTesting
   ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException {
     ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
     ReplicationPeer peer = replicationPeers.getConnectedPeer(id);
-    ReplicationSourceInterface src = getReplicationSource(this.conf, this.fs, this,
-      this.replicationQueues, this.replicationPeers, server, id, this.clusterId, peerConfig, peer,
-      walFileLengthProvider);
+    ReplicationSourceInterface src = getReplicationSource(id, peerConfig, peer);
     synchronized (this.walsById) {
       this.sources.add(src);
       Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
@@ -287,11 +324,10 @@ public class ReplicationSourceManager implements ReplicationListener {
             logs.add(name);
             walsByGroup.put(walPrefix, logs);
             try {
-              this.replicationQueues.addLog(id, name);
+              this.queueStorage.addWAL(server.getServerName(), id, name);
             } catch (ReplicationException e) {
-              String message =
-                  "Cannot add log to queue when creating a new source, queueId=" + id
-                      + ", filename=" + name;
+              String message = "Cannot add log to queue when creating a new source, queueId=" + id +
+                ", filename=" + name;
               server.stop(message);
               throw e;
             }
@@ -316,7 +352,7 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param peerId Id of the peer cluster queue of wals to delete
    */
   public void deleteSource(String peerId, boolean closeConnection) {
-    this.replicationQueues.removeQueue(peerId);
+    abortWhenFail(() -> this.queueStorage.removeQueue(server.getServerName(), peerId));
     if (closeConnection) {
       this.replicationPeers.peerDisconnected(peerId);
     }
@@ -376,8 +412,8 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   @VisibleForTesting
-  List<String> getAllQueues() {
-    return replicationQueues.getAllQueues();
+  List<String> getAllQueues() throws ReplicationException {
+    return queueStorage.getAllQueues(server.getServerName());
   }
 
   void preLogRoll(Path newLog) throws IOException {
@@ -411,10 +447,10 @@ public class ReplicationSourceManager implements ReplicationListener {
     synchronized (replicationPeers) {
       for (String id : replicationPeers.getConnectedPeerIds()) {
         try {
-          this.replicationQueues.addLog(id, logName);
+          this.queueStorage.addWAL(server.getServerName(), id, logName);
         } catch (ReplicationException e) {
-          throw new IOException("Cannot add log to replication queue"
-              + " when creating a new source, queueId=" + id + ", filename=" + logName, e);
+          throw new IOException("Cannot add log to replication queue" +
+            " when creating a new source, queueId=" + id + ", filename=" + logName, e);
         }
       }
     }
@@ -461,19 +497,11 @@ public class ReplicationSourceManager implements ReplicationListener {
 
   /**
    * Factory method to create a replication source
-   * @param conf the configuration to use
-   * @param fs the file system to use
-   * @param manager the manager to use
-   * @param server the server object for this region server
    * @param peerId the id of the peer cluster
    * @return the created source
-   * @throws IOException
    */
-  private ReplicationSourceInterface getReplicationSource(Configuration conf, FileSystem fs,
-      ReplicationSourceManager manager, ReplicationQueues replicationQueues,
-      ReplicationPeers replicationPeers, Server server, String peerId, UUID clusterId,
-      ReplicationPeerConfig peerConfig, ReplicationPeer replicationPeer,
-      WALFileLengthProvider walFileLengthProvider) throws IOException {
+  private ReplicationSourceInterface getReplicationSource(String peerId,
+      ReplicationPeerConfig peerConfig, ReplicationPeer replicationPeer) throws IOException {
     RegionServerCoprocessorHost rsServerHost = null;
     TableDescriptors tableDescriptors = null;
     if (server instanceof HRegionServer) {
@@ -490,9 +518,8 @@ public class ReplicationSourceManager implements ReplicationListener {
         // Default to HBase inter-cluster replication endpoint
         replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
       }
-      @SuppressWarnings("rawtypes")
-      Class c = Class.forName(replicationEndpointImpl);
-      replicationEndpoint = (ReplicationEndpoint) c.newInstance();
+      replicationEndpoint = Class.forName(replicationEndpointImpl)
+          .asSubclass(ReplicationEndpoint.class).newInstance();
       if(rsServerHost != null) {
         ReplicationEndpoint newReplicationEndPoint = rsServerHost
             .postCreateReplicationEndPoint(replicationEndpoint);
@@ -509,7 +536,7 @@ public class ReplicationSourceManager implements ReplicationListener {
 
     MetricsSource metrics = new MetricsSource(peerId);
     // init replication source
-    src.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerId, clusterId,
+    src.init(conf, fs, this, queueStorage, replicationPeers, server, peerId, clusterId,
       replicationEndpoint, walFileLengthProvider, metrics);
 
     // init replication endpoint
@@ -520,21 +547,21 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
-   * Transfer all the queues of the specified to this region server.
-   * First it tries to grab a lock and if it works it will move the
-   * znodes and finally will delete the old znodes.
-   *
+   * Transfer all the queues of the specified to this region server. First it tries to grab a lock
+   * and if it works it will move the znodes and finally will delete the old znodes.
+   * <p>
    * It creates one old source for any type of source of the old rs.
-   * @param rsZnode
    */
-  private void transferQueues(String rsZnode) {
-    NodeFailoverWorker transfer =
-        new NodeFailoverWorker(rsZnode, this.replicationQueues, this.replicationPeers,
-            this.clusterId);
+  private void transferQueues(ServerName deadRS) {
+    if (server.getServerName().equals(deadRS)) {
+      // it's just us, give up
+      return;
+    }
+    NodeFailoverWorker transfer = new NodeFailoverWorker(deadRS);
     try {
       this.executor.execute(transfer);
     } catch (RejectedExecutionException ex) {
-      LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage());
+      LOG.info("Cancelling the transfer of " + deadRS + " because of " + ex.getMessage());
     }
   }
 
@@ -571,7 +598,7 @@ public class ReplicationSourceManager implements ReplicationListener {
       LOG.info("Peer " + id + " connected success, trying to start the replication source thread.");
       addSource(id);
       if (replicationForBulkLoadDataEnabled) {
-        this.replicationQueues.addPeerToHFileRefs(id);
+        this.queueStorage.addPeerToHFileRefs(id);
       }
     }
   }
@@ -624,12 +651,12 @@ public class ReplicationSourceManager implements ReplicationListener {
       deleteSource(id, true);
     }
     // Remove HFile Refs znode from zookeeper
-    this.replicationQueues.removePeerFromHFileRefs(id);
+    abortWhenFail(() -> this.queueStorage.removePeerFromHFileRefs(id));
   }
 
   @Override
   public void regionServerRemoved(String regionserver) {
-    transferQueues(regionserver);
+    transferQueues(ServerName.valueOf(regionserver));
   }
 
   /**
@@ -638,37 +665,21 @@ public class ReplicationSourceManager implements ReplicationListener {
    */
   class NodeFailoverWorker extends Thread {
 
-    private String rsZnode;
-    private final ReplicationQueues rq;
-    private final ReplicationPeers rp;
-    private final UUID clusterId;
+    private final ServerName deadRS;
 
-    /**
-     * @param rsZnode
-     */
-    public NodeFailoverWorker(String rsZnode) {
-      this(rsZnode, replicationQueues, replicationPeers, ReplicationSourceManager.this.clusterId);
-    }
-
-    public NodeFailoverWorker(String rsZnode, final ReplicationQueues replicationQueues,
-        final ReplicationPeers replicationPeers, final UUID clusterId) {
-      super("Failover-for-"+rsZnode);
-      this.rsZnode = rsZnode;
-      this.rq = replicationQueues;
-      this.rp = replicationPeers;
-      this.clusterId = clusterId;
+    @VisibleForTesting
+    public NodeFailoverWorker(ServerName deadRS) {
+      super("Failover-for-" + deadRS);
+      this.deadRS = deadRS;
     }
 
     @Override
     public void run() {
-      if (this.rq.isThisOurRegionServer(rsZnode)) {
-        return;
-      }
       // Wait a bit before transferring the queues, we may be shutting down.
       // This sleep may not be enough in some cases.
       try {
         Thread.sleep(sleepBeforeFailover +
-            (long) (ThreadLocalRandom.current().nextFloat() * sleepBeforeFailover));
+          (long) (ThreadLocalRandom.current().nextFloat() * sleepBeforeFailover));
       } catch (InterruptedException e) {
         LOG.warn("Interrupted while waiting before transferring a queue.");
         Thread.currentThread().interrupt();
@@ -679,25 +690,30 @@ public class ReplicationSourceManager implements ReplicationListener {
         return;
       }
       Map<String, Set<String>> newQueues = new HashMap<>();
-      List<String> peers = rq.getUnClaimedQueueIds(rsZnode);
-      while (peers != null && !peers.isEmpty()) {
-        Pair<String, SortedSet<String>> peer = this.rq.claimQueue(rsZnode,
-          peers.get(ThreadLocalRandom.current().nextInt(peers.size())));
-        long sleep = sleepBeforeFailover/2;
-        if (peer != null) {
-          newQueues.put(peer.getFirst(), peer.getSecond());
-          sleep = sleepBeforeFailover;
+      try {
+        List<String> peers = queueStorage.getAllQueues(deadRS);
+        while (!peers.isEmpty()) {
+          Pair<String, SortedSet<String>> peer = queueStorage.claimQueue(deadRS,
+            peers.get(ThreadLocalRandom.current().nextInt(peers.size())), server.getServerName());
+          long sleep = sleepBeforeFailover / 2;
+          if (!peer.getSecond().isEmpty()) {
+            newQueues.put(peer.getFirst(), peer.getSecond());
+            sleep = sleepBeforeFailover;
+          }
+          try {
+            Thread.sleep(sleep);
+          } catch (InterruptedException e) {
+            LOG.warn("Interrupted while waiting before transferring a queue.");
+            Thread.currentThread().interrupt();
+          }
+          peers = queueStorage.getAllQueues(deadRS);
         }
-        try {
-          Thread.sleep(sleep);
-        } catch (InterruptedException e) {
-          LOG.warn("Interrupted while waiting before transferring a queue.");
-          Thread.currentThread().interrupt();
+        if (!peers.isEmpty()) {
+          queueStorage.removeReplicatorIfQueueIsEmpty(deadRS);
         }
-        peers = rq.getUnClaimedQueueIds(rsZnode);
-      }
-      if (peers != null) {
-        rq.removeReplicatorIfQueueIsEmpty(rsZnode);
+      } catch (ReplicationException e) {
+        server.abort("Failed to claim queue from dead regionserver", e);
+        return;
       }
       // Copying over the failed queue is completed.
       if (newQueues.isEmpty()) {
@@ -722,8 +738,8 @@ public class ReplicationSourceManager implements ReplicationListener {
                 + ex);
           }
           if (peer == null || peerConfig == null) {
-            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode);
-            replicationQueues.removeQueue(peerId);
+            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS);
+            abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
             continue;
           }
           // track sources in walsByIdRecoveredQueues
@@ -740,13 +756,11 @@ public class ReplicationSourceManager implements ReplicationListener {
           }
 
           // enqueue sources
-          ReplicationSourceInterface src =
-              getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
-                server, peerId, this.clusterId, peerConfig, peer, walFileLengthProvider);
+          ReplicationSourceInterface src = getReplicationSource(peerId, peerConfig, peer);
           // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
           // see removePeer
           synchronized (oldsources) {
-            if (!this.rp.getConnectedPeerIds().contains(src.getPeerId())) {
+            if (!replicationPeers.getConnectedPeerIds().contains(src.getPeerId())) {
               src.terminate("Recovered queue doesn't belong to any current peer");
               closeRecoveredQueue(src);
               continue;
@@ -765,29 +779,6 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
   }
 
-  class AdoptAbandonedQueuesWorker extends Thread{
-
-    public AdoptAbandonedQueuesWorker() {}
-
-    @Override
-    public void run() {
-      List<String> currentReplicators = replicationQueues.getListOfReplicators();
-      if (currentReplicators == null || currentReplicators.isEmpty()) {
-        return;
-      }
-      List<String> otherRegionServers = replicationTracker.getListOfRegionServers();
-      LOG.info("Current list of replicators: " + currentReplicators + " other RSs: "
-        + otherRegionServers);
-
-      // Look if there's anything to process after a restart
-      for (String rs : currentReplicators) {
-        if (!otherRegionServers.contains(rs)) {
-          transferQueues(rs);
-        }
-      }
-    }
-  }
-
   /**
    * Get the directory where wals are archived
    * @return the directory where wals are archived
@@ -846,7 +837,11 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   public void cleanUpHFileRefs(String peerId, List<String> files) {
-    this.replicationQueues.removeHFileRefs(peerId, files);
+    abortWhenFail(() -> this.queueStorage.removeHFileRefs(peerId, files));
+  }
+
+  int activeFailoverTaskCount() {
+    return executor.getActiveCount();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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 21b8ac5..9ec244a 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
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,22 +36,19 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
- * In a scenario of Replication based Disaster/Recovery, when hbase
- * Master-Cluster crashes, this tool is used to sync-up the delta from Master to
- * Slave using the info from ZooKeeper. The tool will run on Master-Cluser, and
- * assume ZK, Filesystem and NetWork still available after hbase crashes
+ * In a scenario of Replication based Disaster/Recovery, when hbase Master-Cluster crashes, this
+ * tool is used to sync-up the delta from Master to Slave using the info from ZooKeeper. The tool
+ * will run on Master-Cluser, and assume ZK, Filesystem and NetWork still available after hbase
+ * crashes
  *
+ * <pre>
  * hbase org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp
+ * </pre>
  */
-
 public class ReplicationSyncUp extends Configured implements Tool {
 
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationSyncUp.class.getName());
-
   private static Configuration conf;
 
   private static final long SLEEP_TIME = 10000;
@@ -105,13 +101,14 @@ public class ReplicationSyncUp extends Configured implements Tool {
     System.out.println("Start Replication Server start");
     replication = new Replication(new DummyServer(zkw), fs, logDir, oldLogDir);
     manager = replication.getReplicationManager();
-    manager.init();
+    manager.init().get();
 
     try {
-      int numberOfOldSource = 1; // default wait once
-      while (numberOfOldSource > 0) {
+      while (manager.activeFailoverTaskCount() > 0) {
+        Thread.sleep(SLEEP_TIME);
+      }
+      while (manager.getOldSources().size() > 0) {
         Thread.sleep(SLEEP_TIME);
-        numberOfOldSource = manager.getOldSources().size();
       }
     } catch (InterruptedException e) {
       System.err.println("didn't wait long enough:" + e);
@@ -121,7 +118,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
     manager.join();
     zkw.close();
 
-    return (0);
+    return 0;
   }
 
   static class DummyServer implements Server {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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 1e75959..2de6608 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
@@ -44,9 +44,8 @@ import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -111,9 +110,8 @@ public class TestLogsCleaner {
 
     Replication.decorateMasterConfiguration(conf);
     Server server = new DummyServer();
-    ReplicationQueues repQueues = ReplicationFactory.getReplicationQueues(
-        new ReplicationQueuesArguments(conf, server, server.getZooKeeper()));
-    repQueues.init(server.getServerName().toString());
+    ReplicationQueueStorage queueStorage =
+        ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
     final Path oldLogDir = new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_OLDLOGDIR_NAME);
     final Path oldProcedureWALDir = new Path(oldLogDir, "masterProcedureWALs");
     String fakeMachineName = URLEncoder.encode(server.getServerName().toString(), "UTF8");
@@ -144,7 +142,7 @@ public class TestLogsCleaner {
       // Case 4: put 3 WALs in ZK indicating that they are scheduled for replication so these
       // files would pass TimeToLiveLogCleaner but would be rejected by ReplicationLogCleaner
       if (i % (30 / 3) == 1) {
-        repQueues.addLog(fakeMachineName, fileName.getName());
+        queueStorage.addWAL(server.getServerName(), fakeMachineName, fileName.getName());
         LOG.info("Replication log file: " + fileName);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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 f83695f..8802e36 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
@@ -46,9 +46,8 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -72,19 +71,16 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 @Category({ MasterTests.class, SmallTests.class })
 public class TestReplicationHFileCleaner {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationHFileCleaner.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Server server;
-  private static ReplicationQueues rq;
+  private static ReplicationQueueStorage rq;
   private static ReplicationPeers rp;
   private static final String peerId = "TestReplicationHFileCleaner";
   private static Configuration conf = TEST_UTIL.getConfiguration();
   static FileSystem fs = null;
   Path root;
 
-  /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniZKCluster();
@@ -93,20 +89,10 @@ public class TestReplicationHFileCleaner {
     Replication.decorateMasterConfiguration(conf);
     rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf, server);
     rp.init();
-    rq = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, server, server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
-    try {
-      fs = FileSystem.get(conf);
-    } finally {
-      if (fs != null) {
-        fs.close();
-      }
-    }
+    rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
+    fs = FileSystem.get(conf);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniZKCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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 8178266..2ad8bd7 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
@@ -26,10 +26,8 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -49,14 +47,12 @@ public class TestReplicationZKNodeCleaner {
 
   private final Configuration conf;
   private final ZKWatcher zkw;
-  private final ReplicationQueues repQueues;
+  private final ReplicationQueueStorage repQueues;
 
   public TestReplicationZKNodeCleaner() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     zkw = new ZKWatcher(conf, "TestReplicationZKNodeCleaner", null);
-    repQueues = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null,
-        zkw));
-    assertTrue(repQueues instanceof ReplicationQueuesZKImpl);
+    repQueues = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
   }
 
   @BeforeClass
@@ -72,9 +68,8 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleaner() throws Exception {
-    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
-    repQueues.addLog(ID_ONE, "file1");
+    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
 
     ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
     Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
@@ -84,7 +79,7 @@ public class TestReplicationZKNodeCleaner {
     assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE));
 
     // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addLog(ID_TWO + "-" + SERVER_TWO, "file2");
+    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
 
     undeletedQueues = cleaner.getUnDeletedQueues();
     assertEquals(1, undeletedQueues.size());
@@ -100,11 +95,10 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleanerChore() throws Exception {
-    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
-    repQueues.addLog(ID_ONE, "file1");
+    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
     // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addLog(ID_TWO + "-" + SERVER_TWO, "file2");
+    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
 
     // Wait the cleaner chore to run
     Thread.sleep(20000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index 7ea79f9..14c5e56 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -46,9 +45,10 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   MetricsSource metrics;
   WALFileLengthProvider walFileLengthProvider;
   AtomicBoolean startup = new AtomicBoolean(false);
+
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues rq, ReplicationPeers rp, Server server, String peerClusterId,
+      ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
       UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     this.manager = manager;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
index 0a602ad..0313b3b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -64,7 +64,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
 
   @Before
   public void setUp() throws Exception {
-
     HColumnDescriptor fam;
 
     t1_syncupSource = new HTableDescriptor(t1_su);
@@ -100,7 +99,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
    * check's gone Also check the puts and deletes are not replicated back to
    * the originating cluster.
    */
-  @Test(timeout = 300000)
+  @Test
   public void testSyncUpTool() throws Exception {
 
     /**
@@ -176,7 +175,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
      * verify correctly replicated to Slave
      */
     mimicSyncUpAfterPut();
-
   }
 
   protected void setupReplication() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/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 b5a7461..325012d 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
@@ -40,7 +40,6 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -67,10 +66,10 @@ import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -95,11 +94,13 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 /**
  * An abstract class that tests ReplicationSourceManager. Classes that extend this class should
  * set up the proper config for this class and initialize the proper cluster using
@@ -328,18 +329,14 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testClaimQueues() throws Exception {
-    final Server server = new DummyServer("hostname0.example.org");
-
-
-    ReplicationQueues rq =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-          server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname0.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     files.add("log1");
     files.add("log2");
     for (String file : files) {
-      rq.addLog("1", file);
+      rq.addWAL(server.getServerName(), "1", file);
     }
     // create 3 DummyServers
     Server s1 = new DummyServer("dummyserver1.example.org");
@@ -347,12 +344,9 @@ public abstract class TestReplicationSourceManager {
     Server s3 = new DummyServer("dummyserver3.example.org");
 
     // create 3 DummyNodeFailoverWorkers
-    DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s1);
-    DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s2);
-    DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s3);
+    DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(server.getServerName(), s1);
+    DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(server.getServerName(), s2);
+    DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(server.getServerName(), s3);
 
     latch = new CountDownLatch(3);
     // start the threads
@@ -371,11 +365,9 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testCleanupFailoverQueues() throws Exception {
-    final Server server = new DummyServer("hostname1.example.org");
-    ReplicationQueues rq =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-          server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname1.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     SortedSet<String> files = new TreeSet<>();
     String group = "testgroup";
@@ -384,19 +376,14 @@ public abstract class TestReplicationSourceManager {
     files.add(file1);
     files.add(file2);
     for (String file : files) {
-      rq.addLog("1", file);
+      rq.addWAL(server.getServerName(), "1", file);
     }
     Server s1 = new DummyServer("dummyserver1.example.org");
-    ReplicationQueues rq1 =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1,
-            s1.getZooKeeper()));
-    rq1.init(s1.getServerName().toString());
     ReplicationPeers rp1 =
         ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration(), s1);
     rp1.init();
     NodeFailoverWorker w1 =
-        manager.new NodeFailoverWorker(server.getServerName().getServerName(), rq1, rp1, new UUID(
-            new Long(1), new Long(2)));
+        manager.new NodeFailoverWorker(server.getServerName());
     w1.run();
     assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
     String id = "1-" + server.getServerName().getServerName();
@@ -408,17 +395,16 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testCleanupUnknownPeerZNode() throws Exception {
-    final Server server = new DummyServer("hostname2.example.org");
-    ReplicationQueues rq = ReplicationFactory.getReplicationQueues(
-      new ReplicationQueuesArguments(server.getConfiguration(), server, server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname2.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     // add log to an unknown peer
     String group = "testgroup";
-    rq.addLog("2", group + ".log1");
-    rq.addLog("2", group + ".log2");
+    rq.addWAL(server.getServerName(), "2", group + ".log1");
+    rq.addWAL(server.getServerName(), "2", group + ".log2");
 
-    NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName().getServerName());
+    NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName());
     w1.run();
 
     // The log of the unknown peer should be removed from zk
@@ -481,10 +467,8 @@ public abstract class TestReplicationSourceManager {
         .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase");
     try {
       DummyServer server = new DummyServer();
-      final ReplicationQueues rq =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(
-              server.getConfiguration(), server, server.getZooKeeper()));
-      rq.init(server.getServerName().toString());
+      ReplicationQueueStorage rq = ReplicationStorageFactory
+          .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
       // Purposely fail ReplicationSourceManager.addSource() by causing ReplicationSourceInterface
       // initialization to throw an exception.
       conf.set("replication.replicationsource.implementation",
@@ -498,11 +482,11 @@ public abstract class TestReplicationSourceManager {
       assertNull(manager.getSource(peerId));
 
       // Create a replication queue for the fake peer
-      rq.addLog(peerId, "FakeFile");
+      rq.addWAL(server.getServerName(), peerId, "FakeFile");
       // Unregister peer, this should remove the peer and clear all queues associated with it
       // Need to wait for the ReplicationTracker to pick up the changes and notify listeners.
       removePeerAndWait(peerId);
-      assertFalse(rq.getAllQueues().contains(peerId));
+      assertFalse(rq.getAllQueues(server.getServerName()).contains(peerId));
     } finally {
       conf.set("replication.replicationsource.implementation", replicationSourceImplName);
       removePeerAndWait(peerId);
@@ -625,11 +609,12 @@ public abstract class TestReplicationSourceManager {
       }
     }
     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
-      @Override public boolean evaluate() throws Exception {
+      @Override
+      public boolean evaluate() throws Exception {
         List<String> peers = rp.getAllPeerIds();
-        return (!manager.getAllQueues().contains(peerId)) && (rp.getConnectedPeer(peerId) == null)
-            && (!peers.contains(peerId))
-            && manager.getSource(peerId) == null;
+        return (!manager.getAllQueues().contains(peerId)) &&
+          (rp.getConnectedPeer(peerId) == null) && (!peers.contains(peerId)) &&
+          manager.getSource(peerId) == null;
       }
     });
   }
@@ -672,25 +657,24 @@ public abstract class TestReplicationSourceManager {
   static class DummyNodeFailoverWorker extends Thread {
     private Map<String, Set<String>> logZnodesMap;
     Server server;
-    private String deadRsZnode;
-    ReplicationQueues rq;
+    private ServerName deadRS;
+    ReplicationQueueStorage rq;
 
-    public DummyNodeFailoverWorker(String znode, Server s) throws Exception {
-      this.deadRsZnode = znode;
+    public DummyNodeFailoverWorker(ServerName deadRS, Server s) throws Exception {
+      this.deadRS = deadRS;
       this.server = s;
-      this.rq =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-            server.getZooKeeper()));
-      this.rq.init(this.server.getServerName().toString());
+      this.rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(),
+        server.getConfiguration());
     }
 
     @Override
     public void run() {
       try {
         logZnodesMap = new HashMap<>();
-        List<String> queues = rq.getUnClaimedQueueIds(deadRsZnode);
-        for(String queue:queues){
-          Pair<String, SortedSet<String>> pair = rq.claimQueue(deadRsZnode, queue);
+        List<String> queues = rq.getAllQueues(deadRS);
+        for (String queue : queues) {
+          Pair<String, SortedSet<String>> pair =
+              rq.claimQueue(deadRS, queue, server.getServerName());
           if (pair != null) {
             logZnodesMap.put(pair.getFirst(), pair.getSecond());
           }
@@ -729,7 +713,7 @@ public abstract class TestReplicationSourceManager {
 
     @Override
     public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-        ReplicationQueues rq, ReplicationPeers rp, Server server, String peerClusterId,
+        ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
         UUID clusterId, ReplicationEndpoint replicationEndpoint,
         WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
       throw new IOException("Failing deliberately");

http://git-wip-us.apache.org/repos/asf/hbase/blob/e504f1de/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
index aeab8b0..c6d9eef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
@@ -25,11 +25,10 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.junit.BeforeClass;
@@ -41,8 +40,9 @@ import org.junit.experimental.categories.Category;
  * ReplicationQueuesClientZkImpl. Also includes extra tests outside of those in
  * TestReplicationSourceManager that test ReplicationQueueZkImpl-specific behaviors.
  */
-@Category({ReplicationTests.class, MediumTests.class})
+@Category({ ReplicationTests.class, MediumTests.class })
 public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceManager {
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf = HBaseConfiguration.create();
@@ -58,16 +58,14 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan
   // Tests the naming convention of adopted queues for ReplicationQueuesZkImpl
   @Test
   public void testNodeFailoverDeadServerParsing() throws Exception {
-    final Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com");
-    ReplicationQueues repQueues =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, server,
-        server.getZooKeeper()));
-    repQueues.init(server.getServerName().toString());
+    Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com");
+    ReplicationQueueStorage queueStorage =
+        ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
     // populate some znodes in the peer znode
     files.add("log1");
     files.add("log2");
     for (String file : files) {
-      repQueues.addLog("1", file);
+      queueStorage.addWAL(server.getServerName(), "1", file);
     }
 
     // create 3 DummyServers
@@ -76,30 +74,22 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan
     Server s3 = new DummyServer("ec2-23-20-187-167.compute-1.amazonaws.com");
 
     // simulate three servers fail sequentially
-    ReplicationQueues rq1 =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1,
-        s1.getZooKeeper()));
-    rq1.init(s1.getServerName().toString());
-    String serverName = server.getServerName().getServerName();
-    List<String> unclaimed = rq1.getUnClaimedQueueIds(serverName);
-    rq1.claimQueue(serverName, unclaimed.get(0)).getSecond();
-    rq1.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
-    ReplicationQueues rq2 =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s2.getConfiguration(), s2,
-        s2.getZooKeeper()));
-    rq2.init(s2.getServerName().toString());
-    serverName = s1.getServerName().getServerName();
-    unclaimed = rq2.getUnClaimedQueueIds(serverName);
-    rq2.claimQueue(serverName, unclaimed.get(0)).getSecond();
-    rq2.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
-    ReplicationQueues rq3 =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s3.getConfiguration(), s3,
-        s3.getZooKeeper()));
-    rq3.init(s3.getServerName().toString());
-    serverName = s2.getServerName().getServerName();
-    unclaimed = rq3.getUnClaimedQueueIds(serverName);
-    String queue3 = rq3.claimQueue(serverName, unclaimed.get(0)).getFirst();
-    rq3.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
+    ServerName serverName = server.getServerName();
+    List<String> unclaimed = queueStorage.getAllQueues(serverName);
+    queueStorage.claimQueue(serverName, unclaimed.get(0), s1.getServerName());
+    queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
+
+    serverName = s1.getServerName();
+    unclaimed = queueStorage.getAllQueues(serverName);
+    queueStorage.claimQueue(serverName, unclaimed.get(0), s2.getServerName());
+    queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
+
+    serverName = s2.getServerName();
+    unclaimed = queueStorage.getAllQueues(serverName);
+    String queue3 =
+        queueStorage.claimQueue(serverName, unclaimed.get(0), s3.getServerName()).getFirst();
+    queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
+
     ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queue3);
     List<ServerName> result = replicationQueueInfo.getDeadRegionServers();
     // verify


[44/48] hbase git commit: HBASE-19623 Create replication endpoint asynchronously when adding a replication source

Posted by zh...@apache.org.
HBASE-19623 Create replication endpoint asynchronously when adding a replication source


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 7fe3a9046ef6c9ebd762c0c0d66bdaab9ccc88f7
Parents: 6b28956
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 2 13:25:58 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationPeer.java      |   8 ++
 .../hbase/replication/ReplicationPeers.java     |  18 +--
 .../replication/ZKReplicationPeerStorage.java   |   7 +-
 .../replication/TestReplicationStateBasic.java  |  20 +---
 .../TestZKReplicationPeerStorage.java           |  14 +--
 .../HBaseInterClusterReplicationEndpoint.java   |  17 ++-
 .../RecoveredReplicationSource.java             |  13 +--
 .../regionserver/ReplicationSource.java         | 110 +++++++++++--------
 .../ReplicationSourceInterface.java             |   8 +-
 .../regionserver/ReplicationSourceManager.java  |  47 +-------
 .../client/TestAsyncReplicationAdminApi.java    |   2 -
 .../replication/TestReplicationAdmin.java       |   2 -
 .../replication/ReplicationSourceDummy.java     |   7 +-
 .../replication/TestReplicationSource.java      |  27 +++--
 .../TestReplicationSourceManager.java           |   8 +-
 15 files changed, 127 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
index 4846018..2da3cce 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
@@ -54,6 +54,14 @@ public interface ReplicationPeer {
   PeerState getPeerState();
 
   /**
+   * Test whether the peer is enabled.
+   * @return {@code true} if enabled, otherwise {@code false}.
+   */
+  default boolean isPeerEnabled() {
+    return getPeerState() == PeerState.ENABLED;
+  }
+
+  /**
    * Get the peer config object
    * @return the ReplicationPeerConfig for this peer
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index 422801b..45940a5 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -86,21 +87,6 @@ public class ReplicationPeers {
   }
 
   /**
-   * Get the peer state for the specified connected remote slave cluster. The value might be read
-   * from cache, so it is recommended to use {@link #peerStorage } to read storage directly if
-   * reading the state after enabling or disabling it.
-   * @param peerId a short that identifies the cluster
-   * @return true if replication is enabled, false otherwise.
-   */
-  public boolean isPeerEnabled(String peerId) {
-    ReplicationPeer replicationPeer = this.peerCache.get(peerId);
-    if (replicationPeer == null) {
-      throw new IllegalArgumentException("Peer with id= " + peerId + " is not cached");
-    }
-    return replicationPeer.getPeerState() == PeerState.ENABLED;
-  }
-
-  /**
    * Returns the ReplicationPeerImpl for the specified cached peer. This ReplicationPeer will
    * continue to track changes to the Peer's state and config. This method returns null if no peer
    * has been cached with the given peerId.
@@ -117,7 +103,7 @@ public class ReplicationPeers {
    * @return a Set of Strings for peerIds
    */
   public Set<String> getAllPeerIds() {
-    return peerCache.keySet();
+    return Collections.unmodifiableSet(peerCache.keySet());
   }
 
   public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
index bf448e8..42d4b3f 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication;
 
 import java.util.Arrays;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -30,8 +29,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 
@@ -41,8 +38,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 @InterfaceAudience.Private
 class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements ReplicationPeerStorage {
 
-  private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationPeerStorage.class);
-
   public static final byte[] ENABLED_ZNODE_BYTES =
     toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
   public static final byte[] DISABLED_ZNODE_BYTES =
@@ -126,7 +121,7 @@ class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements Repli
   @Override
   public List<String> listPeerIds() throws ReplicationException {
     try {
-      return CollectionUtils.nullToEmpty(ZKUtil.listChildrenAndWatchThem(zookeeper, peersZNode));
+      return CollectionUtils.nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, peersZNode));
     } catch (KeeperException e) {
       throw new ReplicationException("Cannot get the list of peers", e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 07c6c15..f3eeccc 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -225,11 +225,6 @@ public abstract class TestReplicationStateBasic {
       fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
     } catch (ReplicationException e) {
     }
-    try {
-      rp.isPeerEnabled("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
 
     try {
       assertFalse(rp.addPeer("bogus"));
@@ -245,12 +240,6 @@ public abstract class TestReplicationStateBasic {
     rp.getPeerStorage().addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), true);
     assertNumberOfPeers(2);
 
-    // Test methods with a peer that is added but not connected
-    try {
-      rp.isPeerEnabled(ID_ONE);
-      fail("There are no connected peers, should have thrown an IllegalArgumentException");
-    } catch (IllegalArgumentException e) {
-    }
     assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationPeers
         .getPeerClusterConfiguration(rp.getPeerStorage().getPeerConfig(ID_ONE), rp.getConf())));
     rp.getPeerStorage().removePeer(ID_ONE);
@@ -261,7 +250,7 @@ public abstract class TestReplicationStateBasic {
     rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true);
     rp.addPeer(ID_ONE);
     assertNumberOfPeers(2);
-    assertTrue(rp.isPeerEnabled(ID_ONE));
+    assertTrue(rp.getPeer(ID_ONE).isPeerEnabled());
     rp.getPeerStorage().setPeerState(ID_ONE, false);
     // now we do not rely on zk watcher to trigger the state change so we need to trigger it
     // manually...
@@ -279,11 +268,6 @@ public abstract class TestReplicationStateBasic {
     // Disconnect peer
     rp.removePeer(ID_ONE);
     assertNumberOfPeers(2);
-    try {
-      rp.isPeerEnabled(ID_ONE);
-      fail("There are no connected peers, should have thrown an IllegalArgumentException");
-    } catch (IllegalArgumentException e) {
-    }
   }
 
   protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
@@ -292,7 +276,7 @@ public abstract class TestReplicationStateBasic {
       fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK");
     }
     while (true) {
-      if (status == rp.isPeerEnabled(peerId)) {
+      if (status == rp.getPeer(peerId).isPeerEnabled()) {
         return;
       }
       if (zkTimeoutCount < ZK_MAX_COUNT) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
index e8098c8..3eb11da 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
@@ -80,15 +80,11 @@ public class TestZKReplicationPeerStorage {
 
   private ReplicationPeerConfig getConfig(int seed) {
     Random rand = new Random(seed);
-    ReplicationPeerConfig config = new ReplicationPeerConfig();
-    config.setClusterKey(Long.toHexString(rand.nextLong()));
-    config.setReplicationEndpointImpl(Long.toHexString(rand.nextLong()));
-    config.setNamespaces(randNamespaces(rand));
-    config.setExcludeNamespaces(randNamespaces(rand));
-    config.setTableCFsMap(randTableCFs(rand));
-    config.setReplicateAllUserTables(rand.nextBoolean());
-    config.setBandwidth(rand.nextInt(1000));
-    return config;
+    return ReplicationPeerConfig.newBuilder().setClusterKey(Long.toHexString(rand.nextLong()))
+        .setReplicationEndpointImpl(Long.toHexString(rand.nextLong()))
+        .setNamespaces(randNamespaces(rand)).setExcludeNamespaces(randNamespaces(rand))
+        .setTableCFsMap(randTableCFs(rand)).setReplicateAllUserTables(rand.nextBoolean())
+        .setBandwidth(rand.nextInt(1000)).build();
   }
 
   private void assertSetEquals(Set<String> expected, Set<String> actual) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 40f59b7..1ef1198 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.SocketTimeoutException;
@@ -38,7 +36,6 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -47,22 +44,24 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
-import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
 
 /**
  * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint}
@@ -415,7 +414,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
   }
 
   protected boolean isPeerEnabled() {
-    return ctx.getReplicationPeer().getPeerState() == PeerState.ENABLED;
+    return ctx.getReplicationPeer().isPeerEnabled();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index e0c45d5..7bceb78 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -28,8 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
@@ -51,11 +50,11 @@ public class RecoveredReplicationSource extends ReplicationSource {
 
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
-      String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
-      WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
-    super.init(conf, fs, manager, queueStorage, replicationPeers, server, peerClusterZnode,
-      clusterId, replicationEndpoint, walFileLengthProvider, metrics);
+      ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
+      String peerClusterZnode, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
+      MetricsSource metrics) throws IOException {
+    super.init(conf, fs, manager, queueStorage, replicationPeer, server, peerClusterZnode,
+      clusterId, walFileLengthProvider, metrics);
     this.actualPeerId = this.replicationQueueInfo.getPeerId();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 1f4729b..0b44ba4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -38,14 +38,16 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
+import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
 import org.apache.hadoop.hbase.replication.ClusterMarkingEntryFilter;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
@@ -83,7 +85,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   // per group queue size, keep no more than this number of logs in each wal group
   protected int queueSizePerGroup;
   protected ReplicationQueueStorage queueStorage;
-  private ReplicationPeers replicationPeers;
+  private ReplicationPeer replicationPeer;
 
   protected Configuration conf;
   protected ReplicationQueueInfo replicationQueueInfo;
@@ -111,8 +113,10 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   private volatile boolean sourceRunning = false;
   // Metrics for this source
   private MetricsSource metrics;
-  //WARN threshold for the number of queued logs, defaults to 2
+  // WARN threshold for the number of queued logs, defaults to 2
   private int logQueueWarnThreshold;
+  // whether the replication endpoint has been initialized
+  private volatile boolean endpointInitialized = false;
   // ReplicationEndpoint which will handle the actual replication
   private ReplicationEndpoint replicationEndpoint;
   // A filter (or a chain of filters) for the WAL entries.
@@ -134,22 +138,19 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
 
   /**
    * Instantiation method used by region servers
-   *
    * @param conf configuration to use
    * @param fs file system to use
    * @param manager replication manager to ping to
    * @param server the server for this region server
    * @param peerClusterZnode the name of our znode
    * @param clusterId unique UUID for the cluster
-   * @param replicationEndpoint the replication endpoint implementation
    * @param metrics metrics for replication source
-   * @throws IOException
    */
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
-      String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
-      WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
+      ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
+      String peerClusterZnode, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
+      MetricsSource metrics) throws IOException {
     this.server = server;
     this.conf = HBaseConfiguration.create(conf);
     this.waitOnEndpointSeconds =
@@ -161,7 +162,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
         this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per
     this.queueSizePerGroup = this.conf.getInt("hbase.regionserver.maxlogs", 32);
     this.queueStorage = queueStorage;
-    this.replicationPeers = replicationPeers;
+    this.replicationPeer = replicationPeer;
     this.manager = manager;
     this.fs = fs;
     this.metrics = metrics;
@@ -172,7 +173,6 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     // ReplicationQueueInfo parses the peerId out of the znode for us
     this.peerId = this.replicationQueueInfo.getPeerId();
     this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
-    this.replicationEndpoint = replicationEndpoint;
 
     defaultBandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
     currentBandwidth = getCurrentBandwidth();
@@ -197,7 +197,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     if (queue == null) {
       queue = new PriorityBlockingQueue<>(queueSizePerGroup, new LogsComparator());
       queues.put(logPrefix, queue);
-      if (this.sourceRunning) {
+      if (this.isSourceActive() && this.endpointInitialized) {
         // new wal group observed after source startup, start a new worker thread to track it
         // notice: it's possible that log enqueued when this.running is set but worker thread
         // still not launched, so it's necessary to check workerThreads before start the worker
@@ -223,7 +223,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       // A peerId will not have "-" in its name, see HBASE-11394
       peerId = peerClusterZnode.split("-")[0];
     }
-    Map<TableName, List<String>> tableCFMap = replicationPeers.getPeer(peerId).getTableCFs();
+    Map<TableName, List<String>> tableCFMap = replicationPeer.getTableCFs();
     if (tableCFMap != null) {
       List<String> tableCfs = tableCFMap.get(tableName);
       if (tableCFMap.containsKey(tableName)
@@ -242,21 +242,59 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     }
   }
 
+  private void initAndStartReplicationEndpoint() throws Exception {
+    RegionServerCoprocessorHost rsServerHost = null;
+    TableDescriptors tableDescriptors = null;
+    if (server instanceof HRegionServer) {
+      rsServerHost = ((HRegionServer) server).getRegionServerCoprocessorHost();
+      tableDescriptors = ((HRegionServer) server).getTableDescriptors();
+    }
+    String replicationEndpointImpl = replicationPeer.getPeerConfig().getReplicationEndpointImpl();
+    if (replicationEndpointImpl == null) {
+      // Default to HBase inter-cluster replication endpoint
+      replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
+    }
+    replicationEndpoint =
+        Class.forName(replicationEndpointImpl).asSubclass(ReplicationEndpoint.class).newInstance();
+    if (rsServerHost != null) {
+      ReplicationEndpoint newReplicationEndPoint =
+          rsServerHost.postCreateReplicationEndPoint(replicationEndpoint);
+      if (newReplicationEndPoint != null) {
+        // Override the newly created endpoint from the hook with configured end point
+        replicationEndpoint = newReplicationEndPoint;
+      }
+    }
+    replicationEndpoint
+        .init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), fs, peerId,
+            clusterId, replicationPeer, metrics, tableDescriptors, server));
+    replicationEndpoint.start();
+    replicationEndpoint.awaitRunning(waitOnEndpointSeconds, TimeUnit.SECONDS);
+  }
+
   @Override
   public void run() {
     // mark we are running now
     this.sourceRunning = true;
-    try {
-      // start the endpoint, connect to the cluster
-      this.replicationEndpoint.start();
-      this.replicationEndpoint.awaitRunning(this.waitOnEndpointSeconds, TimeUnit.SECONDS);
-    } catch (Exception ex) {
-      LOG.warn("Error starting ReplicationEndpoint, exiting", ex);
-      uninitialize();
-      throw new RuntimeException(ex);
-    }
 
     int sleepMultiplier = 1;
+    while (this.isSourceActive()) {
+      try {
+        initAndStartReplicationEndpoint();
+        break;
+      } catch (Exception e) {
+        LOG.warn("Error starting ReplicationEndpoint, retrying", e);
+        if (replicationEndpoint != null) {
+          replicationEndpoint.stop();
+          replicationEndpoint = null;
+        }
+        if (sleepForRetries("Error starting ReplicationEndpoint", sleepMultiplier)) {
+          sleepMultiplier++;
+        }
+      }
+    }
+    this.endpointInitialized = true;
+
+    sleepMultiplier = 1;
     // delay this until we are in an asynchronous thread
     while (this.isSourceActive() && this.peerClusterId == null) {
       this.peerClusterId = replicationEndpoint.getPeerUUID();
@@ -289,8 +327,8 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
 
   private void initializeWALEntryFilter() {
     // get the WALEntryFilter from ReplicationEndpoint and add it to default filters
-    ArrayList<WALEntryFilter> filters = Lists.newArrayList(
-      (WALEntryFilter)new SystemTableWALEntryFilter());
+    ArrayList<WALEntryFilter> filters =
+      Lists.<WALEntryFilter> newArrayList(new SystemTableWALEntryFilter());
     WALEntryFilter filterFromEndpoint = this.replicationEndpoint.getWALEntryfilter();
     if (filterFromEndpoint != null) {
       filters.add(filterFromEndpoint);
@@ -310,7 +348,6 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       worker.startup(getUncaughtExceptionHandler());
       worker.setWALReader(startNewWALReader(worker.getName(), walGroupId, queue,
         worker.getStartPosition()));
-      workerThreads.put(walGroupId, worker);
     }
   }
 
@@ -371,25 +408,11 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   }
 
   private long getCurrentBandwidth() {
-    ReplicationPeer replicationPeer = this.replicationPeers.getPeer(peerId);
-    long peerBandwidth = replicationPeer != null ? replicationPeer.getPeerBandwidth() : 0;
+    long peerBandwidth = replicationPeer.getPeerBandwidth();
     // user can set peer bandwidth to 0 to use default bandwidth
     return peerBandwidth != 0 ? peerBandwidth : defaultBandwidth;
   }
 
-  private void uninitialize() {
-    LOG.debug("Source exiting " + this.peerId);
-    metrics.clear();
-    if (this.replicationEndpoint.isRunning() || this.replicationEndpoint.isStarting()) {
-      this.replicationEndpoint.stop();
-      try {
-        this.replicationEndpoint.awaitTerminated(this.waitOnEndpointSeconds, TimeUnit.SECONDS);
-      } catch (TimeoutException e) {
-        LOG.warn("Failed termination after " + this.waitOnEndpointSeconds + " seconds.");
-      }
-    }
-  }
-
   /**
    * Do the sleeping logic
    * @param msg Why we sleep
@@ -411,12 +434,11 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
 
   /**
    * check whether the peer is enabled or not
-   *
    * @return true if the peer is enabled, otherwise false
    */
   @Override
   public boolean isPeerEnabled() {
-    return this.replicationPeers.isPeerEnabled(this.peerId);
+    return replicationPeer.isPeerEnabled();
   }
 
   @Override
@@ -428,8 +450,8 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
         LOG.error("Unexpected exception in ReplicationSource", e);
       }
     };
-    Threads
-        .setDaemonThreadRunning(this, n + ".replicationSource," + this.peerClusterZnode, handler);
+    Threads.setDaemonThreadRunning(this, n + ".replicationSource," + this.peerClusterZnode,
+      handler);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index 4b9ed74..4f10c73 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -50,9 +50,9 @@ public interface ReplicationSourceInterface {
    * @param server the server for this region server
    */
   void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
-      String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
-      WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException;
+      ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
+      String peerClusterZnode, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
+      MetricsSource metrics) throws IOException;
 
   /**
    * Add a log to the list of logs to replicate

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 0e50bea..fc978be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -49,13 +49,9 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
-import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationListener;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
@@ -497,49 +493,14 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param peerId the id of the peer cluster
    * @return the created source
    */
-  private ReplicationSourceInterface getReplicationSource(String peerId, ReplicationPeer peer)
-      throws IOException {
-    RegionServerCoprocessorHost rsServerHost = null;
-    TableDescriptors tableDescriptors = null;
-    if (server instanceof HRegionServer) {
-      rsServerHost = ((HRegionServer) server).getRegionServerCoprocessorHost();
-      tableDescriptors = ((HRegionServer) server).getTableDescriptors();
-    }
-
+  private ReplicationSourceInterface getReplicationSource(String peerId,
+      ReplicationPeer replicationPeer) throws IOException {
     ReplicationSourceInterface src = ReplicationSourceFactory.create(conf, peerId);
 
-    ReplicationEndpoint replicationEndpoint = null;
-    try {
-      String replicationEndpointImpl = peer.getPeerConfig().getReplicationEndpointImpl();
-      if (replicationEndpointImpl == null) {
-        // Default to HBase inter-cluster replication endpoint
-        replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
-      }
-      replicationEndpoint = Class.forName(replicationEndpointImpl)
-          .asSubclass(ReplicationEndpoint.class).newInstance();
-      if (rsServerHost != null) {
-        ReplicationEndpoint newReplicationEndPoint =
-            rsServerHost.postCreateReplicationEndPoint(replicationEndpoint);
-        if (newReplicationEndPoint != null) {
-          // Override the newly created endpoint from the hook with configured end point
-          replicationEndpoint = newReplicationEndPoint;
-        }
-      }
-    } catch (Exception e) {
-      LOG.warn("Passed replication endpoint implementation throws errors" +
-        " while initializing ReplicationSource for peer: " + peerId, e);
-      throw new IOException(e);
-    }
-
     MetricsSource metrics = new MetricsSource(peerId);
     // init replication source
-    src.init(conf, fs, this, queueStorage, replicationPeers, server, peerId, clusterId,
-      replicationEndpoint, walFileLengthProvider, metrics);
-
-    // init replication endpoint
-    replicationEndpoint.init(new ReplicationEndpoint.Context(conf, peer.getConfiguration(), fs,
-        peerId, clusterId, peer, metrics, tableDescriptors, server));
-
+    src.init(conf, fs, this, queueStorage, replicationPeer, server, peerId, clusterId,
+      walFileLengthProvider, metrics);
     return src;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
index b28eaaf..5225add 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -70,7 +69,6 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 120000);
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
     TEST_UTIL.getConfiguration().setInt(START_LOG_ERRORS_AFTER_COUNT_KEY, 0);
-    TEST_UTIL.getConfiguration().setInt(ReadOnlyZKClient.RECOVERY_RETRY, 1);
     TEST_UTIL.startMiniCluster();
     ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/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 dbbc78f..f9629bd 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
@@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterR
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -84,7 +83,6 @@ public class TestReplicationAdmin {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
-    TEST_UTIL.getConfiguration().setInt(ReadOnlyZKClient.RECOVERY_RETRY, 1);
     TEST_UTIL.startMiniCluster();
     admin = new ReplicationAdmin(TEST_UTIL.getConfiguration());
     hbaseAdmin = TEST_UTIL.getAdmin();

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index 14c5e56..38ec598 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -48,9 +49,9 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
 
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
-      UUID clusterId, ReplicationEndpoint replicationEndpoint,
-      WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
+      ReplicationQueueStorage rq, ReplicationPeer rp, Server server, String peerClusterId,
+      UUID clusterId, WALFileLengthProvider walFileLengthProvider, MetricsSource metrics)
+      throws IOException {
     this.manager = manager;
     this.peerClusterId = peerClusterId;
     this.metrics = metrics;

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
index 48d8924..2194000 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
@@ -32,28 +32,28 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.replication.regionserver.Replication;
-import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
-import org.apache.hadoop.hbase.wal.WALProvider;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint;
+import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hadoop.hbase.wal.WALProvider;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -159,15 +159,14 @@ public class TestReplicationSource {
       }
     };
     replicationEndpoint.start();
-    ReplicationPeers mockPeers = Mockito.mock(ReplicationPeers.class);
     ReplicationPeer mockPeer = Mockito.mock(ReplicationPeer.class);
     Mockito.when(mockPeer.getPeerBandwidth()).thenReturn(0L);
     Configuration testConf = HBaseConfiguration.create();
     testConf.setInt("replication.source.maxretriesmultiplier", 1);
     ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class);
     Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong());
-    source.init(testConf, null, manager, null, mockPeers, null, "testPeer", null,
-      replicationEndpoint, p -> OptionalLong.empty(), null);
+    source.init(testConf, null, manager, null, mockPeer, null, "testPeer", null,
+      p -> OptionalLong.empty(), null);
     ExecutorService executor = Executors.newSingleThreadExecutor();
     Future<?> future = executor.submit(new Runnable() {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7fe3a904/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 dd56a76..f4d3901 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
@@ -62,8 +62,8 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 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.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@@ -712,9 +712,9 @@ public abstract class TestReplicationSourceManager {
 
     @Override
     public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-        ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
-        UUID clusterId, ReplicationEndpoint replicationEndpoint,
-        WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
+        ReplicationQueueStorage rq, ReplicationPeer rp, Server server, String peerClusterId,
+        UUID clusterId, WALFileLengthProvider walFileLengthProvider, MetricsSource metrics)
+        throws IOException {
       throw new IOException("Failing deliberately");
     }
   }


[11/48] hbase git commit: HBASE-19793 Minor improvements on Master/RS startup

Posted by zh...@apache.org.
HBASE-19793 Minor improvements on Master/RS startup


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 204a4539c4a3dbab45dd26ae79bd9650b123101b
Parents: 8dd4bf8
Author: zhangduo <zh...@apache.org>
Authored: Sun Jan 14 20:45:31 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 17:54:01 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java |  13 +-
 .../hadoop/hbase/master/MasterFileSystem.java   |   7 +-
 .../hbase/regionserver/HRegionServer.java       | 130 +++++++++----------
 .../master/assignment/MockMasterServices.java   |   2 +-
 4 files changed, 74 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/204a4539/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 971ff08..2683a6a 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
@@ -53,6 +53,7 @@ import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterId;
@@ -536,6 +537,11 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
+  @Override
+  protected String getUseThisHostnameInstead(Configuration conf) {
+    return conf.get(MASTER_HOSTNAME_KEY);
+  }
+
   // Main run loop. Calls through to the regionserver run loop AFTER becoming active Master; will
   // block in here until then.
   @Override
@@ -608,7 +614,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     masterJettyServer.addConnector(connector);
     masterJettyServer.setStopAtShutdown(true);
 
-    final String redirectHostname = shouldUseThisHostnameInstead() ? useThisHostnameInstead : null;
+    final String redirectHostname =
+        StringUtils.isBlank(useThisHostnameInstead) ? null : useThisHostnameInstead;
 
     final RedirectServlet redirect = new RedirectServlet(infoServer, redirectHostname);
     final WebAppContext context = new WebAppContext(null, "/", null, null, null, null, WebAppContext.NO_SESSIONS);
@@ -785,7 +792,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
     // Initialize the chunkCreator
     initializeMemStoreChunkCreator();
-    this.fileSystemManager = new MasterFileSystem(this);
+    this.fileSystemManager = new MasterFileSystem(conf);
     this.walManager = new MasterWalManager(this);
 
     // enable table descriptors cache
@@ -803,7 +810,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     ClusterId clusterId = fileSystemManager.getClusterId();
     status.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
     ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
-    this.clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
+    this.clusterId = clusterId.toString();
 
     this.serverManager = createServerManager(this);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/204a4539/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 8c2c9fd..a37fd4e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -98,11 +98,8 @@ public class MasterFileSystem {
 
   private boolean isSecurityEnabled;
 
-  private final MasterServices services;
-
-  public MasterFileSystem(MasterServices services) throws IOException {
-    this.conf = services.getConfiguration();
-    this.services = services;
+  public MasterFileSystem(Configuration conf) throws IOException {
+    this.conf = conf;
     // Set filesystem to be that of this.rootdir else we get complaints about
     // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
     // default localfs.  Presumption is that rootdir is fully-qualified before

http://git-wip-us.apache.org/repos/asf/hbase/blob/204a4539/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 8e91702..37ec595 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
@@ -28,6 +28,7 @@ import java.lang.reflect.Constructor;
 import java.net.BindException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -45,13 +46,12 @@ import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
 
 import org.apache.commons.lang3.RandomUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -168,13 +168,13 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
@@ -379,13 +379,13 @@ public class HRegionServer extends HasThread implements
   final AtomicBoolean online = new AtomicBoolean(false);
 
   // zookeeper connection and watcher
-  protected ZKWatcher zooKeeper;
+  protected final ZKWatcher zooKeeper;
 
   // master address tracker
-  private MasterAddressTracker masterAddressTracker;
+  private final MasterAddressTracker masterAddressTracker;
 
   // Cluster Status Tracker
-  protected ClusterStatusTracker clusterStatusTracker;
+  protected final ClusterStatusTracker clusterStatusTracker;
 
   // Log Splitting Worker
   private SplitLogWorker splitLogWorker;
@@ -518,7 +518,6 @@ public class HRegionServer extends HasThread implements
   private final boolean masterless;
   static final String MASTERLESS_CONFIG_NAME = "hbase.masterless";
 
-
   /**
    * Starts a HRegionServer at the default location
    */
@@ -565,23 +564,10 @@ public class HRegionServer extends HasThread implements
       this.stopped = false;
 
       rpcServices = createRpcServices();
-      if (this instanceof HMaster) {
-        useThisHostnameInstead = conf.get(MASTER_HOSTNAME_KEY);
-      } else {
-        useThisHostnameInstead = conf.get(RS_HOSTNAME_KEY);
-        if (conf.getBoolean(RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY, false)) {
-          if (shouldUseThisHostnameInstead()) {
-            String msg = RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY + " and " + RS_HOSTNAME_KEY +
-                " are mutually exclusive. Do not set " + RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY +
-                " to true while " + RS_HOSTNAME_KEY + " is used";
-            throw new IOException(msg);
-          } else {
-            useThisHostnameInstead = rpcServices.isa.getHostName();
-          }
-        }
-      }
-      String hostName = shouldUseThisHostnameInstead() ?
-          this.useThisHostnameInstead : this.rpcServices.isa.getHostName();
+      useThisHostnameInstead = getUseThisHostnameInstead(conf);
+      String hostName =
+          StringUtils.isBlank(useThisHostnameInstead) ? this.rpcServices.isa.getHostName()
+              : this.useThisHostnameInstead;
       serverName = ServerName.valueOf(hostName, this.rpcServices.isa.getPort(), this.startcode);
 
       rpcControllerFactory = RpcControllerFactory.instantiate(this.conf);
@@ -617,7 +603,6 @@ public class HRegionServer extends HasThread implements
         // Open connection to zookeeper and set primary watcher
         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.
         if (!this.masterless) {
           this.csm = new ZkCoordinatedStateManager(this);
@@ -627,7 +612,14 @@ public class HRegionServer extends HasThread implements
 
           clusterStatusTracker = new ClusterStatusTracker(zooKeeper, this);
           clusterStatusTracker.start();
+        } else {
+          masterAddressTracker = null;
+          clusterStatusTracker = null;
         }
+      } else {
+        zooKeeper = null;
+        masterAddressTracker = null;
+        clusterStatusTracker = null;
       }
       // This violates 'no starting stuff in Constructor' but Master depends on the below chore
       // and executor being created and takes a different startup route. Lots of overlap between HRS
@@ -646,6 +638,23 @@ public class HRegionServer extends HasThread implements
     }
   }
 
+  // HMaster should override this method to load the specific config for master
+  protected String getUseThisHostnameInstead(Configuration conf) throws IOException {
+    String hostname = conf.get(RS_HOSTNAME_KEY);
+    if (conf.getBoolean(RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY, false)) {
+      if (!StringUtils.isBlank(hostname)) {
+        String msg = RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY + " and " + RS_HOSTNAME_KEY +
+          " are mutually exclusive. Do not set " + RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY +
+          " to true while " + RS_HOSTNAME_KEY + " is used";
+        throw new IOException(msg);
+      } else {
+        return rpcServices.isa.getHostName();
+      }
+    } else {
+      return hostname;
+    }
+  }
+
   /**
    * If running on Windows, do windows-specific setup.
    */
@@ -695,13 +704,6 @@ public class HRegionServer extends HasThread implements
     return null;
   }
 
-  /*
-   * Returns true if configured hostname should be used
-   */
-  protected boolean shouldUseThisHostnameInstead() {
-    return useThisHostnameInstead != null && !useThisHostnameInstead.isEmpty();
-  }
-
   protected void login(UserProvider user, String host) throws IOException {
     user.login("hbase.regionserver.keytab.file",
       "hbase.regionserver.kerberos.principal", host);
@@ -804,17 +806,14 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * All initialization needed before we go register with Master.
-   * Do bare minimum. Do bulk of initializations AFTER we've connected to the Master.
+   * All initialization needed before we go register with Master.<br>
+   * Do bare minimum. Do bulk of initializations AFTER we've connected to the Master.<br>
    * In here we just put up the RpcServer, setup Connection, and ZooKeeper.
-   *
-   * @throws IOException
-   * @throws InterruptedException
    */
   private void preRegistrationInitialization() {
     try {
-      setupClusterConnection();
       initializeZooKeeper();
+      setupClusterConnection();
       // Setup RPC client for master communication
       this.rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
           this.rpcServices.isa.getAddress(), 0), clusterConnection.getConnectionMetrics());
@@ -827,18 +826,18 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * Bring up connection to zk ensemble and then wait until a master for this
-   * cluster and then after that, wait until cluster 'up' flag has been set.
-   * This is the order in which master does things.
+   * Bring up connection to zk ensemble and then wait until a master for this cluster and then after
+   * that, wait until cluster 'up' flag has been set. This is the order in which master does things.
+   * <p>
    * Finally open long-living server short-circuit connection.
-   * @throws IOException
-   * @throws InterruptedException
    */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE",
     justification="cluster Id znode read would give us correct response")
   private void initializeZooKeeper() throws IOException, InterruptedException {
     // Nothing to do in here if no Master in the mix.
-    if (this.masterless) return;
+    if (this.masterless) {
+      return;
+    }
 
     // Create the master address tracker, register with zk, and start it.  Then
     // block until a master is available.  No point in starting up if no master
@@ -849,17 +848,20 @@ public class HRegionServer extends HasThread implements
     // when ready.
     blockAndCheckIfStopped(this.clusterStatusTracker);
 
-    // Retrieve clusterId
-    // Since cluster status is now up
-    // ID should have already been set by HMaster
-    try {
-      clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
-      if (clusterId == null) {
-        this.abort("Cluster ID has not been set");
+    // If we are HMaster then the cluster id should have already been set.
+    if (clusterId == null) {
+      // Retrieve clusterId
+      // Since cluster status is now up
+      // ID should have already been set by HMaster
+      try {
+        clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
+        if (clusterId == null) {
+          this.abort("Cluster ID has not been set");
+        }
+        LOG.info("ClusterId : " + clusterId);
+      } catch (KeeperException e) {
+        this.abort("Failed to retrieve Cluster ID", e);
       }
-      LOG.info("ClusterId : "+clusterId);
-    } catch (KeeperException e) {
-      this.abort("Failed to retrieve Cluster ID",e);
     }
 
     // In case colocated master, wait here till it's active.
@@ -881,16 +883,6 @@ public class HRegionServer extends HasThread implements
     }
   }
 
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED",
-    justification="We don't care about the return")
-  private void doLatch(final CountDownLatch latch) throws InterruptedException {
-    if (latch != null) {
-      // Result is ignored intentionally but if I remove the below, findbugs complains (the
-      // above justification on this method doesn't seem to suppress it).
-      boolean result = latch.await(20, TimeUnit.SECONDS);
-    }
-  }
-
   /**
    * Utilty method to wait indefinitely on a znode availability while checking
    * if the region server is shut down
@@ -1461,14 +1453,14 @@ public class HRegionServer extends HasThread implements
           String hostnameFromMasterPOV = e.getValue();
           this.serverName = ServerName.valueOf(hostnameFromMasterPOV, rpcServices.isa.getPort(),
               this.startcode);
-          if (shouldUseThisHostnameInstead() &&
+          if (!StringUtils.isBlank(useThisHostnameInstead) &&
               !hostnameFromMasterPOV.equals(useThisHostnameInstead)) {
             String msg = "Master passed us a different hostname to use; was=" +
                 this.useThisHostnameInstead + ", but now=" + hostnameFromMasterPOV;
             LOG.error(msg);
             throw new IOException(msg);
           }
-          if (!shouldUseThisHostnameInstead() &&
+          if (StringUtils.isBlank(useThisHostnameInstead) &&
               !hostnameFromMasterPOV.equals(rpcServices.isa.getHostName())) {
             String msg = "Master passed us a different hostname to use; was=" +
                 rpcServices.isa.getHostName() + ", but now=" + hostnameFromMasterPOV;
@@ -1685,7 +1677,7 @@ public class HRegionServer extends HasThread implements
     CompactionChecker(final HRegionServer h, final int sleepTime, final Stoppable stopper) {
       super("CompactionChecker", stopper, sleepTime);
       this.instance = h;
-      LOG.info(this.getName() + " runs every " + StringUtils.formatTime(sleepTime));
+      LOG.info(this.getName() + " runs every " + Duration.ofMillis(sleepTime));
 
       /* MajorCompactPriority is configurable.
        * If not set, the compaction will use default priority.
@@ -2375,7 +2367,7 @@ public class HRegionServer extends HasThread implements
     // Do our best to report our abort to the master, but this may not work
     try {
       if (cause != null) {
-        msg += "\nCause:\n" + StringUtils.stringifyException(cause);
+        msg += "\nCause:\n" + Throwables.getStackTraceAsString(cause);
       }
       // Report to the master but only if we have already registered with the master.
       if (rssStub != null && this.serverName != null) {
@@ -2603,7 +2595,7 @@ public class HRegionServer extends HasThread implements
       long now = EnvironmentEdgeManager.currentTime();
       int port = rpcServices.isa.getPort();
       RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
-      if (shouldUseThisHostnameInstead()) {
+      if (!StringUtils.isBlank(useThisHostnameInstead)) {
         request.setUseThisHostnameInstead(useThisHostnameInstead);
       }
       request.setPort(port);

http://git-wip-us.apache.org/repos/asf/hbase/blob/204a4539/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
index 45d5b08..83fafff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
@@ -103,7 +103,7 @@ public class MockMasterServices extends MockNoopMasterServices {
     super(conf);
     this.regionsToRegionServers = regionsToRegionServers;
     Superusers.initialize(conf);
-    this.fileSystemManager = new MasterFileSystem(this);
+    this.fileSystemManager = new MasterFileSystem(conf);
     this.walManager = new MasterWalManager(this);
     // Mock an AM.
     this.assignmentManager = new AssignmentManager(this, new MockRegionStateStore(this)) {


[42/48] hbase git commit: HBASE-19707 Race in start and terminate of a replication source after we async start replicatione endpoint

Posted by zh...@apache.org.
HBASE-19707 Race in start and terminate of a replication source after we async start replicatione endpoint


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 01035d102eadd597b69d79765945a8897ed0cb9f
Parents: 1f0bfe0
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 5 18:28:44 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../RecoveredReplicationSource.java             |  16 +-
 .../regionserver/ReplicationSource.java         | 202 ++++++++++---------
 2 files changed, 116 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/01035d10/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index 1be9a88..3cae0f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -68,7 +68,7 @@ public class RecoveredReplicationSource extends ReplicationSource {
       LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId);
     } else {
       LOG.debug("Starting up worker for wal group " + walGroupId);
-      worker.startup(getUncaughtExceptionHandler());
+      worker.startup(this::uncaughtException);
       worker.setWALReader(
         startNewWALReader(worker.getName(), walGroupId, queue, worker.getStartPosition()));
       workerThreads.put(walGroupId, worker);
@@ -76,13 +76,13 @@ public class RecoveredReplicationSource extends ReplicationSource {
   }
 
   @Override
-  protected ReplicationSourceWALReader startNewWALReader(String threadName,
-      String walGroupId, PriorityBlockingQueue<Path> queue, long startPosition) {
-    ReplicationSourceWALReader walReader = new RecoveredReplicationSourceWALReader(fs,
-        conf, queue, startPosition, walEntryFilter, this);
-    Threads.setDaemonThreadRunning(walReader, threadName
-        + ".replicationSource.replicationWALReaderThread." + walGroupId + "," + queueId,
-      getUncaughtExceptionHandler());
+  protected ReplicationSourceWALReader startNewWALReader(String threadName, String walGroupId,
+      PriorityBlockingQueue<Path> queue, long startPosition) {
+    ReplicationSourceWALReader walReader =
+      new RecoveredReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this);
+    Threads.setDaemonThreadRunning(walReader,
+      threadName + ".replicationSource.replicationWALReaderThread." + walGroupId + "," + queueId,
+      this::uncaughtException);
     return walReader;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/01035d10/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 6b622ee..923d893 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -76,7 +76,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
  * </p>
  */
 @InterfaceAudience.Private
-public class ReplicationSource extends Thread implements ReplicationSourceInterface {
+public class ReplicationSource implements ReplicationSourceInterface {
 
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationSource.class);
   // Queues of logs to process, entry in format of walGroupId->queue,
@@ -115,10 +115,8 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   private MetricsSource metrics;
   // WARN threshold for the number of queued logs, defaults to 2
   private int logQueueWarnThreshold;
-  // whether the replication endpoint has been initialized
-  private volatile boolean endpointInitialized = false;
   // ReplicationEndpoint which will handle the actual replication
-  private ReplicationEndpoint replicationEndpoint;
+  private volatile ReplicationEndpoint replicationEndpoint;
   // A filter (or a chain of filters) for the WAL entries.
   protected WALEntryFilter walEntryFilter;
   // throttler
@@ -136,6 +134,8 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   public static final int DEFAULT_WAIT_ON_ENDPOINT_SECONDS = 30;
   private int waitOnEndpointSeconds = -1;
 
+  private Thread initThread;
+
   /**
    * Instantiation method used by region servers
    * @param conf configuration to use
@@ -197,7 +197,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     if (queue == null) {
       queue = new PriorityBlockingQueue<>(queueSizePerGroup, new LogsComparator());
       queues.put(logPrefix, queue);
-      if (this.isSourceActive() && this.endpointInitialized) {
+      if (this.isSourceActive() && this.replicationEndpoint != null) {
         // new wal group observed after source startup, start a new worker thread to track it
         // notice: it's possible that log enqueued when this.running is set but worker thread
         // still not launched, so it's necessary to check workerThreads before start the worker
@@ -236,28 +236,36 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     }
   }
 
-  private void initAndStartReplicationEndpoint() throws Exception {
+  private ReplicationEndpoint createReplicationEndpoint()
+      throws InstantiationException, IllegalAccessException, ClassNotFoundException, IOException {
     RegionServerCoprocessorHost rsServerHost = null;
-    TableDescriptors tableDescriptors = null;
     if (server instanceof HRegionServer) {
       rsServerHost = ((HRegionServer) server).getRegionServerCoprocessorHost();
-      tableDescriptors = ((HRegionServer) server).getTableDescriptors();
     }
     String replicationEndpointImpl = replicationPeer.getPeerConfig().getReplicationEndpointImpl();
     if (replicationEndpointImpl == null) {
       // Default to HBase inter-cluster replication endpoint
       replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
     }
-    replicationEndpoint =
-        Class.forName(replicationEndpointImpl).asSubclass(ReplicationEndpoint.class).newInstance();
+    ReplicationEndpoint replicationEndpoint =
+      Class.forName(replicationEndpointImpl).asSubclass(ReplicationEndpoint.class).newInstance();
     if (rsServerHost != null) {
       ReplicationEndpoint newReplicationEndPoint =
-          rsServerHost.postCreateReplicationEndPoint(replicationEndpoint);
+        rsServerHost.postCreateReplicationEndPoint(replicationEndpoint);
       if (newReplicationEndPoint != null) {
         // Override the newly created endpoint from the hook with configured end point
         replicationEndpoint = newReplicationEndPoint;
       }
     }
+    return replicationEndpoint;
+  }
+
+  private void initAndStartReplicationEndpoint(ReplicationEndpoint replicationEndpoint)
+      throws IOException, TimeoutException {
+    TableDescriptors tableDescriptors = null;
+    if (server instanceof HRegionServer) {
+      tableDescriptors = ((HRegionServer) server).getTableDescriptors();
+    }
     replicationEndpoint
         .init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), fs, peerId,
             clusterId, replicationPeer, metrics, tableDescriptors, server));
@@ -265,60 +273,6 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     replicationEndpoint.awaitRunning(waitOnEndpointSeconds, TimeUnit.SECONDS);
   }
 
-  @Override
-  public void run() {
-    // mark we are running now
-    this.sourceRunning = true;
-
-    int sleepMultiplier = 1;
-    while (this.isSourceActive()) {
-      try {
-        initAndStartReplicationEndpoint();
-        break;
-      } catch (Exception e) {
-        LOG.warn("Error starting ReplicationEndpoint, retrying", e);
-        if (replicationEndpoint != null) {
-          replicationEndpoint.stop();
-          replicationEndpoint = null;
-        }
-        if (sleepForRetries("Error starting ReplicationEndpoint", sleepMultiplier)) {
-          sleepMultiplier++;
-        }
-      }
-    }
-    this.endpointInitialized = true;
-
-    sleepMultiplier = 1;
-    // delay this until we are in an asynchronous thread
-    while (this.isSourceActive() && this.peerClusterId == null) {
-      this.peerClusterId = replicationEndpoint.getPeerUUID();
-      if (this.isSourceActive() && this.peerClusterId == null) {
-        if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
-          sleepMultiplier++;
-        }
-      }
-    }
-
-    // In rare case, zookeeper setting may be messed up. That leads to the incorrect
-    // peerClusterId value, which is the same as the source clusterId
-    if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) {
-      this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
-          + peerClusterId + " which is not allowed by ReplicationEndpoint:"
-          + replicationEndpoint.getClass().getName(), null, false);
-      this.manager.removeSource(this);
-      return;
-    }
-    LOG.info("Replicating " + clusterId + " -> " + peerClusterId);
-
-    initializeWALEntryFilter();
-    // start workers
-    for (Map.Entry<String, PriorityBlockingQueue<Path>> entry : queues.entrySet()) {
-      String walGroupId = entry.getKey();
-      PriorityBlockingQueue<Path> queue = entry.getValue();
-      tryStartNewShipper(walGroupId, queue);
-    }
-  }
-
   private void initializeWALEntryFilter() {
     // get the WALEntryFilter from ReplicationEndpoint and add it to default filters
     ArrayList<WALEntryFilter> filters =
@@ -332,37 +286,31 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   }
 
   protected void tryStartNewShipper(String walGroupId, PriorityBlockingQueue<Path> queue) {
-    final ReplicationSourceShipper worker = new ReplicationSourceShipper(conf,
-        walGroupId, queue, this);
+    ReplicationSourceShipper worker = new ReplicationSourceShipper(conf, walGroupId, queue, this);
     ReplicationSourceShipper extant = workerThreads.putIfAbsent(walGroupId, worker);
     if (extant != null) {
       LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId);
     } else {
       LOG.debug("Starting up worker for wal group " + walGroupId);
-      worker.startup(getUncaughtExceptionHandler());
-      worker.setWALReader(startNewWALReader(worker.getName(), walGroupId, queue,
-        worker.getStartPosition()));
+      worker.startup(this::uncaughtException);
+      worker.setWALReader(
+        startNewWALReader(worker.getName(), walGroupId, queue, worker.getStartPosition()));
     }
   }
 
   protected ReplicationSourceWALReader startNewWALReader(String threadName, String walGroupId,
       PriorityBlockingQueue<Path> queue, long startPosition) {
     ReplicationSourceWALReader walReader =
-        new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this);
+      new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this);
     return (ReplicationSourceWALReader) Threads.setDaemonThreadRunning(walReader,
       threadName + ".replicationSource.wal-reader." + walGroupId + "," + queueId,
-      getUncaughtExceptionHandler());
+      this::uncaughtException);
   }
 
-  public Thread.UncaughtExceptionHandler getUncaughtExceptionHandler() {
-    return new Thread.UncaughtExceptionHandler() {
-      @Override
-      public void uncaughtException(final Thread t, final Throwable e) {
-        RSRpcServices.exitIfOOME(e);
-        LOG.error("Unexpected exception in " + t.getName() + " currentPath=" + getCurrentPath(), e);
-        server.stop("Unexpected exception in " + t.getName());
-      }
-    };
+  protected final void uncaughtException(Thread t, Throwable e) {
+    RSRpcServices.exitIfOOME(e);
+    LOG.error("Unexpected exception in " + t.getName() + " currentPath=" + getCurrentPath(), e);
+    server.abort("Unexpected exception in " + t.getName(), e);
   }
 
   @Override
@@ -435,17 +383,76 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     return replicationPeer.isPeerEnabled();
   }
 
+  private void initialize() {
+    int sleepMultiplier = 1;
+    while (this.isSourceActive()) {
+      ReplicationEndpoint replicationEndpoint;
+      try {
+        replicationEndpoint = createReplicationEndpoint();
+      } catch (Exception e) {
+        LOG.warn("error creating ReplicationEndpoint, retry", e);
+        if (sleepForRetries("Error creating ReplicationEndpoint", sleepMultiplier)) {
+          sleepMultiplier++;
+        }
+        continue;
+      }
+
+      try {
+        initAndStartReplicationEndpoint(replicationEndpoint);
+        this.replicationEndpoint = replicationEndpoint;
+        break;
+      } catch (Exception e) {
+        LOG.warn("Error starting ReplicationEndpoint, retry", e);
+        replicationEndpoint.stop();
+        if (sleepForRetries("Error starting ReplicationEndpoint", sleepMultiplier)) {
+          sleepMultiplier++;
+        }
+      }
+    }
+
+    if (!this.isSourceActive()) {
+      return;
+    }
+
+    sleepMultiplier = 1;
+    // delay this until we are in an asynchronous thread
+    while (this.isSourceActive() && this.peerClusterId == null) {
+      this.peerClusterId = replicationEndpoint.getPeerUUID();
+      if (this.isSourceActive() && this.peerClusterId == null) {
+        if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
+          sleepMultiplier++;
+        }
+      }
+    }
+
+    // In rare case, zookeeper setting may be messed up. That leads to the incorrect
+    // peerClusterId value, which is the same as the source clusterId
+    if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) {
+      this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
+          + peerClusterId + " which is not allowed by ReplicationEndpoint:"
+          + replicationEndpoint.getClass().getName(), null, false);
+      this.manager.removeSource(this);
+      return;
+    }
+    LOG.info("Replicating " + clusterId + " -> " + peerClusterId);
+
+    initializeWALEntryFilter();
+    // start workers
+    for (Map.Entry<String, PriorityBlockingQueue<Path>> entry : queues.entrySet()) {
+      String walGroupId = entry.getKey();
+      PriorityBlockingQueue<Path> queue = entry.getValue();
+      tryStartNewShipper(walGroupId, queue);
+    }
+  }
+
   @Override
   public void startup() {
-    String n = Thread.currentThread().getName();
-    Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() {
-      @Override
-      public void uncaughtException(final Thread t, final Throwable e) {
-        LOG.error("Unexpected exception in ReplicationSource", e);
-      }
-    };
-    Threads.setDaemonThreadRunning(this, n + ".replicationSource," + this.queueId,
-      handler);
+    // mark we are running now
+    this.sourceRunning = true;
+    initThread = new Thread(this::initialize);
+    Threads.setDaemonThreadRunning(initThread,
+      Thread.currentThread().getName() + ".replicationSource," + this.queueId,
+      this::uncaughtException);
   }
 
   @Override
@@ -466,6 +473,13 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
         cause);
     }
     this.sourceRunning = false;
+    if (initThread != null && Thread.currentThread() != initThread) {
+      // This usually won't happen but anyway, let's wait until the initialization thread exits.
+      // And notice that we may call terminate directly from the initThread so here we need to
+      // avoid join on ourselves.
+      initThread.interrupt();
+      Threads.shutdown(initThread, this.sleepForRetries);
+    }
     Collection<ReplicationSourceShipper> workers = workerThreads.values();
     for (ReplicationSourceShipper worker : workers) {
       worker.stopWorker();
@@ -482,11 +496,11 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       }
       if (this.replicationEndpoint != null) {
         try {
-          this.replicationEndpoint.awaitTerminated(sleepForRetries * maxRetriesMultiplier, TimeUnit.MILLISECONDS);
+          this.replicationEndpoint.awaitTerminated(sleepForRetries * maxRetriesMultiplier,
+            TimeUnit.MILLISECONDS);
         } catch (TimeoutException te) {
-          LOG.warn("Got exception while waiting for endpoint to shutdown for replication source :"
-              + this.queueId,
-            te);
+          LOG.warn("Got exception while waiting for endpoint to shutdown for replication source :" +
+            this.queueId, te);
         }
       }
     }


[08/48] hbase git commit: HBASE-19772 Do not close connection to zk when there are still pending request in ReadOnlyZKClient

Posted by zh...@apache.org.
HBASE-19772 Do not close connection to zk when there are still pending request in ReadOnlyZKClient


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 842f794a627dcbfc72a78f1742645271d4d4bca6
Parents: f91589d
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 12 09:43:56 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Jan 14 17:08:30 2018 +0800

----------------------------------------------------------------------
 .../hbase/zookeeper/ReadOnlyZKClient.java       | 79 ++++++++++----------
 .../hbase/zookeeper/TestReadOnlyZKClient.java   | 71 +++++++++++++-----
 2 files changed, 92 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/842f794a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
index 82c011b..ad70740 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
@@ -29,9 +29,8 @@ import java.util.concurrent.DelayQueue;
 import java.util.concurrent.Delayed;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
@@ -39,6 +38,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -114,7 +114,10 @@ public final class ReadOnlyZKClient implements Closeable {
 
   private final AtomicBoolean closed = new AtomicBoolean(false);
 
-  private ZooKeeper zookeeper;
+  @VisibleForTesting
+  ZooKeeper zookeeper;
+
+  private int pendingRequests = 0;
 
   private String getId() {
     return String.format("0x%08x", System.identityHashCode(this));
@@ -127,12 +130,12 @@ public final class ReadOnlyZKClient implements Closeable {
     this.retryIntervalMs =
         conf.getInt(RECOVERY_RETRY_INTERVAL_MILLIS, DEFAULT_RECOVERY_RETRY_INTERVAL_MILLIS);
     this.keepAliveTimeMs = conf.getInt(KEEPALIVE_MILLIS, DEFAULT_KEEPALIVE_MILLIS);
-    LOG.info("Start read only zookeeper connection " + getId() + " to " + connectString +
-        ", session timeout " + sessionTimeoutMs + " ms, retries " + maxRetries +
-        ", retry interval " + retryIntervalMs + " ms, keep alive " + keepAliveTimeMs + " ms");
-    Thread t = new Thread(this::run, "ReadOnlyZKClient");
-    t.setDaemon(true);
-    t.start();
+    LOG.info(
+      "Start read only zookeeper connection {} to {}, " + "session timeout {} ms, retries {}, " +
+        "retry interval {} ms, keep alive {} ms",
+      getId(), connectString, sessionTimeoutMs, maxRetries, retryIntervalMs, keepAliveTimeMs);
+    Threads.setDaemonThreadRunning(new Thread(this::run),
+      "ReadOnlyZKClient-" + connectString + "@" + getId());
   }
 
   private abstract class ZKTask<T> extends Task {
@@ -156,6 +159,7 @@ public final class ReadOnlyZKClient implements Closeable {
 
         @Override
         public void exec(ZooKeeper alwaysNull) {
+          pendingRequests--;
           Code code = Code.get(rc);
           if (code == Code.OK) {
             future.complete(ret);
@@ -169,19 +173,19 @@ public final class ReadOnlyZKClient implements Closeable {
             future.completeExceptionally(KeeperException.create(code, path));
           } else {
             if (code == Code.SESSIONEXPIRED) {
-              LOG.warn(getId() + " session expired, close and reconnect");
+              LOG.warn("{} to {} session expired, close and reconnect", getId(), connectString);
               try {
                 zk.close();
               } catch (InterruptedException e) {
               }
             }
             if (ZKTask.this.delay(retryIntervalMs, maxRetries)) {
-              LOG.warn(getId() + " failed for " + operationType + " of " + path + ", code = " +
-                  code + ", retries = " + ZKTask.this.retries);
+              LOG.warn("{} to {} failed for {} of {}, code = {}, retries = {}", getId(),
+                connectString, operationType, path, code, ZKTask.this.retries);
               tasks.add(ZKTask.this);
             } else {
-              LOG.warn(getId() + " failed for " + operationType + " of " + path + ", code = " +
-                  code + ", retries = " + ZKTask.this.retries + ", give up");
+              LOG.warn("{} to {} failed for {} of {}, code = {}, retries = {}, give up", getId(),
+                connectString, operationType, path, code, ZKTask.this.retries);
               future.completeExceptionally(KeeperException.create(code, path));
             }
           }
@@ -205,6 +209,14 @@ public final class ReadOnlyZKClient implements Closeable {
       return true;
     }
 
+    protected abstract void doExec(ZooKeeper zk);
+
+    @Override
+    public final void exec(ZooKeeper zk) {
+      pendingRequests++;
+      doExec(zk);
+    }
+
     public boolean delay(long intervalMs, int maxRetries) {
       if (retries >= maxRetries) {
         return false;
@@ -217,14 +229,12 @@ public final class ReadOnlyZKClient implements Closeable {
     @Override
     public void connectFailed(IOException e) {
       if (delay(retryIntervalMs, maxRetries)) {
-        LOG.warn(getId() + " failed to connect to zk for " + operationType + " of " + path +
-            ", retries = " + retries,
-          e);
+        LOG.warn("{} to {} failed to connect to zk fo {} of {}, retries = {}", getId(),
+          connectString, operationType, path, retries, e);
         tasks.add(this);
       } else {
-        LOG.warn(getId() + " failed to connect to zk for " + operationType + " of " + path +
-            ", retries = " + retries + ", give up",
-          e);
+        LOG.warn("{} to {} failed to connect to zk fo {} of {}, retries = {}, give up", getId(),
+          connectString, operationType, path, retries, e);
         future.completeExceptionally(e);
       }
     }
@@ -249,7 +259,7 @@ public final class ReadOnlyZKClient implements Closeable {
     tasks.add(new ZKTask<byte[]>(path, future, "get") {
 
       @Override
-      public void exec(ZooKeeper zk) {
+      protected void doExec(ZooKeeper zk) {
         zk.getData(path, false,
             (rc, path, ctx, data, stat) -> onComplete(zk, rc, data, true), null);
       }
@@ -265,7 +275,7 @@ public final class ReadOnlyZKClient implements Closeable {
     tasks.add(new ZKTask<Stat>(path, future, "exists") {
 
       @Override
-      public void exec(ZooKeeper zk) {
+      protected void doExec(ZooKeeper zk) {
         zk.exists(path, false, (rc, path, ctx, stat) -> onComplete(zk, rc, stat, false), null);
       }
     });
@@ -286,16 +296,6 @@ public final class ReadOnlyZKClient implements Closeable {
     // may be closed when session expired
     if (zookeeper == null || !zookeeper.getState().isAlive()) {
       zookeeper = new ZooKeeper(connectString, sessionTimeoutMs, e -> {});
-      int timeout = 10000;
-      try {
-        // Before returning, try and ensure we are connected. Don't wait long in case
-        // we are trying to connect to a cluster that is down. If we fail to connect,
-        // just catch the exception and carry-on. The first usage will fail and we'll
-        // cleanup.
-        zookeeper = ZooKeeperHelper.ensureConnectedZooKeeper(zookeeper, timeout);
-      } catch (ZooKeeperConnectionException e) {
-        LOG.warn("Failed connecting after waiting " + timeout + "ms; " + zookeeper);
-      }
     }
     return zookeeper;
   }
@@ -311,9 +311,11 @@ public final class ReadOnlyZKClient implements Closeable {
       if (task == CLOSE) {
         break;
       }
-      if (task == null) {
-        LOG.info(getId() + " no activities for " + keepAliveTimeMs +
-            " ms, close active connection. Will reconnect next time when there are new requests.");
+      if (task == null && pendingRequests == 0) {
+        LOG.debug(
+          "{} to {} no activities for {} ms, close active connection. " +
+            "Will reconnect next time when there are new requests",
+          getId(), connectString, keepAliveTimeMs);
         closeZk();
         continue;
       }
@@ -339,17 +341,12 @@ public final class ReadOnlyZKClient implements Closeable {
   @Override
   public void close() {
     if (closed.compareAndSet(false, true)) {
-      LOG.info("Close zookeeper connection " + getId() + " to " + connectString);
+      LOG.info("Close zookeeper connection {} to {}", getId(), connectString);
       tasks.add(CLOSE);
     }
   }
 
   @VisibleForTesting
-  ZooKeeper getZooKeeper() {
-    return zookeeper;
-  }
-
-  @VisibleForTesting
   public String getConnectString() {
     return connectString;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/842f794a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java
index c478121..211e776 100644
--- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -27,26 +27,36 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ZKTests;
+import org.apache.zookeeper.AsyncCallback.StatCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @Category({ ZKTests.class, MediumTests.class })
 public class TestReadOnlyZKClient {
@@ -67,8 +77,7 @@ public class TestReadOnlyZKClient {
   public static void setUp() throws Exception {
     PORT = UTIL.startMiniZKCluster().getClientPort();
 
-    ZooKeeper zk = ZooKeeperHelper.
-        getConnectedZooKeeper("localhost:" + PORT, 10000);
+    ZooKeeper zk = ZooKeeperHelper.getConnectedZooKeeper("localhost:" + PORT, 10000);
     DATA = new byte[10];
     ThreadLocalRandom.current().nextBytes(DATA);
     zk.create(PATH, DATA, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
@@ -83,7 +92,7 @@ public class TestReadOnlyZKClient {
     conf.setInt(ReadOnlyZKClient.KEEPALIVE_MILLIS, 3000);
     RO_ZK = new ReadOnlyZKClient(conf);
     // only connect when necessary
-    assertNull(RO_ZK.getZooKeeper());
+    assertNull(RO_ZK.zookeeper);
   }
 
   @AfterClass
@@ -93,17 +102,13 @@ public class TestReadOnlyZKClient {
     UTIL.cleanupTestDir();
   }
 
-  @Test
-  public void testGetAndExists() throws Exception {
-    assertArrayEquals(DATA, RO_ZK.get(PATH).get());
-    assertEquals(CHILDREN, RO_ZK.exists(PATH).get().getNumChildren());
-    assertNotNull(RO_ZK.getZooKeeper());
+  private void waitForIdleConnectionClosed() throws Exception {
     // The zookeeper client should be closed finally after the keep alive time elapsed
     UTIL.waitFor(10000, new ExplainingPredicate<Exception>() {
 
       @Override
       public boolean evaluate() throws Exception {
-        return RO_ZK.getZooKeeper() == null;
+        return RO_ZK.zookeeper == null;
       }
 
       @Override
@@ -114,6 +119,14 @@ public class TestReadOnlyZKClient {
   }
 
   @Test
+  public void testGetAndExists() throws Exception {
+    assertArrayEquals(DATA, RO_ZK.get(PATH).get());
+    assertEquals(CHILDREN, RO_ZK.exists(PATH).get().getNumChildren());
+    assertNotNull(RO_ZK.zookeeper);
+    waitForIdleConnectionClosed();
+  }
+
+  @Test
   public void testNoNode() throws InterruptedException, ExecutionException {
     String pathNotExists = PATH + "_whatever";
     try {
@@ -132,15 +145,39 @@ public class TestReadOnlyZKClient {
   @Test
   public void testSessionExpire() throws Exception {
     assertArrayEquals(DATA, RO_ZK.get(PATH).get());
-    ZooKeeper zk = RO_ZK.getZooKeeper();
+    ZooKeeper zk = RO_ZK.zookeeper;
     long sessionId = zk.getSessionId();
     UTIL.getZkCluster().getZooKeeperServers().get(0).closeSession(sessionId);
     // should not reach keep alive so still the same instance
-    assertSame(zk, RO_ZK.getZooKeeper());
-    byte [] got = RO_ZK.get(PATH).get();
+    assertSame(zk, RO_ZK.zookeeper);
+    byte[] got = RO_ZK.get(PATH).get();
     assertArrayEquals(DATA, got);
-    assertNotNull(RO_ZK.getZooKeeper());
-    assertNotSame(zk, RO_ZK.getZooKeeper());
-    assertNotEquals(sessionId, RO_ZK.getZooKeeper().getSessionId());
+    assertNotNull(RO_ZK.zookeeper);
+    assertNotSame(zk, RO_ZK.zookeeper);
+    assertNotEquals(sessionId, RO_ZK.zookeeper.getSessionId());
+  }
+
+  @Test
+  public void testNotCloseZkWhenPending() throws Exception {
+    assertArrayEquals(DATA, RO_ZK.get(PATH).get());
+    ZooKeeper mockedZK = spy(RO_ZK.zookeeper);
+    CountDownLatch latch = new CountDownLatch(1);
+    doAnswer(new Answer<Object>() {
+
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        latch.await();
+        return invocation.callRealMethod();
+      }
+    }).when(mockedZK).exists(anyString(), anyBoolean(), any(StatCallback.class), any());
+    RO_ZK.zookeeper = mockedZK;
+    CompletableFuture<Stat> future = RO_ZK.exists(PATH);
+    // 2 * keep alive time to ensure that we will not close the zk when there are pending requests
+    Thread.sleep(6000);
+    assertNotNull(RO_ZK.zookeeper);
+    latch.countDown();
+    assertEquals(CHILDREN, future.get().getNumChildren());
+    // now we will close the idle connection.
+    waitForIdleConnectionClosed();
   }
 }


[20/48] hbase git commit: HBASE-19536 Client side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19536 Client side changes for moving peer modification from zk watcher to procedure

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: eb9f8d5da138386b4d78557d962c125b91edaac7
Parents: 1966abf
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 19 15:50:57 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  87 ++++++++++-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 149 ++++++++++++++-----
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  82 +++++-----
 3 files changed, 238 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eb9f8d5d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index f61b32e..6729473 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -2466,7 +2466,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Add a new replication peer for replicating data to slave cluster.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig configuration for the replication slave cluster
+   * @param peerConfig configuration for the replication peer
    * @throws IOException if a remote or network exception occurs
    */
   default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig)
@@ -2477,7 +2477,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Add a new replication peer for replicating data to slave cluster.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig configuration for the replication slave cluster
+   * @param peerConfig configuration for the replication peer
    * @param enabled peer state, true if ENABLED and false if DISABLED
    * @throws IOException if a remote or network exception occurs
    */
@@ -2485,6 +2485,37 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
+   * Add a new replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig configuration for the replication peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  default Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig)
+      throws IOException {
+    return addReplicationPeerAsync(peerId, peerConfig, true);
+  }
+
+  /**
+   * Add a new replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig configuration for the replication peer
+   * @param enabled peer state, true if ENABLED and false if DISABLED
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
+      boolean enabled) throws IOException;
+
+  /**
    * Remove a peer and stop the replication.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2492,6 +2523,18 @@ public interface Admin extends Abortable, Closeable {
   void removeReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Remove a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> removeReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Restart the replication stream to the specified peer.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2499,6 +2542,18 @@ public interface Admin extends Abortable, Closeable {
   void enableReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Enable a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> enableReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Stop the replication stream to the specified peer.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2506,6 +2561,18 @@ public interface Admin extends Abortable, Closeable {
   void disableReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Disable a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> disableReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Returns the configured ReplicationPeerConfig for the specified peer.
    * @param peerId a short name that identifies the peer
    * @return ReplicationPeerConfig for the peer
@@ -2516,13 +2583,27 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Update the peerConfig for the specified peer.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig new config for the peer
+   * @param peerConfig new config for the replication peer
    * @throws IOException if a remote or network exception occurs
    */
   void updateReplicationPeerConfig(String peerId,
       ReplicationPeerConfig peerConfig) throws IOException;
 
   /**
+   * Update the peerConfig for the specified peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig new config for the replication peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig)
+      throws IOException;
+
+  /**
    * Append the replicable table column family config from the specified peer.
    * @param id a short that identifies the cluster
    * @param tableCfs A map from tableName to column family names

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb9f8d5d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 4ac1c21..1a87b48 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -200,7 +201,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 
 /**
@@ -3772,6 +3778,25 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  private static class ReplicationFuture extends ProcedureFuture<Void> {
+    private final String peerId;
+    private final Supplier<String> getOperation;
+
+    public ReplicationFuture(HBaseAdmin admin, String peerId, Long procId,
+        Supplier<String> getOperation) {
+      super(admin, procId);
+      this.peerId = peerId;
+      this.getOperation = getOperation;
+    }
+
+    @Override
+    public String toString() {
+      return "Operation: " + getOperation.get() + ", peerId: " + peerId;
+    }
+  }
+
   @Override
   public List<SecurityCapability> getSecurityCapabilities() throws IOException {
     try {
@@ -3844,50 +3869,82 @@ public class HBaseAdmin implements Admin {
   @Override
   public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.addReplicationPeer(getRpcController(),
-          RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled));
-        return null;
-      }
-    });
+    get(addReplicationPeerAsync(peerId, peerConfig, enabled), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
+      boolean enabled) throws IOException {
+    AddReplicationPeerResponse response = executeCallable(
+      new MasterCallable<AddReplicationPeerResponse>(getConnection(), getRpcControllerFactory()) {
+        @Override
+        protected AddReplicationPeerResponse rpcCall() throws Exception {
+          return master.addReplicationPeer(getRpcController(),
+            RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(), () -> "ADD_REPLICATION_PEER");
   }
 
   @Override
   public void removeReplicationPeer(String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.removeReplicationPeer(getRpcController(),
-          RequestConverter.buildRemoveReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(removeReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
+    RemoveReplicationPeerResponse response =
+      executeCallable(new MasterCallable<RemoveReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected RemoveReplicationPeerResponse rpcCall() throws Exception {
+          return master.removeReplicationPeer(getRpcController(),
+            RequestConverter.buildRemoveReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "REMOVE_REPLICATION_PEER");
   }
 
   @Override
   public void enableReplicationPeer(final String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.enableReplicationPeer(getRpcController(),
-          RequestConverter.buildEnableReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(enableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> enableReplicationPeerAsync(final String peerId) throws IOException {
+    EnableReplicationPeerResponse response =
+      executeCallable(new MasterCallable<EnableReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected EnableReplicationPeerResponse rpcCall() throws Exception {
+          return master.enableReplicationPeer(getRpcController(),
+            RequestConverter.buildEnableReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "ENABLE_REPLICATION_PEER");
   }
 
   @Override
   public void disableReplicationPeer(final String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.disableReplicationPeer(getRpcController(),
-          RequestConverter.buildDisableReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(disableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> disableReplicationPeerAsync(final String peerId) throws IOException {
+    DisableReplicationPeerResponse response =
+      executeCallable(new MasterCallable<DisableReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected DisableReplicationPeerResponse rpcCall() throws Exception {
+          return master.disableReplicationPeer(getRpcController(),
+            RequestConverter.buildDisableReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "DISABLE_REPLICATION_PEER");
   }
 
   @Override
@@ -3906,14 +3963,24 @@ public class HBaseAdmin implements Admin {
   @Override
   public void updateReplicationPeerConfig(final String peerId,
       final ReplicationPeerConfig peerConfig) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.updateReplicationPeerConfig(getRpcController(),
-          RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig));
-        return null;
-      }
-    });
+    get(updateReplicationPeerConfigAsync(peerId, peerConfig), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> updateReplicationPeerConfigAsync(final String peerId,
+      final ReplicationPeerConfig peerConfig) throws IOException {
+    UpdateReplicationPeerConfigResponse response =
+      executeCallable(new MasterCallable<UpdateReplicationPeerConfigResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected UpdateReplicationPeerConfigResponse rpcCall() throws Exception {
+          return master.updateReplicationPeerConfig(getRpcController(),
+            RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "UPDATE_REPLICATION_PEER_CONFIG");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb9f8d5d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 5111bfc..5a18afe 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -36,6 +36,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -1529,47 +1530,34 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> addReplicationPeer(String peerId,
       ReplicationPeerConfig peerConfig, boolean enabled) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<AddReplicationPeerRequest, AddReplicationPeerResponse, Void> call(controller, stub,
-                RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled), (s,
-                    c, req, done) -> s.addReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<AddReplicationPeerRequest, AddReplicationPeerResponse> procedureCall(
+      RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled),
+      (s, c, req, done) -> s.addReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "ADD_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> removeReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<RemoveReplicationPeerRequest, RemoveReplicationPeerResponse, Void> call(controller,
-                stub, RequestConverter.buildRemoveReplicationPeerRequest(peerId),
-                (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<RemoveReplicationPeerRequest, RemoveReplicationPeerResponse> procedureCall(
+      RequestConverter.buildRemoveReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "REMOVE_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> enableReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<EnableReplicationPeerRequest, EnableReplicationPeerResponse, Void> call(controller,
-                stub, RequestConverter.buildEnableReplicationPeerRequest(peerId),
-                (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<EnableReplicationPeerRequest, EnableReplicationPeerResponse> procedureCall(
+      RequestConverter.buildEnableReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "ENABLE_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> disableReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<DisableReplicationPeerRequest, DisableReplicationPeerResponse, Void> call(
-                controller, stub, RequestConverter.buildDisableReplicationPeerRequest(peerId), (s,
-                    c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> null))
-        .call();
+    return this.<DisableReplicationPeerRequest, DisableReplicationPeerResponse> procedureCall(
+      RequestConverter.buildDisableReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "DISABLE_REPLICATION_PEER"));
   }
 
   @Override
@@ -1588,13 +1576,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
       ReplicationPeerConfig peerConfig) {
     return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<UpdateReplicationPeerConfigRequest, UpdateReplicationPeerConfigResponse, Void> call(
-                controller, stub, RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId,
-                  peerConfig), (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done), (
-                    resp) -> null)).call();
+        .<UpdateReplicationPeerConfigRequest, UpdateReplicationPeerConfigResponse> procedureCall(
+          RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig),
+          (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done),
+          (resp) -> resp.getProcId(),
+          new ReplicationProcedureBiConsumer(peerId, () -> "UPDATE_REPLICATION_PEER_CONFIG"));
   }
 
   @Override
@@ -2553,6 +2539,30 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
+  private class ReplicationProcedureBiConsumer extends ProcedureBiConsumer {
+    private final String peerId;
+    private final Supplier<String> getOperation;
+
+    ReplicationProcedureBiConsumer(String peerId, Supplier<String> getOperation) {
+      this.peerId = peerId;
+      this.getOperation = getOperation;
+    }
+
+    String getDescription() {
+      return "Operation: " + getOperation.get() + ", peerId: " + peerId;
+    }
+
+    @Override
+    void onFinished() {
+      LOG.info(getDescription() + " completed");
+    }
+
+    @Override
+    void onError(Throwable error) {
+      LOG.info(getDescription() + " failed with " + error.getMessage());
+    }
+  }
+
   private CompletableFuture<Void> waitProcedureResult(CompletableFuture<Long> procFuture) {
     CompletableFuture<Void> future = new CompletableFuture<>();
     procFuture.whenComplete((procId, error) -> {


[27/48] hbase git commit: HBASE-19630 Add peer cluster key check when add new replication peer

Posted by zh...@apache.org.
HBASE-19630 Add peer cluster key check when add new replication peer

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: b8af6a4798877b7be761b74994720d8a78e7b104
Parents: 82a4272
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 26 21:10:00 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerManager.java     | 54 ++++++++++++--------
 .../replication/TestReplicationAdmin.java       | 22 ++++++++
 2 files changed, 54 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b8af6a47/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index 84abfeb..b78cbce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.replication;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -216,36 +218,36 @@ public final class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
-  /**
-   * If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
-   * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer
-   * cluster.
-   * <p>
-   * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
-   * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
-   */
-  private static void checkPeerConfig(ReplicationPeerConfig peerConfig)
-      throws DoNotRetryIOException {
+  private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetryIOException {
+    checkClusterKey(peerConfig.getClusterKey());
+
     if (peerConfig.replicateAllUserTables()) {
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
-        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " +
-          "when you want replicate all cluster");
+      // If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
+      // Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer
+      // cluster.
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty())
+          || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly "
+            + "when you want replicate all cluster");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
         peerConfig.getExcludeTableCFsMap());
     } else {
-      if ((peerConfig.getExcludeNamespaces() != null &&
-        !peerConfig.getExcludeNamespaces().isEmpty()) ||
-        (peerConfig.getExcludeTableCFsMap() != null &&
-          !peerConfig.getExcludeTableCFsMap().isEmpty())) {
+      // If replicate_all flag is false, it means all user tables can't be replicated to peer
+      // cluster. Then allow to config namespaces or table-cfs which will be replicated to peer
+      // cluster.
+      if ((peerConfig.getExcludeNamespaces() != null
+          && !peerConfig.getExcludeNamespaces().isEmpty())
+          || (peerConfig.getExcludeTableCFsMap() != null
+              && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
         throw new DoNotRetryIOException(
-            "Need clean exclude-namespaces or exclude-table-cfs config firstly" +
-              " when replicate_all flag is false");
+            "Need clean exclude-namespaces or exclude-table-cfs config firstly"
+                + " when replicate_all flag is false");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
         peerConfig.getTableCFsMap());
     }
+
     checkConfiguredWALEntryFilters(peerConfig);
   }
 
@@ -268,7 +270,7 @@ public final class ReplicationPeerManager {
    * exclude namespace.</li>
    * </ol>
    */
-  private static void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
+  private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
       Map<TableName, ? extends Collection<String>> tableCfs) throws DoNotRetryIOException {
     if (namespaces == null || namespaces.isEmpty()) {
       return;
@@ -285,7 +287,7 @@ public final class ReplicationPeerManager {
     }
   }
 
-  private static void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
+  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
       throws DoNotRetryIOException {
     String filterCSV = peerConfig.getConfiguration()
         .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
@@ -302,6 +304,14 @@ public final class ReplicationPeerManager {
     }
   }
 
+  private void checkClusterKey(String clusterKey) throws DoNotRetryIOException {
+    try {
+      ZKConfig.validateClusterKey(clusterKey);
+    } catch (IOException e) {
+      throw new DoNotRetryIOException("Invalid cluster key: " + clusterKey, e);
+    }
+  }
+
   public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
       throws ReplicationException {
     ReplicationPeerStorage peerStorage =

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8af6a47/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 56a6cfe..dbbc78f 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
@@ -113,6 +113,28 @@ public class TestReplicationAdmin {
     }
   }
 
+  @Test
+  public void testAddInvalidPeer() {
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_ONE);
+    try {
+      String invalidPeerId = "1-2";
+      hbaseAdmin.addReplicationPeer(invalidPeerId, builder.build());
+      fail("Should fail as the peer id: " + invalidPeerId + " is invalid");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      String invalidClusterKey = "2181:/hbase";
+      builder.setClusterKey(invalidClusterKey);
+      hbaseAdmin.addReplicationPeer(ID_ONE, builder.build());
+      fail("Should fail as the peer cluster key: " + invalidClusterKey + " is invalid");
+    } catch (Exception e) {
+      // OK
+    }
+  }
+
   /**
    * Simple testing of adding and removing peers, basically shows that
    * all interactions with ZK work


[26/48] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly

Posted by zh...@apache.org.
HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 050617daa547b6e060bcd28cf10567729f3c9cb7
Parents: ef8c27c
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 25 18:49:56 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |  19 +-
 .../replication/ReplicationPeersZKImpl.java     |  21 +-
 .../replication/ReplicationQueueStorage.java    |  26 +-
 .../replication/ReplicationQueuesClient.java    |  93 -----
 .../ReplicationQueuesClientArguments.java       |  40 --
 .../ReplicationQueuesClientZKImpl.java          | 176 ---------
 .../replication/ZKReplicationQueueStorage.java  |  90 ++++-
 .../replication/TestReplicationStateBasic.java  | 378 +++++++++++++++++++
 .../replication/TestReplicationStateZKImpl.java | 148 ++++++++
 .../TestZKReplicationQueueStorage.java          |  74 ++++
 .../cleaner/ReplicationZKNodeCleaner.java       |  71 ++--
 .../cleaner/ReplicationZKNodeCleanerChore.java  |   5 +-
 .../replication/ReplicationPeerManager.java     |  31 +-
 .../master/ReplicationHFileCleaner.java         | 109 ++----
 .../master/ReplicationLogCleaner.java           |  35 +-
 .../regionserver/DumpReplicationQueues.java     |  78 ++--
 .../hbase/util/hbck/ReplicationChecker.java     |  14 +-
 .../client/TestAsyncReplicationAdminApi.java    |  31 +-
 .../replication/TestReplicationAdmin.java       |   2 +
 .../hbase/master/cleaner/TestLogsCleaner.java   |  30 +-
 .../cleaner/TestReplicationHFileCleaner.java    |  59 +--
 .../cleaner/TestReplicationZKNodeCleaner.java   |  12 +-
 .../replication/TestReplicationStateBasic.java  | 378 -------------------
 .../replication/TestReplicationStateZKImpl.java | 227 -----------
 .../TestReplicationSourceManagerZkImpl.java     |  84 ++---
 25 files changed, 905 insertions(+), 1326 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 9f4ad18..6c1c213 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
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -37,20 +36,14 @@ public class ReplicationFactory {
       args);
   }
 
-  public static ReplicationQueuesClient
-      getReplicationQueuesClient(ReplicationQueuesClientArguments args) throws Exception {
-    return (ReplicationQueuesClient) ConstructorUtils
-        .invokeConstructor(ReplicationQueuesClientZKImpl.class, args);
-  }
-
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
-                                                     Abortable abortable) {
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
+      Abortable abortable) {
     return getReplicationPeers(zk, conf, null, abortable);
   }
 
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
-                                                     final ReplicationQueuesClient queuesClient, Abortable abortable) {
-    return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable);
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
+      ReplicationQueueStorage queueStorage, Abortable abortable) {
+    return new ReplicationPeersZKImpl(zk, conf, queueStorage, abortable);
   }
 
   public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 8ed0150..7de4619 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
@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -80,17 +81,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
 
   // Map of peer clusters keyed by their id
   private Map<String, ReplicationPeerZKImpl> peerClusters;
-  private final ReplicationQueuesClient queuesClient;
+  private final ReplicationQueueStorage queueStorage;
   private Abortable abortable;
 
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeersZKImpl.class);
 
-  public ReplicationPeersZKImpl(final ZKWatcher zk, final Configuration conf,
-                                final ReplicationQueuesClient queuesClient, Abortable abortable) {
+  public ReplicationPeersZKImpl(ZKWatcher zk, Configuration conf,
+      ReplicationQueueStorage queueStorage, Abortable abortable) {
     super(zk, conf, abortable);
     this.abortable = abortable;
     this.peerClusters = new ConcurrentHashMap<>();
-    this.queuesClient = queuesClient;
+    this.queueStorage = queueStorage;
   }
 
   @Override
@@ -510,14 +511,16 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   private void checkQueuesDeleted(String peerId) throws ReplicationException {
-    if (queuesClient == null) return;
+    if (queueStorage == null) {
+      return;
+    }
     try {
-      List<String> replicators = queuesClient.getListOfReplicators();
+      List<ServerName> replicators = queueStorage.getListOfReplicators();
       if (replicators == null || replicators.isEmpty()) {
         return;
       }
-      for (String replicator : replicators) {
-        List<String> queueIds = queuesClient.getAllQueues(replicator);
+      for (ServerName replicator : replicators) {
+        List<String> queueIds = queueStorage.getAllQueues(replicator);
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           if (queueInfo.getPeerId().equals(peerId)) {
@@ -528,7 +531,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       }
       // Check for hfile-refs queue
       if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode)
-          && queuesClient.getAllPeersFromHFileRefsQueue().contains(peerId)) {
+          && queueStorage.getAllPeersFromHFileRefsQueue().contains(peerId)) {
         throw new IllegalArgumentException("Undeleted queue for peerId: " + peerId
             + ", found in hfile-refs node path " + hfileRefsZNode);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
index 7210d9a..e774148 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -78,6 +78,14 @@ public interface ReplicationQueueStorage {
       throws ReplicationException;
 
   /**
+   * Get a list of all WALs in the given queue on the given region server.
+   * @param serverName the server name of the region server that owns the queue
+   * @param queueId a String that identifies the queue
+   * @return a list of WALs
+   */
+  List<String> getWALsInQueue(ServerName serverName, String queueId) throws ReplicationException;
+
+  /**
    * Get a list of all queues for the specified region server.
    * @param serverName the server name of the region server that owns the set of queues
    * @return a list of queueIds
@@ -108,8 +116,8 @@ public interface ReplicationQueueStorage {
 
   /**
    * Load all wals in all replication queues. This method guarantees to return a snapshot which
-   * contains all WALs in the zookeeper at the start of this call even there is concurrent queue
-   * failover. However, some newly created WALs during the call may not be included.
+   * contains all WALs at the start of this call even there is concurrent queue failover. However,
+   * some newly created WALs during the call may not be included.
    */
   Set<String> getAllWALs() throws ReplicationException;
 
@@ -143,13 +151,6 @@ public interface ReplicationQueueStorage {
   void removeHFileRefs(String peerId, List<String> files) throws ReplicationException;
 
   /**
-   * Get the change version number of replication hfile references node. This can be used as
-   * optimistic locking to get a consistent snapshot of the replication queues of hfile references.
-   * @return change version number of hfile references node
-   */
-  int getHFileRefsNodeChangeVersion() throws ReplicationException;
-
-  /**
    * Get list of all peers from hfile reference queue.
    * @return a list of peer ids
    */
@@ -161,4 +162,11 @@ public interface ReplicationQueueStorage {
    * @return a list of hfile references
    */
   List<String> getReplicableHFiles(String peerId) throws ReplicationException;
+
+  /**
+   * Load all hfile references in all replication queues. This method guarantees to return a
+   * snapshot which contains all hfile references at the start of this call. However, some newly
+   * created hfile references during the call may not be included.
+   */
+  Set<String> getAllHFileRefs() throws ReplicationException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
deleted file mode 100644
index 2c513fa..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
+++ /dev/null
@@ -1,93 +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.replication;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * This provides an interface for clients of replication to view replication queues. These queues
- * keep track of the sources(WALs/HFile references) that still need to be replicated to remote
- * clusters.
- */
-@InterfaceAudience.Private
-public interface ReplicationQueuesClient {
-
-  /**
-   * Initialize the replication queue client interface.
-   */
-  public void init() throws ReplicationException;
-
-  /**
-   * Get a list of all region servers that have outstanding replication queues. These servers could
-   * be alive, dead or from a previous run of the cluster.
-   * @return a list of server names
-   * @throws KeeperException zookeeper exception
-   */
-  List<String> getListOfReplicators() throws KeeperException;
-
-  /**
-   * Get a list of all WALs in the given queue on the given region server.
-   * @param serverName the server name of the region server that owns the queue
-   * @param queueId a String that identifies the queue
-   * @return a list of WALs, null if this region server is dead and has no outstanding queues
-   * @throws KeeperException zookeeper exception
-   */
-  List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException;
-
-  /**
-   * Get a list of all queues for the specified region server.
-   * @param serverName the server name of the region server that owns the set of queues
-   * @return a list of queueIds, null if this region server is not a replicator.
-   */
-  List<String> getAllQueues(String serverName) throws KeeperException;
-
-  /**
-   * Load all wals in all replication queues from ZK. This method guarantees to return a
-   * snapshot which contains all WALs in the zookeeper at the start of this call even there
-   * is concurrent queue failover. However, some newly created WALs during the call may
-   * not be included.
-   */
-   Set<String> getAllWALs() throws KeeperException;
-
-  /**
-   * Get the change version number of replication hfile references node. This can be used as
-   * optimistic locking to get a consistent snapshot of the replication queues of hfile references.
-   * @return change version number of hfile references node
-   */
-  int getHFileRefsNodeChangeVersion() throws KeeperException;
-
-  /**
-   * Get list of all peers from hfile reference queue.
-   * @return a list of peer ids
-   * @throws KeeperException zookeeper exception
-   */
-  List<String> getAllPeersFromHFileRefsQueue() throws KeeperException;
-
-  /**
-   * Get a list of all hfile references in the given peer.
-   * @param peerId a String that identifies the peer
-   * @return a list of hfile references, null if not found any
-   * @throws KeeperException zookeeper exception
-   */
-  List<String> getReplicableHFiles(String peerId) throws KeeperException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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
deleted file mode 100644
index 9b79294..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
+++ /dev/null
@@ -1,40 +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.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;
-
-/**
- * Wrapper around common arguments used to construct ReplicationQueuesClient. Used to construct
- * various ReplicationQueuesClient Implementations with different constructor arguments by
- * reflection.
- */
-@InterfaceAudience.Private
-public class ReplicationQueuesClientArguments extends ReplicationQueuesArguments {
-  public ReplicationQueuesClientArguments(Configuration conf, Abortable abort,
-     ZKWatcher zk) {
-    super(conf, abort, zk);
-  }
-  public ReplicationQueuesClientArguments(Configuration conf, Abortable abort) {
-    super(conf, abort);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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
deleted file mode 100644
index e00a7a2..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ /dev/null
@@ -1,176 +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.replication;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
-import org.apache.hbase.thirdparty.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.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@InterfaceAudience.Private
-public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implements
-    ReplicationQueuesClient {
-
-  Logger LOG = LoggerFactory.getLogger(ReplicationQueuesClientZKImpl.class);
-
-  public ReplicationQueuesClientZKImpl(ReplicationQueuesClientArguments args) {
-    this(args.getZk(), args.getConf(), args.getAbortable());
-  }
-
-  public ReplicationQueuesClientZKImpl(final ZKWatcher zk, Configuration conf,
-                                       Abortable abortable) {
-    super(zk, conf, abortable);
-  }
-
-  @Override
-  public void init() throws ReplicationException {
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) {
-        ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Internal error while initializing a queues client", e);
-    }
-  }
-
-  @Override
-  public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
-    znode = ZNodePaths.joinZNode(znode, queueId);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of wals for queueId=" + queueId
-          + " and serverName=" + serverName, e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public List<String> getAllQueues(String serverName) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of queues for serverName=" + serverName, e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public Set<String> getAllWALs() throws KeeperException {
-    /**
-     * Load all wals in all replication queues from ZK. This method guarantees to return a
-     * snapshot which contains all WALs in the zookeeper at the start of this call even there
-     * is concurrent queue failover. However, some newly created WALs during the call may
-     * not be included.
-     */
-    for (int retry = 0; ; retry++) {
-      int v0 = getQueuesZNodeCversion();
-      List<String> rss = getListOfReplicators();
-      if (rss == null || rss.isEmpty()) {
-        LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
-        return ImmutableSet.of();
-      }
-      Set<String> wals = Sets.newHashSet();
-      for (String rs : rss) {
-        List<String> listOfPeers = getAllQueues(rs);
-        // if rs just died, this will be null
-        if (listOfPeers == null) {
-          continue;
-        }
-        for (String id : listOfPeers) {
-          List<String> peersWals = getLogsInQueue(rs, id);
-          if (peersWals != null) {
-            wals.addAll(peersWals);
-          }
-        }
-      }
-      int v1 = getQueuesZNodeCversion();
-      if (v0 == v1) {
-        return wals;
-      }
-      LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
-        v0, v1, retry));
-    }
-  }
-
-  public int getQueuesZNodeCversion() throws KeeperException {
-    try {
-      Stat stat = new Stat();
-      ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
-      return stat.getCversion();
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get stat of replication rs node", e);
-      throw e;
-    }
-  }
-
-  @Override
-  public int getHFileRefsNodeChangeVersion() throws KeeperException {
-    Stat stat = new Stat();
-    try {
-      ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get stat of replication hfile references node.", e);
-      throw e;
-    }
-    return stat.getCversion();
-  }
-
-  @Override
-  public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of all peers in hfile references node.", e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public List<String> getReplicableHFiles(String peerId) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e);
-      throw e;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 7015d7f..0275d52 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.util.CollectionUtils.nullToEmpty;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
@@ -49,7 +50,7 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
  * ZK based replication queue storage.
@@ -61,7 +62,7 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationQueueStorage.class);
 
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
-    "zookeeper.znode.replication.hfile.refs";
+      "zookeeper.znode.replication.hfile.refs";
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
 
   /**
@@ -256,11 +257,23 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     }
   }
 
-  private List<String> getLogsInQueue0(ServerName serverName, String queueId)
+  private List<String> getWALsInQueue0(ServerName serverName, String queueId)
       throws KeeperException {
     return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getQueueNode(serverName, queueId)));
   }
 
+  @Override
+  public List<String> getWALsInQueue(ServerName serverName, String queueId)
+      throws ReplicationException {
+    try {
+      return getWALsInQueue0(serverName, queueId);
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Failed to get wals in queue (serverName=" + serverName + ", queueId=" + queueId + ")",
+          e);
+    }
+  }
+
   private List<String> getAllQueues0(ServerName serverName) throws KeeperException {
     return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getRsNode(serverName)));
   }
@@ -274,7 +287,9 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     }
   }
 
-  private int getQueuesZNodeCversion() throws KeeperException {
+  // will be overridden in UTs
+  @VisibleForTesting
+  protected int getQueuesZNodeCversion() throws KeeperException {
     Stat stat = new Stat();
     ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
     return stat.getCversion();
@@ -290,10 +305,10 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
           LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
           return Collections.emptySet();
         }
-        Set<String> wals = Sets.newHashSet();
+        Set<String> wals = new HashSet<>();
         for (ServerName rs : rss) {
           for (String queueId : getAllQueues0(rs)) {
-            wals.addAll(getLogsInQueue0(rs, queueId));
+            wals.addAll(getWALsInQueue0(rs, queueId));
           }
         }
         int v1 = getQueuesZNodeCversion();
@@ -356,9 +371,9 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     if (debugEnabled) {
       LOG.debug("Adding hfile references " + pairs + " in queue " + peerNode);
     }
-    List<ZKUtilOp> listOfOps =
-      pairs.stream().map(p -> p.getSecond().getName()).map(n -> getHFileNode(peerNode, n))
-          .map(f -> ZKUtilOp.createAndFailSilent(f, HConstants.EMPTY_BYTE_ARRAY)).collect(toList());
+    List<ZKUtilOp> listOfOps = pairs.stream().map(p -> p.getSecond().getName())
+        .map(n -> getHFileNode(peerNode, n))
+        .map(f -> ZKUtilOp.createAndFailSilent(f, HConstants.EMPTY_BYTE_ARRAY)).collect(toList());
     if (debugEnabled) {
       LOG.debug("The multi list size for adding hfile references in zk for node " + peerNode +
         " is " + listOfOps.size());
@@ -391,35 +406,70 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     }
   }
 
-  @Override
-  public int getHFileRefsNodeChangeVersion() throws ReplicationException {
-    Stat stat = new Stat();
-    try {
-      ZKUtil.getDataNoWatch(zookeeper, hfileRefsZNode, stat);
-    } catch (KeeperException e) {
-      throw new ReplicationException("Failed to get stat of replication hfile references node.", e);
-    }
-    return stat.getCversion();
+  private List<String> getAllPeersFromHFileRefsQueue0() throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode));
   }
 
   @Override
   public List<String> getAllPeersFromHFileRefsQueue() throws ReplicationException {
     try {
-      return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode));
+      return getAllPeersFromHFileRefsQueue0();
     } catch (KeeperException e) {
       throw new ReplicationException("Failed to get list of all peers in hfile references node.",
           e);
     }
   }
 
+  private List<String> getReplicableHFiles0(String peerId) throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId)));
+  }
+
   @Override
   public List<String> getReplicableHFiles(String peerId) throws ReplicationException {
     try {
-      return nullToEmpty(ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId)));
+      return getReplicableHFiles0(peerId);
     } catch (KeeperException e) {
       throw new ReplicationException("Failed to get list of hfile references for peer " + peerId,
           e);
     }
   }
 
+  // will be overridden in UTs
+  @VisibleForTesting
+  protected int getHFileRefsZNodeCversion() throws ReplicationException {
+    Stat stat = new Stat();
+    try {
+      ZKUtil.getDataNoWatch(zookeeper, hfileRefsZNode, stat);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get stat of replication hfile references node.", e);
+    }
+    return stat.getCversion();
+  }
+
+  @Override
+  public Set<String> getAllHFileRefs() throws ReplicationException {
+    try {
+      for (int retry = 0;; retry++) {
+        int v0 = getHFileRefsZNodeCversion();
+        List<String> peers = getAllPeersFromHFileRefsQueue();
+        if (peers.isEmpty()) {
+          LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions.");
+          return Collections.emptySet();
+        }
+        Set<String> hfileRefs = new HashSet<>();
+        for (String peer : peers) {
+          hfileRefs.addAll(getReplicableHFiles0(peer));
+        }
+        int v1 = getHFileRefsZNodeCversion();
+        if (v0 == v1) {
+          return hfileRefs;
+        }
+        LOG.debug(String.format(
+          "Replication hfile references node cversion changed from " + "%d to %d, retry = %d", v0,
+          v1, retry));
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get all hfile refs", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
new file mode 100644
index 0000000..6fe869c
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -0,0 +1,378 @@
+/**
+ * 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.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
+import org.apache.zookeeper.KeeperException;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * White box testing for replication state interfaces. Implementations should extend this class, and
+ * initialize the interfaces properly.
+ */
+public abstract class TestReplicationStateBasic {
+
+  protected ReplicationQueues rq1;
+  protected ReplicationQueues rq2;
+  protected ReplicationQueues rq3;
+  protected ReplicationQueueStorage rqs;
+  protected ServerName server1 = ServerName.valueOf("hostname1.example.org", 1234, 12345);
+  protected ServerName server2 = ServerName.valueOf("hostname2.example.org", 1234, 12345);
+  protected ServerName server3 = ServerName.valueOf("hostname3.example.org", 1234, 12345);
+  protected ReplicationPeers rp;
+  protected static final String ID_ONE = "1";
+  protected static final String ID_TWO = "2";
+  protected static String KEY_ONE;
+  protected static String KEY_TWO;
+
+  // For testing when we try to replicate to ourself
+  protected String OUR_ID = "3";
+  protected String OUR_KEY;
+
+  protected static int zkTimeoutCount;
+  protected static final int ZK_MAX_COUNT = 300;
+  protected static final int ZK_SLEEP_INTERVAL = 100; // millis
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class);
+
+  @Test
+  public void testReplicationQueueStorage() throws ReplicationException {
+    // Test methods with empty state
+    assertEquals(0, rqs.getListOfReplicators().size());
+    assertTrue(rqs.getWALsInQueue(server1, "qId1").isEmpty());
+    assertTrue(rqs.getAllQueues(server1).isEmpty());
+
+    /*
+     * Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each --
+     * server2: zero queues
+     */
+    rq1.init(server1.getServerName());
+    rq2.init(server2.getServerName());
+    rq1.addLog("qId1", "trash");
+    rq1.removeLog("qId1", "trash");
+    rq1.addLog("qId2", "filename1");
+    rq1.addLog("qId3", "filename2");
+    rq1.addLog("qId3", "filename3");
+    rq2.addLog("trash", "trash");
+    rq2.removeQueue("trash");
+
+    List<ServerName> reps = rqs.getListOfReplicators();
+    assertEquals(2, reps.size());
+    assertTrue(server1.getServerName(), reps.contains(server1));
+    assertTrue(server2.getServerName(), reps.contains(server2));
+
+    assertTrue(rqs.getWALsInQueue(ServerName.valueOf("bogus", 12345, 12345), "bogus").isEmpty());
+    assertTrue(rqs.getWALsInQueue(server1, "bogus").isEmpty());
+    assertEquals(0, rqs.getWALsInQueue(server1, "qId1").size());
+    assertEquals(1, rqs.getWALsInQueue(server1, "qId2").size());
+    assertEquals("filename1", rqs.getWALsInQueue(server1, "qId2").get(0));
+
+    assertTrue(rqs.getAllQueues(ServerName.valueOf("bogus", 12345, -1L)).isEmpty());
+    assertEquals(0, rqs.getAllQueues(server2).size());
+    List<String> list = rqs.getAllQueues(server1);
+    assertEquals(3, list.size());
+    assertTrue(list.contains("qId2"));
+    assertTrue(list.contains("qId3"));
+  }
+
+  @Test
+  public void testReplicationQueues() throws ReplicationException {
+    rq1.init(server1.getServerName());
+    rq2.init(server2.getServerName());
+    rq3.init(server3.getServerName());
+    // Initialize ReplicationPeer so we can add peers (we don't transfer lone queues)
+    rp.init();
+
+    // 3 replicators should exist
+    assertEquals(3, rq1.getListOfReplicators().size());
+    rq1.removeQueue("bogus");
+    rq1.removeLog("bogus", "bogus");
+    rq1.removeAllQueues();
+    assertEquals(0, rq1.getAllQueues().size());
+    assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
+    assertNull(rq1.getLogsInQueue("bogus"));
+    assertNull(rq1.getUnClaimedQueueIds(ServerName.valueOf("bogus", 1234, -1L).toString()));
+
+    rq1.setLogPosition("bogus", "bogus", 5L);
+
+    populateQueues();
+
+    assertEquals(3, rq1.getListOfReplicators().size());
+    assertEquals(0, rq2.getLogsInQueue("qId1").size());
+    assertEquals(5, rq3.getLogsInQueue("qId5").size());
+    assertEquals(0, rq3.getLogPosition("qId1", "filename0"));
+    rq3.setLogPosition("qId5", "filename4", 354L);
+    assertEquals(354L, rq3.getLogPosition("qId5", "filename4"));
+
+    assertEquals(5, rq3.getLogsInQueue("qId5").size());
+    assertEquals(0, rq2.getLogsInQueue("qId1").size());
+    assertEquals(0, rq1.getAllQueues().size());
+    assertEquals(1, rq2.getAllQueues().size());
+    assertEquals(5, rq3.getAllQueues().size());
+
+    assertEquals(0, rq3.getUnClaimedQueueIds(server1.getServerName()).size());
+    rq3.removeReplicatorIfQueueIsEmpty(server1.getServerName());
+    assertEquals(2, rq3.getListOfReplicators().size());
+
+    List<String> queues = rq2.getUnClaimedQueueIds(server3.getServerName());
+    assertEquals(5, queues.size());
+    for (String queue : queues) {
+      rq2.claimQueue(server3.getServerName(), queue);
+    }
+    rq2.removeReplicatorIfQueueIsEmpty(server3.getServerName());
+    assertEquals(1, rq2.getListOfReplicators().size());
+
+    // Try to claim our own queues
+    assertNull(rq2.getUnClaimedQueueIds(server2.getServerName()));
+    rq2.removeReplicatorIfQueueIsEmpty(server2.getServerName());
+
+    assertEquals(6, rq2.getAllQueues().size());
+
+    rq2.removeAllQueues();
+
+    assertEquals(0, rq2.getListOfReplicators().size());
+  }
+
+  @Test
+  public void testInvalidClusterKeys() throws ReplicationException, KeeperException {
+    rp.init();
+
+    try {
+      rp.registerPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"));
+      fail("Should throw an IllegalArgumentException because " +
+        "zookeeper.znode.parent is missing leading '/'.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+
+    try {
+      rp.registerPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"));
+      fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+
+    try {
+      rp.registerPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"));
+      fail("Should throw an IllegalArgumentException because " +
+        "hbase.zookeeper.property.clientPort is missing.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+  }
+
+  @Test
+  public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
+    rp.init();
+    rq1.init(server1.getServerName());
+
+    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
+    files1.add(new Pair<>(null, new Path("file_1")));
+    files1.add(new Pair<>(null, new Path("file_2")));
+    files1.add(new Pair<>(null, new Path("file_3")));
+    assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
+    assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
+    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rq1.addPeerToHFileRefs(ID_ONE);
+    rq1.addHFileRefs(ID_ONE, files1);
+    assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
+    assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
+    List<String> hfiles2 = new ArrayList<>(files1.size());
+    for (Pair<Path, Path> p : files1) {
+      hfiles2.add(p.getSecond().getName());
+    }
+    String removedString = hfiles2.remove(0);
+    rq1.removeHFileRefs(ID_ONE, hfiles2);
+    assertEquals(1, rqs.getReplicableHFiles(ID_ONE).size());
+    hfiles2 = new ArrayList<>(1);
+    hfiles2.add(removedString);
+    rq1.removeHFileRefs(ID_ONE, hfiles2);
+    assertEquals(0, rqs.getReplicableHFiles(ID_ONE).size());
+    rp.unregisterPeer(ID_ONE);
+  }
+
+  @Test
+  public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
+    rq1.init(server1.getServerName());
+
+    rp.init();
+    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rq1.addPeerToHFileRefs(ID_ONE);
+    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
+    rq1.addPeerToHFileRefs(ID_TWO);
+
+    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
+    files1.add(new Pair<>(null, new Path("file_1")));
+    files1.add(new Pair<>(null, new Path("file_2")));
+    files1.add(new Pair<>(null, new Path("file_3")));
+    rq1.addHFileRefs(ID_ONE, files1);
+    rq1.addHFileRefs(ID_TWO, files1);
+    assertEquals(2, rqs.getAllPeersFromHFileRefsQueue().size());
+    assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
+    assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
+
+    rp.unregisterPeer(ID_ONE);
+    rq1.removePeerFromHFileRefs(ID_ONE);
+    assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
+    assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
+    assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
+
+    rp.unregisterPeer(ID_TWO);
+    rq1.removePeerFromHFileRefs(ID_TWO);
+    assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
+    assertTrue(rqs.getReplicableHFiles(ID_TWO).isEmpty());
+  }
+
+  @Test
+  public void testReplicationPeers() throws Exception {
+    rp.init();
+
+    // Test methods with non-existent peer ids
+    try {
+      rp.unregisterPeer("bogus");
+      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
+    } catch (IllegalArgumentException e) {
+    }
+    try {
+      rp.enablePeer("bogus");
+      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
+    } catch (IllegalArgumentException e) {
+    }
+    try {
+      rp.disablePeer("bogus");
+      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
+    } catch (IllegalArgumentException e) {
+    }
+    try {
+      rp.getStatusOfPeer("bogus");
+      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
+    } catch (IllegalArgumentException e) {
+    }
+    assertFalse(rp.peerConnected("bogus"));
+    rp.peerDisconnected("bogus");
+
+    assertNull(rp.getPeerConf("bogus"));
+    assertNumberOfPeers(0);
+
+    // Add some peers
+    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    assertNumberOfPeers(1);
+    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
+    assertNumberOfPeers(2);
+
+    // Test methods with a peer that is added but not connected
+    try {
+      rp.getStatusOfPeer(ID_ONE);
+      fail("There are no connected peers, should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+    }
+    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond()));
+    rp.unregisterPeer(ID_ONE);
+    rp.peerDisconnected(ID_ONE);
+    assertNumberOfPeers(1);
+
+    // Add one peer
+    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rp.peerConnected(ID_ONE);
+    assertNumberOfPeers(2);
+    assertTrue(rp.getStatusOfPeer(ID_ONE));
+    rp.disablePeer(ID_ONE);
+    // now we do not rely on zk watcher to trigger the state change so we need to trigger it
+    // manually...
+    assertEquals(PeerState.DISABLED, rp.getConnectedPeer(ID_ONE).getPeerState(true));
+    assertConnectedPeerStatus(false, ID_ONE);
+    rp.enablePeer(ID_ONE);
+    // now we do not rely on zk watcher to trigger the state change so we need to trigger it
+    // manually...
+    assertEquals(PeerState.ENABLED, rp.getConnectedPeer(ID_ONE).getPeerState(true));
+    assertConnectedPeerStatus(true, ID_ONE);
+
+    // Disconnect peer
+    rp.peerDisconnected(ID_ONE);
+    assertNumberOfPeers(2);
+    try {
+      rp.getStatusOfPeer(ID_ONE);
+      fail("There are no connected peers, should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+    }
+  }
+
+  protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
+    // we can first check if the value was changed in the store, if it wasn't then fail right away
+    if (status != rp.getStatusOfPeerFromBackingStore(peerId)) {
+      fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK");
+    }
+    while (true) {
+      if (status == rp.getStatusOfPeer(peerId)) {
+        return;
+      }
+      if (zkTimeoutCount < ZK_MAX_COUNT) {
+        LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status +
+          ", sleeping and trying again.");
+        Thread.sleep(ZK_SLEEP_INTERVAL);
+      } else {
+        fail("Timed out waiting for ConnectedPeerStatus to be " + status);
+      }
+    }
+  }
+
+  protected void assertNumberOfPeers(int total) {
+    assertEquals(total, rp.getAllPeerConfigs().size());
+    assertEquals(total, rp.getAllPeerIds().size());
+    assertEquals(total, rp.getAllPeerIds().size());
+  }
+
+  /*
+   * three replicators: rq1 has 0 queues, rq2 has 1 queue with no logs, rq3 has 5 queues with 1, 2,
+   * 3, 4, 5 log files respectively
+   */
+  protected void populateQueues() throws ReplicationException {
+    rq1.addLog("trash", "trash");
+    rq1.removeQueue("trash");
+
+    rq2.addLog("qId1", "trash");
+    rq2.removeLog("qId1", "trash");
+
+    for (int i = 1; i < 6; i++) {
+      for (int j = 0; j < i; j++) {
+        rq3.addLog("qId" + i, "filename" + j);
+      }
+      // Add peers for the corresponding queues so they are not orphans
+      rp.registerPeer("qId" + i,
+        new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
new file mode 100644
index 0000000..5fe7c55
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -0,0 +1,148 @@
+/**
+ * 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.replication;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.ClusterId;
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+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.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateZKImpl.class);
+
+  private static Configuration conf;
+  private static HBaseZKTestingUtility utility;
+  private static ZKWatcher zkw;
+  private static String replicationZNode;
+  private ReplicationQueuesZKImpl rqZK;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    utility = new HBaseZKTestingUtility();
+    utility.startMiniZKCluster();
+    conf = utility.getConfiguration();
+    conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
+    zkw = utility.getZooKeeperWatcher();
+    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
+    replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
+    KEY_ONE = initPeerClusterState("/hbase1");
+    KEY_TWO = initPeerClusterState("/hbase2");
+  }
+
+  private static String initPeerClusterState(String baseZKNode)
+      throws IOException, KeeperException {
+    // Add a dummy region server and set up the cluster id
+    Configuration testConf = new Configuration(conf);
+    testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode);
+    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());
+    return ZKConfig.getZooKeeperClusterKey(testConf);
+  }
+
+  @Before
+  public void setUp() {
+    zkTimeoutCount = 0;
+    WarnOnlyAbortable abortable = new WarnOnlyAbortable();
+    try {
+      rq1 = ReplicationFactory
+          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
+      rq2 = ReplicationFactory
+          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
+      rq3 = ReplicationFactory
+          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
+      rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
+    } catch (Exception e) {
+      // This should not occur, because getReplicationQueues() only throws for
+      // TableBasedReplicationQueuesImpl
+      fail("ReplicationFactory.getReplicationQueues() threw an IO Exception");
+    }
+    rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
+    OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
+    rqZK = new ReplicationQueuesZKImpl(zkw, conf, abortable);
+  }
+
+  @After
+  public void tearDown() throws KeeperException, IOException {
+    ZKUtil.deleteNodeRecursively(zkw, replicationZNode);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    utility.shutdownMiniZKCluster();
+  }
+
+  @Test
+  public void testIsPeerPath_PathToParentOfPeerNode() {
+    assertFalse(rqZK.isPeerPath(rqZK.peersZNode));
+  }
+
+  @Test
+  public void testIsPeerPath_PathToChildOfPeerNode() {
+    String peerChild = ZNodePaths.joinZNode(ZNodePaths.joinZNode(rqZK.peersZNode, "1"), "child");
+    assertFalse(rqZK.isPeerPath(peerChild));
+  }
+
+  @Test
+  public void testIsPeerPath_ActualPeerPath() {
+    String peerPath = ZNodePaths.joinZNode(rqZK.peersZNode, "1");
+    assertTrue(rqZK.isPeerPath(peerPath));
+  }
+
+  private static class WarnOnlyAbortable implements Abortable {
+
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.warn("TestReplicationStateZKImpl received abort, ignoring.  Reason: " + why);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(e.toString(), e);
+      }
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
index d5bba0d..786730f 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -23,15 +23,18 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -115,6 +118,15 @@ public class TestZKReplicationQueueStorage {
     assertEquals(2, queueIds.size());
     assertThat(queueIds, hasItems("1", "2"));
 
+    List<String> wals1 = STORAGE.getWALsInQueue(serverName1, queue1);
+    List<String> wals2 = STORAGE.getWALsInQueue(serverName1, queue2);
+    assertEquals(10, wals1.size());
+    assertEquals(10, wals1.size());
+    for (int i = 0; i < 10; i++) {
+      assertThat(wals1, hasItems(getFileName("file1", i)));
+      assertThat(wals2, hasItems(getFileName("file2", i)));
+    }
+
     for (int i = 0; i < 10; i++) {
       assertEquals(0, STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
       assertEquals(0, STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
@@ -157,10 +169,20 @@ public class TestZKReplicationQueueStorage {
     queueIds = STORAGE.getAllQueues(serverName1);
     assertEquals(1, queueIds.size());
     assertThat(queueIds, hasItems("2"));
+    wals2 = STORAGE.getWALsInQueue(serverName1, queue2);
+    assertEquals(5, wals2.size());
+    for (i = 0; i < 10; i += 2) {
+      assertThat(wals2, hasItems(getFileName("file2", i)));
+    }
 
     queueIds = STORAGE.getAllQueues(serverName2);
     assertEquals(1, queueIds.size());
     assertThat(queueIds, hasItems(peer1.getFirst()));
+    wals1 = STORAGE.getWALsInQueue(serverName2, peer1.getFirst());
+    assertEquals(5, wals1.size());
+    for (i = 1; i < 10; i += 2) {
+      assertThat(wals1, hasItems(getFileName("file1", i)));
+    }
 
     Set<String> allWals = STORAGE.getAllWALs();
     assertEquals(10, allWals.size());
@@ -168,4 +190,56 @@ public class TestZKReplicationQueueStorage {
       assertThat(allWals, hasItems(i % 2 == 0 ? getFileName("file2", i) : getFileName("file1", i)));
     }
   }
+
+  // For HBASE-12865
+  @Test
+  public void testClaimQueueChangeCversion() throws ReplicationException, KeeperException {
+    ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
+    STORAGE.addWAL(serverName1, "1", "file");
+
+    int v0 = STORAGE.getQueuesZNodeCversion();
+    ServerName serverName2 = ServerName.valueOf("127.0.0.1", 8001, 10001);
+    STORAGE.claimQueue(serverName1, "1", serverName2);
+    int v1 = STORAGE.getQueuesZNodeCversion();
+    // cversion should increase by 1 since a child node is deleted
+    assertEquals(1, v1 - v0);
+  }
+
+  private ZKReplicationQueueStorage createWithUnstableCversion() throws IOException {
+    return new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()) {
+
+      private int called = 0;
+
+      @Override
+      protected int getQueuesZNodeCversion() throws KeeperException {
+        if (called < 4) {
+          called++;
+        }
+        return called;
+      }
+    };
+  }
+
+  @Test
+  public void testGetAllWALsCversionChange() throws IOException, ReplicationException {
+    ZKReplicationQueueStorage storage = createWithUnstableCversion();
+    storage.addWAL(getServerName(0), "1", "file");
+    // This should return eventually when cversion stabilizes
+    Set<String> allWals = storage.getAllWALs();
+    assertEquals(1, allWals.size());
+    assertThat(allWals, hasItems("file"));
+  }
+
+  // For HBASE-14621
+  @Test
+  public void testGetAllHFileRefsCversionChange() throws IOException, ReplicationException {
+    ZKReplicationQueueStorage storage = createWithUnstableCversion();
+    storage.addPeerToHFileRefs("1");
+    Path p = new Path("/test");
+    storage.addHFileRefs("1", Arrays.asList(Pair.newPair(p, p)));
+    // This should return eventually when cversion stabilizes
+    Set<String> allHFileRefs = storage.getAllHFileRefs();
+    assertEquals(1, allHFileRefs.size());
+    assertThat(allHFileRefs, hasItems("test"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 97deab5..af41399 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
@@ -23,21 +23,23 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.Map.Entry;
+import java.util.Set;
 
 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.ServerName;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 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.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,23 +50,19 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public class ReplicationZKNodeCleaner {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleaner.class);
-  private final ZKWatcher zkw;
-  private final ReplicationQueuesClient queuesClient;
+  private final ReplicationQueueStorage queueStorage;
   private final ReplicationPeers replicationPeers;
   private final ReplicationQueueDeletor queueDeletor;
 
   public ReplicationZKNodeCleaner(Configuration conf, ZKWatcher zkw, Abortable abortable)
       throws IOException {
     try {
-      this.zkw = zkw;
-      this.queuesClient = ReplicationFactory
-          .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw));
-      this.queuesClient.init();
-      this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.queuesClient,
-        abortable);
+      this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
+      this.replicationPeers =
+          ReplicationFactory.getReplicationPeers(zkw, conf, this.queueStorage, abortable);
       this.replicationPeers.init();
       this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, abortable);
-    } catch (Exception e) {
+    } catch (ReplicationException e) {
       throw new IOException("failed to construct ReplicationZKNodeCleaner", e);
     }
   }
@@ -73,16 +71,16 @@ public class ReplicationZKNodeCleaner {
    * @return undeletedQueues replicator with its queueIds for removed peers
    * @throws IOException
    */
-  public Map<String, List<String>> getUnDeletedQueues() throws IOException {
-    Map<String, List<String>> undeletedQueues = new HashMap<>();
+  public Map<ServerName, List<String>> getUnDeletedQueues() throws IOException {
+    Map<ServerName, List<String>> undeletedQueues = new HashMap<>();
     Set<String> peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds());
     try {
-      List<String> replicators = this.queuesClient.getListOfReplicators();
+      List<ServerName> replicators = this.queueStorage.getListOfReplicators();
       if (replicators == null || replicators.isEmpty()) {
         return undeletedQueues;
       }
-      for (String replicator : replicators) {
-        List<String> queueIds = this.queuesClient.getAllQueues(replicator);
+      for (ServerName replicator : replicators) {
+        List<String> queueIds = this.queueStorage.getAllQueues(replicator);
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           if (!peerIds.contains(queueInfo.getPeerId())) {
@@ -96,7 +94,7 @@ public class ReplicationZKNodeCleaner {
           }
         }
       }
-    } catch (KeeperException ke) {
+    } catch (ReplicationException ke) {
       throw new IOException("Failed to get the replication queues of all replicators", ke);
     }
     return undeletedQueues;
@@ -105,25 +103,21 @@ public class ReplicationZKNodeCleaner {
   /**
    * @return undeletedHFileRefsQueue replicator with its undeleted queueIds for removed peers in
    *         hfile-refs queue
-   * @throws IOException
    */
   public Set<String> getUnDeletedHFileRefsQueues() throws IOException {
     Set<String> undeletedHFileRefsQueue = new HashSet<>();
     Set<String> peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds());
     String hfileRefsZNode = queueDeletor.getHfileRefsZNode();
     try {
-      if (-1 == ZKUtil.checkExists(zkw, hfileRefsZNode)) {
-        return null;
-      }
-      List<String> listOfPeers = this.queuesClient.getAllPeersFromHFileRefsQueue();
+      List<String> listOfPeers = this.queueStorage.getAllPeersFromHFileRefsQueue();
       Set<String> peers = new HashSet<>(listOfPeers);
       peers.removeAll(peerIds);
       if (!peers.isEmpty()) {
         undeletedHFileRefsQueue.addAll(peers);
       }
-    } catch (KeeperException e) {
-      throw new IOException("Failed to get list of all peers from hfile-refs znode "
-          + hfileRefsZNode, e);
+    } catch (ReplicationException e) {
+      throw new IOException(
+          "Failed to get list of all peers from hfile-refs znode " + hfileRefsZNode, e);
     }
     return undeletedHFileRefsQueue;
   }
@@ -137,21 +131,20 @@ public class ReplicationZKNodeCleaner {
     /**
      * @param replicator The regionserver which has undeleted queue
      * @param queueId The undeleted queue id
-     * @throws IOException
      */
-    public void removeQueue(final String replicator, final String queueId) throws IOException {
-      String queueZnodePath =
-        ZNodePaths.joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator), queueId);
+    public void removeQueue(final ServerName replicator, final String queueId) throws IOException {
+      String queueZnodePath = ZNodePaths
+          .joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator.getServerName()), queueId);
       try {
         ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
         if (!replicationPeers.getAllPeerIds().contains(queueInfo.getPeerId())) {
           ZKUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath);
-          LOG.info("Successfully removed replication queue, replicator: " + replicator
-              + ", queueId: " + queueId);
+          LOG.info("Successfully removed replication queue, replicator: " + replicator +
+            ", queueId: " + queueId);
         }
       } catch (KeeperException e) {
-        throw new IOException("Failed to delete queue, replicator: " + replicator + ", queueId: "
-            + queueId);
+        throw new IOException(
+            "Failed to delete queue, replicator: " + replicator + ", queueId: " + queueId);
       }
     }
 
@@ -183,9 +176,9 @@ public class ReplicationZKNodeCleaner {
    * @param undeletedQueues replicator with its queueIds for removed peers
    * @throws IOException
    */
-  public void removeQueues(final Map<String, List<String>> undeletedQueues) throws IOException {
-    for (Entry<String, List<String>> replicatorAndQueueIds : undeletedQueues.entrySet()) {
-      String replicator = replicatorAndQueueIds.getKey();
+  public void removeQueues(final Map<ServerName, List<String>> undeletedQueues) throws IOException {
+    for (Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueues.entrySet()) {
+      ServerName replicator = replicatorAndQueueIds.getKey();
       for (String queueId : replicatorAndQueueIds.getValue()) {
         queueDeletor.removeQueue(replicator, queueId);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
index 8d5df9b..19ca804 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.cleaner;
 
 import java.io.IOException;
@@ -23,6 +22,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -45,11 +45,10 @@ public class ReplicationZKNodeCleanerChore extends ScheduledChore {
   @Override
   protected void chore() {
     try {
-      Map<String, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
+      Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
       cleaner.removeQueues(undeletedQueues);
     } catch (IOException e) {
       LOG.warn("Failed to clean replication zk node", e);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index 5abd874..84abfeb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
@@ -147,28 +148,13 @@ public final class ReplicationPeerManager {
     }
   }
 
-  private ReplicationPeerConfig copy(ReplicationPeerConfig peerConfig) {
-    ReplicationPeerConfig copiedPeerConfig = new ReplicationPeerConfig();
-    copiedPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
-    copiedPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
-    copiedPeerConfig.setTableCFsMap(peerConfig.getTableCFsMap());
-    copiedPeerConfig.setNamespaces(peerConfig.getNamespaces());
-    copiedPeerConfig.setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap());
-    copiedPeerConfig.setExcludeNamespaces(peerConfig.getExcludeNamespaces());
-    copiedPeerConfig.setBandwidth(peerConfig.getBandwidth());
-    copiedPeerConfig.setReplicateAllUserTables(peerConfig.replicateAllUserTables());
-    copiedPeerConfig.setClusterKey(peerConfig.getClusterKey());
-    copiedPeerConfig.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
-    return copiedPeerConfig;
-  }
-
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
     if (peers.containsKey(peerId)) {
       // this should be a retry, just return
       return;
     }
-    ReplicationPeerConfig copiedPeerConfig = copy(peerConfig);
+    ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build();
     peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
     peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
   }
@@ -205,13 +191,14 @@ public final class ReplicationPeerManager {
     // the checking rules are too complicated here so we give up checking whether this is a retry.
     ReplicationPeerDescription desc = peers.get(peerId);
     ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
-    ReplicationPeerConfig newPeerConfig = copy(peerConfig);
+    ReplicationPeerConfigBuilder newPeerConfigBuilder =
+        ReplicationPeerConfig.newBuilder(peerConfig);
     // we need to use the new conf to overwrite the old one.
-    newPeerConfig.getConfiguration().putAll(oldPeerConfig.getConfiguration());
-    newPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
-    newPeerConfig.getPeerData().putAll(oldPeerConfig.getPeerData());
-    newPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
-
+    newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
+    ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build();
     peerStorage.updatePeerConfig(peerId, newPeerConfig);
     peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 5f1df44..7b62169 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
@@ -1,42 +1,43 @@
-/*
- * 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.
+/**
+ * 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.replication.master;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
-import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-
 import java.io.IOException;
 import java.util.Collections;
-import java.util.List;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 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.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.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.zookeeper.KeeperException;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+
 /**
  * Implementation of a file cleaner that checks if a hfile is still scheduled for replication before
  * deleting it from hfile archive directory.
@@ -45,7 +46,7 @@ import org.slf4j.LoggerFactory;
 public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationHFileCleaner.class);
   private ZKWatcher zkw;
-  private ReplicationQueuesClient rqc;
+  private ReplicationQueueStorage rqs;
   private boolean stopped = false;
 
   @Override
@@ -60,8 +61,8 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     try {
       // The concurrently created new hfile entries in ZK may not be included in the return list,
       // but they won't be deleted because they're not in the checking set.
-      hfileRefs = loadHFileRefsFromPeers();
-    } catch (KeeperException e) {
+      hfileRefs = rqs.getAllHFileRefs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable files");
       return Collections.emptyList();
     }
@@ -82,37 +83,6 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     });
   }
 
-  /**
-   * Load all hfile references in all replication queues from ZK. This method guarantees to return a
-   * snapshot which contains all hfile references in the zookeeper at the start of this call.
-   * However, some newly created hfile references during the call may not be included.
-   */
-  private Set<String> loadHFileRefsFromPeers() throws KeeperException {
-    Set<String> hfileRefs = Sets.newHashSet();
-    List<String> listOfPeers;
-    for (int retry = 0;; retry++) {
-      int v0 = rqc.getHFileRefsNodeChangeVersion();
-      hfileRefs.clear();
-      listOfPeers = rqc.getAllPeersFromHFileRefsQueue();
-      if (listOfPeers == null) {
-        LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions.");
-        return ImmutableSet.of();
-      }
-      for (String id : listOfPeers) {
-        List<String> peerHFileRefs = rqc.getReplicableHFiles(id);
-        if (peerHFileRefs != null) {
-          hfileRefs.addAll(peerHFileRefs);
-        }
-      }
-      int v1 = rqc.getHFileRefsNodeChangeVersion();
-      if (v0 == v1) {
-        return hfileRefs;
-      }
-      LOG.debug(String.format("Replication hfile references node cversion changed from "
-          + "%d to %d, retry = %d", v0, v1, retry));
-    }
-  }
-
   @Override
   public void setConf(Configuration config) {
     // If either replication or replication of bulk load hfiles is disabled, keep all members null
@@ -139,17 +109,15 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
   public void setConf(Configuration conf, ZKWatcher zk) {
     super.setConf(conf);
     try {
-      initReplicationQueuesClient(conf, zk);
+      initReplicationQueueStorage(conf, zk);
     } catch (Exception e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
 
-  private void initReplicationQueuesClient(Configuration conf, ZKWatcher zk)
-      throws Exception {
+  private void initReplicationQueueStorage(Configuration conf, ZKWatcher zk) {
     this.zkw = zk;
-    this.rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(
-        conf, new WarnOnlyAbortable(), zkw));
+    this.rqs = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
   }
 
   @Override
@@ -179,25 +147,12 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     }
 
     try {
-      hfileRefsFromQueue = loadHFileRefsFromPeers();
-    } catch (KeeperException e) {
+      hfileRefsFromQueue = rqs.getAllHFileRefs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable "
           + "file for " + fStat.getPath());
       return false;
     }
     return !hfileRefsFromQueue.contains(fStat.getPath().getName());
   }
-
-  private static class WarnOnlyAbortable implements Abortable {
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.warn("ReplicationHFileCleaner received abort, ignoring.  Reason: " + why);
-      LOG.debug(e.toString(), e);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 86f98da..15aa21a 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
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -24,16 +23,14 @@ import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 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.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,7 +46,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
 public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogCleaner.class);
   private ZKWatcher zkw;
-  private ReplicationQueuesClient replicationQueues;
+  private ReplicationQueueStorage queueStorage;
   private boolean stopped = false;
   private Set<String> wals;
   private long readZKTimestamp = 0;
@@ -60,8 +57,8 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     try {
       // The concurrently created new WALs may not be included in the return list,
       // but they won't be deleted because they're not in the checking set.
-      wals = replicationQueues.getAllWALs();
-    } catch (KeeperException e) {
+      wals = queueStorage.getAllWALs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read zookeeper, skipping checking deletable files");
       wals = null;
     }
@@ -110,9 +107,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     super.setConf(conf);
     try {
       this.zkw = zk;
-      this.replicationQueues = ReplicationFactory.getReplicationQueuesClient(
-          new ReplicationQueuesClientArguments(conf, new WarnOnlyAbortable(), zkw));
-      this.replicationQueues.init();
+      this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
     } catch (Exception e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
@@ -132,18 +127,4 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
   public boolean isStopped() {
     return this.stopped;
   }
-
-  private static class WarnOnlyAbortable implements Abortable {
-
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.warn("ReplicationLogCleaner received abort, ignoring.  Reason: " + why);
-      LOG.debug(e.toString(), e);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-  }
 }


[41/48] hbase git commit: HBASE-19697 Remove TestReplicationAdminUsingProcedure

Posted by zh...@apache.org.
HBASE-19697 Remove TestReplicationAdminUsingProcedure


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 3432e91137ccdd250a981087bc88458b4fd4ba62
Parents: 72dbeed
Author: zhangduo <zh...@apache.org>
Authored: Wed Jan 3 21:13:57 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../TestReplicationAdminUsingProcedure.java     | 225 -------------------
 1 file changed, 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3432e911/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
deleted file mode 100644
index 1300376..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
+++ /dev/null
@@ -1,225 +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.client.replication;
-
-import java.io.IOException;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.replication.TestReplicationBase;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.log4j.Logger;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
-
-@Category({ MediumTests.class, ClientTests.class })
-public class TestReplicationAdminUsingProcedure extends TestReplicationBase {
-
-  private static final String PEER_ID = "2";
-  private static final Logger LOG = Logger.getLogger(TestReplicationAdminUsingProcedure.class);
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    conf1.setInt("hbase.multihconnection.threads.max", 10);
-
-    // Start the master & slave mini cluster.
-    TestReplicationBase.setUpBeforeClass();
-
-    // Remove the replication peer
-    hbaseAdmin.removeReplicationPeer(PEER_ID);
-  }
-
-  private void loadData(int startRowKey, int endRowKey) throws IOException {
-    for (int i = startRowKey; i < endRowKey; i++) {
-      byte[] rowKey = Bytes.add(row, Bytes.toBytes(i));
-      Put put = new Put(rowKey);
-      put.addColumn(famName, null, Bytes.toBytes(i));
-      htable1.put(put);
-    }
-  }
-
-  private void waitForReplication(int expectedRows, int retries)
-      throws IOException, InterruptedException {
-    Scan scan;
-    for (int i = 0; i < retries; i++) {
-      scan = new Scan();
-      if (i == retries - 1) {
-        throw new IOException("Waited too much time for normal batch replication");
-      }
-      try (ResultScanner scanner = htable2.getScanner(scan)) {
-        int count = 0;
-        for (Result res : scanner) {
-          count++;
-        }
-        if (count != expectedRows) {
-          LOG.info("Only got " + count + " rows,  expected rows: " + expectedRows);
-          Thread.sleep(SLEEP_TIME);
-        } else {
-          return;
-        }
-      }
-    }
-  }
-
-  @Before
-  public void setUp() throws IOException {
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(utility2.getClusterKey());
-    hbaseAdmin.addReplicationPeer(PEER_ID, rpc);
-
-    utility1.waitUntilAllRegionsAssigned(tableName);
-    utility2.waitUntilAllRegionsAssigned(tableName);
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    hbaseAdmin.removeReplicationPeer(PEER_ID);
-    truncateBoth();
-  }
-
-  private void truncateBoth() throws IOException {
-    utility1.deleteTableData(tableName);
-    utility2.deleteTableData(tableName);
-  }
-
-  @Test
-  public void testAddPeer() throws Exception {
-    // Load data
-    loadData(0, NB_ROWS_IN_BATCH);
-
-    // Wait the replication finished
-    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
-  }
-
-  @Test
-  public void testRemovePeer() throws Exception {
-    // prev-check
-    waitForReplication(0, NB_RETRIES);
-
-    // Load data
-    loadData(0, NB_ROWS_IN_BATCH);
-
-    // Wait the replication finished
-    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
-
-    // Remove the peer id
-    hbaseAdmin.removeReplicationPeer(PEER_ID);
-
-    // Load data again
-    loadData(NB_ROWS_IN_BATCH, 2 * NB_ROWS_IN_BATCH);
-
-    // Wait the replication again
-    boolean foundException = false;
-    try {
-      waitForReplication(NB_ROWS_IN_BATCH * 2, NB_RETRIES);
-    } catch (IOException e) {
-      foundException = true;
-    }
-    Assert.assertTrue(foundException);
-
-    // Truncate the table in source cluster
-    truncateBoth();
-
-    // Add peer again
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(utility2.getClusterKey());
-    hbaseAdmin.addReplicationPeer(PEER_ID, rpc);
-
-    // Load data again
-    loadData(0, NB_ROWS_IN_BATCH);
-
-    // Wait the replication finished
-    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
-  }
-
-  @Test
-  public void testDisableAndEnablePeer() throws Exception {
-    // disable peer
-    hbaseAdmin.disableReplicationPeer(PEER_ID);
-
-    // Load data
-    loadData(0, NB_ROWS_IN_BATCH);
-
-    // Will failed to wait the replication.
-    boolean foundException = false;
-    try {
-      waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
-    } catch (IOException e) {
-      foundException = true;
-    }
-    Assert.assertTrue(foundException);
-
-    // Enable the peer
-    hbaseAdmin.enableReplicationPeer(PEER_ID);
-    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
-
-    // Load more data
-    loadData(NB_ROWS_IN_BATCH, NB_ROWS_IN_BATCH * 2);
-
-    // Wait replication again.
-    waitForReplication(NB_ROWS_IN_BATCH * 2, NB_RETRIES);
-  }
-
-  @Test
-  public void testUpdatePeerConfig() throws Exception {
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(utility2.getClusterKey());
-    rpc.setExcludeTableCFsMap(
-      ImmutableMap.of(tableName, ImmutableList.of(Bytes.toString(famName))));
-
-    // Update the peer config to exclude the test table name.
-    hbaseAdmin.updateReplicationPeerConfig(PEER_ID, rpc);
-
-    // Load data
-    loadData(0, NB_ROWS_IN_BATCH);
-
-    // Will failed to wait the replication
-    boolean foundException = false;
-    try {
-      waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
-    } catch (IOException e) {
-      foundException = true;
-    }
-    Assert.assertTrue(foundException);
-
-    // Truncate the table in source cluster
-    truncateBoth();
-
-    // Update the peer config to include the test table name.
-    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
-    rpc2.setClusterKey(utility2.getClusterKey());
-    hbaseAdmin.updateReplicationPeerConfig(PEER_ID, rpc2);
-
-    // Load data again
-    loadData(0, NB_ROWS_IN_BATCH);
-
-    // Wait the replication finished
-    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
-  }
-}


[35/48] hbase git commit: HBASE-19719 Fix checkstyle issues

Posted by zh...@apache.org.
HBASE-19719 Fix checkstyle issues


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 8887a12ad2abe741d19fba8c21c687a9c3492293
Parents: 5c837e0
Author: zhangduo <zh...@apache.org>
Authored: Sat Jan 6 08:30:55 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationStorageFactory.java   |  2 +-
 .../master/assignment/RegionTransitionProcedure.java   |  4 ++--
 .../hbase/master/procedure/RSProcedureDispatcher.java  | 13 ++++++-------
 .../master/ReplicationPeerConfigUpgrader.java          |  8 ++++----
 4 files changed, 13 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8887a12a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
index 60d0749..462cfed 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
@@ -27,7 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * For now we only have zk based implementation.
  */
 @InterfaceAudience.Private
-public class ReplicationStorageFactory {
+public final class ReplicationStorageFactory {
 
   private ReplicationStorageFactory() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8887a12a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 1724a38..8277dbe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -36,11 +36,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Base class for the Assign and Unassign Procedure.
  *

http://git-wip-us.apache.org/repos/asf/hbase/blob/8887a12a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 0f68f31..a6d57d2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
@@ -36,6 +35,12 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
@@ -47,12 +52,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
 
-import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
-import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-
 /**
  * A remote procecdure dispatcher for regionservers.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8887a12a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
index b6e8862..f524fd7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -24,7 +23,6 @@ import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLI
 import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLICATION_ZNODE_DEFAULT;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Admin;
@@ -43,9 +41,11 @@ import org.apache.yetus.audience.InterfaceStability;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
 /**
  * This class is used to upgrade TableCFs from HBase 1.0, 1.1, 1.2, 1.3 to HBase 1.4 or 2.x. It will
  * be removed in HBase 3.x. See HBASE-11393


[10/48] hbase git commit: HBASE-19795 Move the tests which only need zookeeper in TestZooKeeper to hbase-zookeeper module

Posted by zh...@apache.org.
HBASE-19795 Move the tests which only need zookeeper in TestZooKeeper to hbase-zookeeper module


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 8dd4bf8e5474157bbb6938cb5973188d3d9445bc
Parents: ffa2850
Author: zhangduo <zh...@apache.org>
Authored: Mon Jan 15 10:57:51 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 10:58:03 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/zookeeper/ZNodePaths.java      |  21 +-
 .../hadoop/hbase/zookeeper/TestZNodePaths.java  |  51 ++++
 .../test/IntegrationTestZKAndFSPermissions.java |   2 +-
 .../org/apache/hadoop/hbase/TestZooKeeper.java  | 207 +--------------
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |   2 +-
 .../hadoop/hbase/zookeeper/ZKWatcher.java       |  17 --
 .../hadoop/hbase/zookeeper/TestZKUtil.java      | 260 ++++++++++++++-----
 .../hbase/zookeeper/TestZKUtilNoServer.java     | 113 ++++++++
 .../hadoop/hbase/zookeeper/TestZKWatcher.java   |  58 -----
 9 files changed, 384 insertions(+), 347 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8dd4bf8e/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
index 51272a6..9e7e51a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
@@ -26,7 +26,6 @@ import static org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
 
 import java.util.Optional;
 import java.util.stream.IntStream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -184,11 +183,23 @@ public class ZNodePaths {
   }
 
   /**
-   * Join the prefix znode name with the suffix znode name to generate a proper
-   * full znode name.
-   *
+   * 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(baseZNode) || isAnyMetaReplicaZNode(node) ||
+      node.equals(masterAddressZNode) || node.equals(clusterIdZNode) || node.equals(rsZNode) ||
+      // /hbase/table and /hbase/table/foo is allowed, /hbase/table-lock is not
+      node.equals(tableZNode) || node.startsWith(tableZNode + "/");
+  }
+
+  /**
+   * Join the prefix znode name with the suffix znode name to generate a proper full znode name.
+   * <p>
    * Assumes prefix does not end with slash and suffix does not begin with it.
-   *
    * @param prefix beginning of znode name
    * @param suffix ending of znode name
    * @return result of properly joining prefix with suffix

http://git-wip-us.apache.org/repos/asf/hbase/blob/8dd4bf8e/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZNodePaths.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZNodePaths.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZNodePaths.java
new file mode 100644
index 0000000..102dde1
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZNodePaths.java
@@ -0,0 +1,51 @@
+/**
+ * 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 org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.ZKTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ZKTests.class, SmallTests.class })
+public class TestZNodePaths {
+
+  @Test
+  public void testIsClientReadable() {
+    ZNodePaths znodePaths = new ZNodePaths(HBaseConfiguration.create());
+    assertTrue(znodePaths.isClientReadable(znodePaths.baseZNode));
+    assertTrue(znodePaths.isClientReadable(znodePaths.getZNodeForReplica(0)));
+    assertTrue(znodePaths.isClientReadable(znodePaths.masterAddressZNode));
+    assertTrue(znodePaths.isClientReadable(znodePaths.clusterIdZNode));
+    assertTrue(znodePaths.isClientReadable(znodePaths.tableZNode));
+    assertTrue(znodePaths.isClientReadable(ZNodePaths.joinZNode(znodePaths.tableZNode, "foo")));
+    assertTrue(znodePaths.isClientReadable(znodePaths.rsZNode));
+
+    assertFalse(znodePaths.isClientReadable(znodePaths.tableLockZNode));
+    assertFalse(znodePaths.isClientReadable(znodePaths.balancerZNode));
+    assertFalse(znodePaths.isClientReadable(znodePaths.regionNormalizerZNode));
+    assertFalse(znodePaths.isClientReadable(znodePaths.clusterStateZNode));
+    assertFalse(znodePaths.isClientReadable(znodePaths.drainingZNode));
+    assertFalse(znodePaths.isClientReadable(znodePaths.splitLogZNode));
+    assertFalse(znodePaths.isClientReadable(znodePaths.backupMasterAddressesZNode));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8dd4bf8e/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 6629f89..f785e94 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
@@ -159,7 +159,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
   private void checkZnodePermsRecursive(ZKWatcher watcher,
       RecoverableZooKeeper zk, String znode) throws KeeperException, InterruptedException {
 
-    boolean expectedWorldReadable = watcher.isClientReadable(znode);
+    boolean expectedWorldReadable = watcher.znodePaths.isClientReadable(znode);
 
     assertZnodePerms(zk, znode, expectedWorldReadable);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8dd4bf8e/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 b41e399..e03a9da 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
@@ -20,23 +20,21 @@ package org.apache.hadoop.hbase;
 
 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 static org.junit.Assert.fail;
 
-import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coordination.ZkSplitLogWorkerCoordination;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.LoadBalancer;
@@ -45,17 +43,9 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-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.ZKWatcher;
-import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Stat;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -67,20 +57,15 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 @Category({MiscTests.class, LargeTests.class})
 public class TestZooKeeper {
   private static final Logger LOG = LoggerFactory.getLogger(TestZooKeeper.class);
 
-  private final static HBaseTestingUtility
-      TEST_UTIL = new HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   @Rule
   public TestName name = new TestName();
 
-  /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     // Test we can first start the ZK cluster by itself
@@ -92,17 +77,11 @@ public class TestZooKeeper {
         LoadBalancer.class);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     TEST_UTIL.startMiniHBaseCluster(2, 2);
@@ -155,13 +134,11 @@ public class TestZooKeeper {
 
   /**
    * Make sure we can use the cluster
-   * @throws Exception
    */
-  private void testSanity(final String testName) throws Exception{
+  private void testSanity(final String testName) throws Exception {
     String tableName = testName + "_" + System.currentTimeMillis();
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
-    HColumnDescriptor family = new HColumnDescriptor("fam");
-    desc.addFamily(family);
+    TableDescriptor desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of("fam")).build();
     LOG.info("Creating table " + tableName);
     Admin admin = TEST_UTIL.getAdmin();
     try {
@@ -179,162 +156,6 @@ public class TestZooKeeper {
   }
 
   /**
-   * Create a znode with data
-   * @throws Exception
-   */
-  @Test
-  public void testCreateWithParents() throws Exception {
-    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);
-    byte[] data = ZKUtil.getData(zkw, "/l1/l2/l3/l4/testCreateWithParents");
-    assertTrue(Bytes.equals(expectedData, data));
-    ZKUtil.deleteNodeRecursively(zkw, "/l1");
-
-    ZKUtil.createWithParents(zkw, "/testCreateWithParents", expectedData);
-    data = ZKUtil.getData(zkw, "/testCreateWithParents");
-    assertTrue(Bytes.equals(expectedData, data));
-    ZKUtil.deleteNodeRecursively(zkw, "/testCreateWithParents");
-  }
-
-  /**
-   * Create a bunch of znodes in a hierarchy, try deleting one that has childs (it will fail), then
-   * delete it recursively, then delete the last znode
-   * @throws Exception
-   */
-  @Test
-  public void testZNodeDeletes() throws Exception {
-    ZKWatcher zkw = new ZKWatcher(
-      new Configuration(TEST_UTIL.getConfiguration()),
-      TestZooKeeper.class.getName(), null);
-    ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4");
-    try {
-      ZKUtil.deleteNode(zkw, "/l1/l2");
-      fail("We should not be able to delete if znode has childs");
-    } catch (KeeperException ex) {
-      assertNotNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2/l3/l4", null));
-    }
-    ZKUtil.deleteNodeRecursively(zkw, "/l1/l2");
-    // make sure it really is deleted
-    assertNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2/l3/l4", null));
-
-    // do the same delete again and make sure it doesn't crash
-    ZKUtil.deleteNodeRecursively(zkw, "/l1/l2");
-
-    ZKUtil.deleteNode(zkw, "/l1");
-    assertNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2", null));
-  }
-
-  /**
-   * A test for HBASE-3238
-   * @throws IOException A connection attempt to zk failed
-   * @throws InterruptedException One of the non ZKUtil actions was interrupted
-   * @throws KeeperException Any of the zookeeper connections had a
-   * KeeperException
-   */
-  @Test
-  public void testCreateSilentIsReallySilent() throws InterruptedException,
-      KeeperException, IOException {
-    Configuration c = TEST_UTIL.getConfiguration();
-
-    String aclZnode = "/aclRoot";
-    String quorumServers = ZKConfig.getZKQuorumServersString(c);
-    int sessionTimeout = 5 * 1000; // 5 seconds
-    ZooKeeper zk = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
-    zk.addAuthInfo("digest", "hbase:rox".getBytes());
-
-    // Assumes the  root of the ZooKeeper space is writable as it creates a node
-    // wherever the cluster home is defined.
-    ZKWatcher zk2 = new ZKWatcher(TEST_UTIL.getConfiguration(),
-      "testCreateSilentIsReallySilent", null);
-
-    // Save the previous ACL
-    Stat s =  null;
-    List<ACL> oldACL = null;
-    while (true) {
-      try {
-        s = new Stat();
-        oldACL = zk.getACL("/", s);
-        break;
-      } catch (KeeperException e) {
-        switch (e.code()) {
-          case CONNECTIONLOSS:
-          case SESSIONEXPIRED:
-          case OPERATIONTIMEOUT:
-            LOG.warn("Possibly transient ZooKeeper exception", e);
-            Threads.sleep(100);
-            break;
-         default:
-            throw e;
-        }
-      }
-    }
-
-    // I set this acl after the attempted creation of the cluster home node.
-    // Add retries in case of retryable zk exceptions.
-    while (true) {
-      try {
-        zk.setACL("/", ZooDefs.Ids.CREATOR_ALL_ACL, -1);
-        break;
-      } catch (KeeperException e) {
-        switch (e.code()) {
-          case CONNECTIONLOSS:
-          case SESSIONEXPIRED:
-          case OPERATIONTIMEOUT:
-            LOG.warn("Possibly transient ZooKeeper exception: " + e);
-            Threads.sleep(100);
-            break;
-         default:
-            throw e;
-        }
-      }
-    }
-
-    while (true) {
-      try {
-        zk.create(aclZnode, null, ZooDefs.Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
-        break;
-      } catch (KeeperException e) {
-        switch (e.code()) {
-          case CONNECTIONLOSS:
-          case SESSIONEXPIRED:
-          case OPERATIONTIMEOUT:
-            LOG.warn("Possibly transient ZooKeeper exception: " + e);
-            Threads.sleep(100);
-            break;
-         default:
-            throw e;
-        }
-      }
-    }
-    zk.close();
-    ZKUtil.createAndFailSilent(zk2, aclZnode);
-
-    // Restore the ACL
-    ZooKeeper zk3 = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
-    zk3.addAuthInfo("digest", "hbase:rox".getBytes());
-    try {
-      zk3.setACL("/", oldACL, -1);
-    } finally {
-      zk3.close();
-    }
- }
-
-  /**
-   * Test should not fail with NPE when getChildDataAndWatchForNewChildren
-   * invoked with wrongNode
-   */
-  @Test
-  @SuppressWarnings("deprecation")
-  public void testGetChildDataAndWatchForNewChildrenShouldNotThrowNPE()
-      throws Exception {
-    ZKWatcher zkw = new ZKWatcher(TEST_UTIL.getConfiguration(), name.getMethodName(), null);
-    ZKUtil.getChildDataAndWatchForNewChildren(zkw, "/wrongNode");
-  }
-
-  /**
    * Tests that the master does not call retainAssignment after recovery from expired zookeeper
    * session. Without the HBASE-6046 fix master always tries to assign all the user regions by
    * calling retainAssignment.
@@ -351,8 +172,9 @@ public class TestZooKeeper {
       byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"),
           Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
           Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i"), Bytes.toBytes("j") };
-      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-      htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+      TableDescriptor htd =
+          TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+              .addColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
       admin.createTable(htd, SPLIT_KEYS);
       TEST_UTIL.waitUntilNoRegionsInTransition(60000);
       m.getZooKeeper().close();
@@ -414,11 +236,10 @@ public class TestZooKeeper {
     Table table = null;
     try {
       byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("1"), Bytes.toBytes("2"),
-        Bytes.toBytes("3"), Bytes.toBytes("4"), Bytes.toBytes("5") };
-
-      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-      HColumnDescriptor hcd = new HColumnDescriptor("col");
-      htd.addFamily(hcd);
+          Bytes.toBytes("3"), Bytes.toBytes("4"), Bytes.toBytes("5") };
+      TableDescriptor htd =
+          TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+              .addColumnFamily(ColumnFamilyDescriptorBuilder.of("col")).build();
       admin.createTable(htd, SPLIT_KEYS);
       TEST_UTIL.waitUntilNoRegionsInTransition(60000);
       table = TEST_UTIL.getConnection().getTable(htd.getTableName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/8dd4bf8e/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
index 952013d..7bb4752 100644
--- 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
@@ -923,7 +923,7 @@ public final class ZKUtil {
       }
       // Certain znodes are accessed directly by the client,
       // so they must be readable by non-authenticated clients
-      if (zkw.isClientReadable(node)) {
+      if (zkw.znodePaths.isClientReadable(node)) {
         acls.addAll(Ids.CREATOR_ALL_ACL);
         acls.addAll(Ids.READ_ACL_UNSAFE);
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8dd4bf8e/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
index b0c6a6f..3aac946 100644
--- 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
@@ -159,23 +159,6 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
     }
   }
 
-  /** 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

http://git-wip-us.apache.org/repos/asf/hbase/blob/8dd4bf8e/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
index 2ea70d6..95dd270 100644
--- 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
@@ -7,105 +7,221 @@
  * "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
+ *     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.
+ * 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.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 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.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ZKTests;
-import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooDefs.Perms;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.junit.Assert;
+import org.apache.zookeeper.data.Stat;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 
-@Category({ ZKTests.class, SmallTests.class })
+@Category({ ZKTests.class, MediumTests.class })
 public class TestZKUtil {
 
+  private static final Logger LOG = LoggerFactory.getLogger(TestZKUtil.class);
+
+  private static HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
+
+  private static ZKWatcher ZKW;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniZKCluster().getClientPort();
+    ZKW = new ZKWatcher(new Configuration(UTIL.getConfiguration()), TestZKUtil.class.getName(),
+        new WarnOnlyAbortable());
+
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    Closeables.close(ZKW, true);
+    UTIL.shutdownMiniZKCluster();
+    UTIL.cleanupTestDir();
+  }
+
+  /**
+   * Create a znode with data
+   */
   @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(1, aclList.size());
-    Assert.assertTrue(aclList.contains(Ids.OPEN_ACL_UNSAFE.iterator().next()));
+  public void testCreateWithParents() throws KeeperException, InterruptedException {
+    byte[] expectedData = new byte[] { 1, 2, 3 };
+    ZKUtil.createWithParents(ZKW, "/l1/l2/l3/l4/testCreateWithParents", expectedData);
+    byte[] data = ZKUtil.getData(ZKW, "/l1/l2/l3/l4/testCreateWithParents");
+    assertTrue(Bytes.equals(expectedData, data));
+    ZKUtil.deleteNodeRecursively(ZKW, "/l1");
+
+    ZKUtil.createWithParents(ZKW, "/testCreateWithParents", expectedData);
+    data = ZKUtil.getData(ZKW, "/testCreateWithParents");
+    assertTrue(Bytes.equals(expectedData, data));
+    ZKUtil.deleteNodeRecursively(ZKW, "/testCreateWithParents");
   }
 
+  /**
+   * Create a bunch of znodes in a hierarchy, try deleting one that has childs (it will fail), then
+   * delete it recursively, then delete the last znode
+   */
   @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(2, aclList.size()); // 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()));
+  public void testZNodeDeletes() throws Exception {
+    ZKUtil.createWithParents(ZKW, "/l1/l2/l3/l4");
+    try {
+      ZKUtil.deleteNode(ZKW, "/l1/l2");
+      fail("We should not be able to delete if znode has childs");
+    } catch (KeeperException ex) {
+      assertNotNull(ZKUtil.getDataNoWatch(ZKW, "/l1/l2/l3/l4", null));
+    }
+    ZKUtil.deleteNodeRecursively(ZKW, "/l1/l2");
+    // make sure it really is deleted
+    assertNull(ZKUtil.getDataNoWatch(ZKW, "/l1/l2/l3/l4", null));
+
+    // do the same delete again and make sure it doesn't crash
+    ZKUtil.deleteNodeRecursively(ZKW, "/l1/l2");
+
+    ZKUtil.deleteNode(ZKW, "/l1");
+    assertNull(ZKUtil.getDataNoWatch(ZKW, "/l1/l2", null));
   }
 
+  /**
+   * A test for HBASE-3238
+   * @throws IOException A connection attempt to zk failed
+   * @throws InterruptedException One of the non ZKUtil actions was interrupted
+   * @throws KeeperException Any of the zookeeper connections had a KeeperException
+   */
   @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(4, aclList.size()); // 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"))));
+  public void testCreateSilentIsReallySilent()
+      throws InterruptedException, KeeperException, IOException {
+    Configuration c = UTIL.getConfiguration();
+
+    String aclZnode = "/aclRoot";
+    String quorumServers = ZKConfig.getZKQuorumServersString(c);
+    int sessionTimeout = 5 * 1000; // 5 seconds
+    ZooKeeper zk = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
+    zk.addAuthInfo("digest", "hbase:rox".getBytes());
+
+    // Save the previous ACL
+    Stat s = null;
+    List<ACL> oldACL = null;
+    while (true) {
+      try {
+        s = new Stat();
+        oldACL = zk.getACL("/", s);
+        break;
+      } catch (KeeperException e) {
+        switch (e.code()) {
+          case CONNECTIONLOSS:
+          case SESSIONEXPIRED:
+          case OPERATIONTIMEOUT:
+            LOG.warn("Possibly transient ZooKeeper exception", e);
+            Threads.sleep(100);
+            break;
+          default:
+            throw e;
+        }
+      }
+    }
+
+    // I set this acl after the attempted creation of the cluster home node.
+    // Add retries in case of retryable zk exceptions.
+    while (true) {
+      try {
+        zk.setACL("/", ZooDefs.Ids.CREATOR_ALL_ACL, -1);
+        break;
+      } catch (KeeperException e) {
+        switch (e.code()) {
+          case CONNECTIONLOSS:
+          case SESSIONEXPIRED:
+          case OPERATIONTIMEOUT:
+            LOG.warn("Possibly transient ZooKeeper exception: " + e);
+            Threads.sleep(100);
+            break;
+          default:
+            throw e;
+        }
+      }
+    }
+
+    while (true) {
+      try {
+        zk.create(aclZnode, null, ZooDefs.Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
+        break;
+      } catch (KeeperException e) {
+        switch (e.code()) {
+          case CONNECTIONLOSS:
+          case SESSIONEXPIRED:
+          case OPERATIONTIMEOUT:
+            LOG.warn("Possibly transient ZooKeeper exception: " + e);
+            Threads.sleep(100);
+            break;
+          default:
+            throw e;
+        }
+      }
+    }
+    zk.close();
+    ZKUtil.createAndFailSilent(ZKW, aclZnode);
+
+    // Restore the ACL
+    ZooKeeper zk3 = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
+    zk3.addAuthInfo("digest", "hbase:rox".getBytes());
+    try {
+      zk3.setACL("/", oldACL, -1);
+    } finally {
+      zk3.close();
+    }
   }
 
+  /**
+   * Test should not fail with NPE when getChildDataAndWatchForNewChildren invoked with wrongNode
+   */
   @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(3, aclList.size()); // 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"))));
+  @SuppressWarnings("deprecation")
+  public void testGetChildDataAndWatchForNewChildrenShouldNotThrowNPE() throws Exception {
+    ZKUtil.getChildDataAndWatchForNewChildren(ZKW, "/wrongNode");
   }
 
-  @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;
+  private static class WarnOnlyAbortable implements Abortable {
+
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.warn("ZKWatcher received abort, ignoring.  Reason: " + why);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(e.toString(), e);
       }
-    };
-    Mockito.doThrow(new InterruptedException()).when(recoverableZk)
-        .getChildren(zkw.znodePaths.baseZNode, null);
-    ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.baseZNode);
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8dd4bf8e/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtilNoServer.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtilNoServer.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtilNoServer.java
new file mode 100644
index 0000000..cc2517b
--- /dev/null
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtilNoServer.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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+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.hbase.testclassification.ZKTests;
+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.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+@Category({ ZKTests.class, SmallTests.class })
+public class TestZKUtilNoServer {
+
+  @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);
+    assertEquals(1, aclList.size());
+    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);
+    assertEquals(2, aclList.size()); // 1+1, since ACL will be set for the creator by default
+    assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
+    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);
+    assertEquals(4, aclList.size()); // 3+1, since ACL will be set for the creator by default
+    assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
+    assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group2"))));
+    assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
+    assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user2"))));
+    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);
+    assertEquals(3, aclList.size()); // 3, since service user the same as one of superuser
+    assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
+    assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("auth", ""))));
+    assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user5"))));
+    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/8dd4bf8e/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
deleted file mode 100644
index f3d0b03..0000000
--- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKWatcher.java
+++ /dev/null
@@ -1,58 +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.apache.hadoop.hbase.testclassification.ZKTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ ZKTests.class, 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();
-  }
-}


[14/48] hbase git commit: HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 853bafb..24a4f30 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -166,7 +166,6 @@ public class ReplicationSourceManager implements ReplicationListener {
     this.clusterId = clusterId;
     this.walFileLengthProvider = walFileLengthProvider;
     this.replicationTracker.registerListener(this);
-    this.replicationPeers.getAllPeerIds();
     // It's preferable to failover 1 RS at a time, but with good zk servers
     // more could be processed at the same time.
     int nbWorkers = conf.getInt("replication.executor.workers", 1);
@@ -270,8 +269,8 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
     List<ServerName> otherRegionServers = replicationTracker.getListOfRegionServers().stream()
         .map(ServerName::valueOf).collect(Collectors.toList());
-    LOG.info(
-      "Current list of replicators: " + currentReplicators + " other RSs: " + otherRegionServers);
+    LOG.info("Current list of replicators: " + currentReplicators + " other RSs: "
+        + otherRegionServers);
 
     // Look if there's anything to process after a restart
     for (ServerName rs : currentReplicators) {
@@ -288,7 +287,7 @@ public class ReplicationSourceManager implements ReplicationListener {
    * The returned future is for adoptAbandonedQueues task.
    */
   Future<?> init() throws IOException, ReplicationException {
-    for (String id : this.replicationPeers.getConnectedPeerIds()) {
+    for (String id : this.replicationPeers.getAllPeerIds()) {
       addSource(id);
       if (replicationForBulkLoadDataEnabled) {
         // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
@@ -307,8 +306,8 @@ public class ReplicationSourceManager implements ReplicationListener {
    */
   @VisibleForTesting
   ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException {
-    ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
-    ReplicationPeer peer = replicationPeers.getConnectedPeer(id);
+    ReplicationPeerConfig peerConfig = replicationPeers.getPeerConfig(id);
+    ReplicationPeer peer = replicationPeers.getPeer(id);
     ReplicationSourceInterface src = getReplicationSource(id, peerConfig, peer);
     synchronized (this.walsById) {
       this.sources.add(src);
@@ -354,7 +353,7 @@ public class ReplicationSourceManager implements ReplicationListener {
   public void deleteSource(String peerId, boolean closeConnection) {
     abortWhenFail(() -> this.queueStorage.removeQueue(server.getServerName(), peerId));
     if (closeConnection) {
-      this.replicationPeers.peerDisconnected(peerId);
+      this.replicationPeers.removePeer(peerId);
     }
   }
 
@@ -445,12 +444,12 @@ public class ReplicationSourceManager implements ReplicationListener {
     // update replication queues on ZK
     // synchronize on replicationPeers to avoid adding source for the to-be-removed peer
     synchronized (replicationPeers) {
-      for (String id : replicationPeers.getConnectedPeerIds()) {
+      for (String id : replicationPeers.getAllPeerIds()) {
         try {
           this.queueStorage.addWAL(server.getServerName(), id, logName);
         } catch (ReplicationException e) {
-          throw new IOException("Cannot add log to replication queue" +
-            " when creating a new source, queueId=" + id + ", filename=" + logName, e);
+          throw new IOException("Cannot add log to replication queue"
+              + " when creating a new source, queueId=" + id + ", filename=" + logName, e);
         }
       }
     }
@@ -593,7 +592,7 @@ public class ReplicationSourceManager implements ReplicationListener {
 
   public void addPeer(String id) throws ReplicationException, IOException {
     LOG.info("Trying to add peer, peerId: " + id);
-    boolean added = this.replicationPeers.peerConnected(id);
+    boolean added = this.replicationPeers.addPeer(id);
     if (added) {
       LOG.info("Peer " + id + " connected success, trying to start the replication source thread.");
       addSource(id);
@@ -729,19 +728,25 @@ public class ReplicationSourceManager implements ReplicationListener {
           // there is not an actual peer defined corresponding to peerId for the failover.
           ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
           String actualPeerId = replicationQueueInfo.getPeerId();
-          ReplicationPeer peer = replicationPeers.getConnectedPeer(actualPeerId);
+
+          ReplicationPeer peer = replicationPeers.getPeer(actualPeerId);
+          if (peer == null) {
+            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS
+                + ", peer is null");
+            abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
+            continue;
+          }
+
           ReplicationPeerConfig peerConfig = null;
           try {
-            peerConfig = replicationPeers.getReplicationPeerConfig(actualPeerId);
-          } catch (ReplicationException ex) {
-            LOG.warn("Received exception while getting replication peer config, skipping replay"
-                + ex);
-          }
-          if (peer == null || peerConfig == null) {
-            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS);
+            peerConfig = replicationPeers.getPeerConfig(actualPeerId);
+          } catch (Exception e) {
+            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS
+                + ", failed to read peer config", e);
             abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
             continue;
           }
+
           // track sources in walsByIdRecoveredQueues
           Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
           walsByIdRecoveredQueues.put(peerId, walsByGroup);
@@ -760,7 +765,7 @@ public class ReplicationSourceManager implements ReplicationListener {
           // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
           // see removePeer
           synchronized (oldsources) {
-            if (!replicationPeers.getConnectedPeerIds().contains(src.getPeerId())) {
+            if (!replicationPeers.getAllPeerIds().contains(src.getPeerId())) {
               src.terminate("Recovered queue doesn't belong to any current peer");
               closeRecoveredQueue(src);
               continue;

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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 8802e36..905ade4 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
@@ -87,7 +87,7 @@ public class TestReplicationHFileCleaner {
     server = new DummyServer();
     conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
     Replication.decorateMasterConfiguration(conf);
-    rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf, server);
+    rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf);
     rp.init();
     rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
     fs = FileSystem.get(conf);
@@ -101,7 +101,8 @@ public class TestReplicationHFileCleaner {
   @Before
   public void setup() throws ReplicationException, IOException {
     root = TEST_UTIL.getDataTestDirOnTestFS();
-    rp.registerPeer(peerId, new ReplicationPeerConfig().setClusterKey(TEST_UTIL.getClusterKey()));
+    rp.getPeerStorage().addPeer(peerId,
+      ReplicationPeerConfig.newBuilder().setClusterKey(TEST_UTIL.getClusterKey()).build(), true);
     rq.addPeerToHFileRefs(peerId);
   }
 
@@ -112,7 +113,7 @@ public class TestReplicationHFileCleaner {
     } catch (IOException e) {
       LOG.warn("Failed to delete files recursively from path " + root);
     }
-    rp.unregisterPeer(peerId);
+    rp.getPeerStorage().removePeer(peerId);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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 c57d9bb..ca4369e 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
@@ -21,9 +21,7 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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 f118ca3..fdfa6b7 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
@@ -22,8 +22,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.conf.Configuration;
@@ -44,6 +42,7 @@ 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.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -73,10 +72,6 @@ public class TestReplicationTrackerZKImpl {
   private ReplicationTracker rt;
   private AtomicInteger rsRemovedCount;
   private String rsRemovedData;
-  private AtomicInteger plChangedCount;
-  private List<String> plChangedData;
-  private AtomicInteger peerRemovedCount;
-  private String peerRemovedData;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -93,7 +88,7 @@ public class TestReplicationTrackerZKImpl {
     String fakeRs1 = ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234");
     try {
       ZKClusterId.setClusterId(zkw, new ClusterId());
-      rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
+      rp = ReplicationFactory.getReplicationPeers(zkw, conf);
       rp.init();
       rt = ReplicationFactory.getReplicationTracker(zkw, rp, conf, zkw, new DummyServer(fakeRs1));
     } catch (Exception e) {
@@ -101,10 +96,6 @@ public class TestReplicationTrackerZKImpl {
     }
     rsRemovedCount = new AtomicInteger(0);
     rsRemovedData = "";
-    plChangedCount = new AtomicInteger(0);
-    plChangedData = new ArrayList<>();
-    peerRemovedCount = new AtomicInteger(0);
-    peerRemovedData = "";
   }
 
   @AfterClass
@@ -157,25 +148,22 @@ public class TestReplicationTrackerZKImpl {
   @Test(timeout = 30000)
   public void testPeerNameControl() throws Exception {
     int exists = 0;
-    int hyphen = 0;
-    rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
+    rp.getPeerStorage().addPeer("6",
+      ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true);
 
-    try{
-      rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    }catch(IllegalArgumentException e){
-      exists++;
+    try {
+      rp.getPeerStorage().addPeer("6",
+        ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true);
+    } catch (ReplicationException e) {
+      if (e.getCause() instanceof KeeperException.NodeExistsException) {
+        exists++;
+      }
     }
 
-    try{
-      rp.registerPeer("6-ec2", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    }catch(IllegalArgumentException e){
-      hyphen++;
-    }
     assertEquals(1, exists);
-    assertEquals(1, hyphen);
 
     // clean up
-    rp.unregisterPeer("6");
+    rp.getPeerStorage().removePeer("6");
   }
 
   private class DummyReplicationListener implements ReplicationListener {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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 325012d..dd56a76 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
@@ -380,7 +380,7 @@ public abstract class TestReplicationSourceManager {
     }
     Server s1 = new DummyServer("dummyserver1.example.org");
     ReplicationPeers rp1 =
-        ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration(), s1);
+        ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration());
     rp1.init();
     NodeFailoverWorker w1 =
         manager.new NodeFailoverWorker(server.getServerName());
@@ -561,7 +561,7 @@ public abstract class TestReplicationSourceManager {
   private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig,
       final boolean waitForSource) throws Exception {
     final ReplicationPeers rp = manager.getReplicationPeers();
-    rp.registerPeer(peerId, peerConfig);
+    rp.getPeerStorage().addPeer(peerId, peerConfig, true);
     try {
       manager.addPeer(peerId);
     } catch (Exception e) {
@@ -588,7 +588,7 @@ public abstract class TestReplicationSourceManager {
         }
         return true;
       } else {
-        return (rp.getConnectedPeer(peerId) != null);
+        return (rp.getPeer(peerId) != null);
       }
     });
   }
@@ -600,8 +600,8 @@ public abstract class TestReplicationSourceManager {
    */
   private void removePeerAndWait(final String peerId) throws Exception {
     final ReplicationPeers rp = manager.getReplicationPeers();
-    if (rp.getAllPeerIds().contains(peerId)) {
-      rp.unregisterPeer(peerId);
+    if (rp.getPeerStorage().listPeerIds().contains(peerId)) {
+      rp.getPeerStorage().removePeer(peerId);
       try {
         manager.removePeer(peerId);
       } catch (Exception e) {
@@ -611,10 +611,9 @@ public abstract class TestReplicationSourceManager {
     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        List<String> peers = rp.getAllPeerIds();
-        return (!manager.getAllQueues().contains(peerId)) &&
-          (rp.getConnectedPeer(peerId) == null) && (!peers.contains(peerId)) &&
-          manager.getSource(peerId) == null;
+        Collection<String> peers = rp.getPeerStorage().listPeerIds();
+        return (!manager.getAllQueues().contains(peerId)) && (rp.getPeer(peerId) == null)
+            && (!peers.contains(peerId)) && manager.getSource(peerId) == null;
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java
index fc31c37..b755c32 100644
--- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java
@@ -182,8 +182,7 @@ public class HBaseZKTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Gets a ZKWatcher.
    */
-  public static ZKWatcher getZooKeeperWatcher(HBaseZKTestingUtility testUtil)
-      throws ZooKeeperConnectionException, IOException {
+  public static ZKWatcher getZooKeeperWatcher(HBaseZKTestingUtility testUtil) throws IOException {
     ZKWatcher zkw = new ZKWatcher(testUtil.getConfiguration(), "unittest", new Abortable() {
       boolean aborted = false;
 


[07/48] hbase git commit: HBASE-19789 Exclude flaky tests from nightly branch-2 runs

Posted by zh...@apache.org.
HBASE-19789 Exclude flaky tests from nightly branch-2 runs


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: f91589d3056f69a0eeac5bfd2736c271ec76ba95
Parents: 026f535
Author: Apekshit Sharma <ap...@apache.org>
Authored: Fri Jan 12 16:22:06 2018 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Fri Jan 12 16:24:45 2018 -0800

----------------------------------------------------------------------
 dev-support/Jenkinsfile            | 6 ++++--
 dev-support/hbase_nightly_yetus.sh | 7 +++++--
 2 files changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f91589d3/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index dcef649..e5d33e0 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -44,8 +44,10 @@ pipeline {
     // These tests currently have known failures. Once they burn down to 0, remove from here so that new problems will cause a failure.
     TESTS_FILTER = 'cc,checkstyle,javac,javadoc,pylint,shellcheck,whitespace,perlcritic,ruby-lint,rubocop,mvnsite'
     BRANCH_SPECIFIC_DOCKERFILE = "${env.BASEDIR}/dev-support/docker/Dockerfile"
-    // Only used for master branch.
-    EXCLUDE_TESTS_URL = 'https://builds.apache.org/job/HBase-Find-Flaky-Tests/lastSuccessfulBuild/artifact/excludes/'
+    // Flaky urls for different branches. Replace '-' and '.' in branch name by '_' because those
+    // characters are not allowed in bash variable name.
+    EXCLUDE_TESTS_URL_master = 'https://builds.apache.org/job/HBase-Find-Flaky-Tests/lastSuccessfulBuild/artifact/excludes/'
+    EXCLUDE_TESTS_URL_branch_2 = 'https://builds.apache.org/job/HBase-Find-Flaky-Tests-branch2.0/lastSuccessfulBuild/artifact/excludes/'
   }
   parameters {
     booleanParam(name: 'USE_YETUS_PRERELEASE', defaultValue: false, description: '''Check to use the current HEAD of apache/yetus rather than our configured release.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f91589d3/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index e1175d2..4e67354 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -69,10 +69,13 @@ YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
 
 # Currently, flaky list is calculated only for master branch.
-if [[ -n "${EXCLUDE_TESTS_URL}" && "${BRANCH_NAME}" == "master" ]]; then
+UNDERSCORED_BRANCH_NAME=$(echo ${BRANCH_NAME} | tr '.-' '_')
+EXCLUDE_TESTS_URL=$(eval echo "\$EXCLUDE_TESTS_URL_${UNDERSCORED_BRANCH_NAME}")
+INCLUDE_TESTS_URL=$(eval echo "\$INCLUDE_TESTS_URL_${UNDERSCORED_BRANCH_NAME}")
+if [[ -n "${EXCLUDE_TESTS_URL}" ]]; then
   YETUS_ARGS=("--exclude-tests-url=${EXCLUDE_TESTS_URL}" "${YETUS_ARGS[@]}")
 fi
-if [[ -n "${INCLUDE_TESTS_URL}" && "${BRANCH_NAME}" == "master" ]]; then
+if [[ -n "${INCLUDE_TESTS_URL}" ]]; then
   YETUS_ARGS=("--include-tests-url=${INCLUDE_TESTS_URL}" "${YETUS_ARGS[@]}")
 fi
 


[06/48] hbase git commit: HBASE-19787 Fix or disable tests broken in branch-2 so can cut beta-1

Posted by zh...@apache.org.
HBASE-19787 Fix or disable tests broken in branch-2 so can cut beta-1

M dev-support/make_rc.sh
  Disable checkstyle building site. Its an issue being fixed over in HBASE-19780

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
  The clusterid was being set into the process only after the
  regionserver registers with the Master. That can be too late for some
  test clients in particular. e.g. TestZKAsyncRegistry needs it as soon
  as it goes to run which could be before Master had called its run
  method  which is regionserver run method which then calls back to the
  master to register itself... and only then do we set the clusterid.
  HBASE-19694 changed start order which made it so this test failed.
  Setting the clusterid right after we set it in zk makes the test pass.

  Another change was that backup masters were not going down on stop.
  Backup masters were sleeping for the default zk period which is 90
  seconds. They were not being woken up to check for stop. On stop
  master now tells active master manager.

M hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
  Prevent creation of acl table. Messes up our being able to go down
  promptly.

M hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java
M hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
M hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
  Disabled for now because it wants to run with regions on the Master...
  currently broke!

M hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
  Add a bit of debugging.

M hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
  Disabled. Fails 40% of the time.

M hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
  Disabled. Fails 33% of the time.

Disabled stochastic load balancer for favored nodes because it fails on
occasion and we are not doing favored nodes in branch-2.


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 026f535a7747b89003252ded9585e827686aa79f
Parents: d8271b0
Author: Michael Stack <st...@apache.org>
Authored: Fri Jan 12 13:39:32 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Jan 12 14:09:23 2018 -0800

----------------------------------------------------------------------
 dev-support/make_rc.sh                          |  7 +++++--
 .../org/apache/hadoop/hbase/master/HMaster.java | 21 ++++++++++++++------
 .../hbase/regionserver/HRegionServer.java       |  2 +-
 .../hadoop/hbase/util/JVMClusterUtil.java       |  1 -
 .../hadoop/hbase/HBaseTestingUtility.java       |  2 +-
 .../hadoop/hbase/TestJMXConnectorServer.java    | 10 +++++++++-
 .../hadoop/hbase/client/TestMultiParallel.java  |  2 ++
 .../hbase/client/TestZKAsyncRegistry.java       | 12 ++++++++---
 .../hadoop/hbase/master/TestDLSAsyncFSWAL.java  |  2 ++
 .../hadoop/hbase/master/TestDLSFSHLog.java      |  2 ++
 .../hbase/master/TestTableStateManager.java     |  2 +-
 .../TestFavoredStochasticLoadBalancer.java      |  1 +
 .../balancer/TestRegionsOnMasterOptions.java    |  5 ++++-
 .../TestRegionServerReadRequestMetrics.java     |  4 +++-
 14 files changed, 55 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/dev-support/make_rc.sh
----------------------------------------------------------------------
diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh
index 8bfdde2..f067ee9 100755
--- a/dev-support/make_rc.sh
+++ b/dev-support/make_rc.sh
@@ -75,9 +75,12 @@ function build_src {
 
 # Build bin tgz
 function build_bin {
-  MAVEN_OPTS="${mvnopts}" ${mvn} clean install -DskipTests -Papache-release -Prelease \
+  MAVEN_OPTS="${mvnopts}" ${mvn} clean install -DskipTests \
+    -Papache-release -Prelease \
     -Dmaven.repo.local=${output_dir}/repository
-  MAVEN_OPTS="${mvnopts}" ${mvn} install -DskipTests site assembly:single -Papache-release -Prelease \
+  MAVEN_OPTS="${mvnopts}" ${mvn} install -DskipTests \
+    -Dcheckstyle.skip=true site assembly:single \
+    -Papache-release -Prelease \
     -Dmaven.repo.local=${output_dir}/repository
   mv ./hbase-assembly/target/hbase-*.tar.gz "${output_dir}"
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/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 ee7cd18..971ff08 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
@@ -55,6 +55,7 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
@@ -796,9 +797,13 @@ public class HMaster extends HRegionServer implements MasterServices {
       this.tableDescriptors.getAll();
     }
 
-    // Publish cluster ID
-    status.setStatus("Publishing Cluster ID in ZooKeeper");
+    // Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
+    // only after it has checked in with the Master. At least a few tests ask Master for clusterId
+    // before it has called its run method and before RegionServer has done the reportForDuty.
+    ClusterId clusterId = fileSystemManager.getClusterId();
+    status.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
     ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
+    this.clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
 
     this.serverManager = createServerManager(this);
 
@@ -845,10 +850,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.balancer instanceof FavoredNodesPromoter) {
       favoredNodesManager = new FavoredNodesManager(this);
     }
-    // Wait for regionserver to finish initialization.
-    if (LoadBalancer.isTablesOnMaster(conf)) {
-      waitForServerOnline();
-    }
 
     //initialize load balancer
     this.balancer.setMasterServices(this);
@@ -2692,6 +2693,14 @@ public class HMaster extends HRegionServer implements MasterServices {
     stop("Stopped by " + Thread.currentThread().getName());
   }
 
+  @Override
+  public void stop(String msg) {
+    super.stop(msg);
+    if (this.activeMasterManager != null) {
+      this.activeMasterManager.stop();
+    }
+  }
+
   void checkServiceStarted() throws ServerNotRunningYetException {
     if (!serviceStarted) {
       throw new ServerNotRunningYetException("Server is not running yet");

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/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 3a52a16..8e91702 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
@@ -444,7 +444,7 @@ public class HRegionServer extends HasThread implements
   /**
    * Unique identifier for the cluster we are a part of.
    */
-  private String clusterId;
+  protected String clusterId;
 
   /**
    * MX Bean for RegionServerInfo

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
index 1ac790f..a85e89e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
@@ -258,7 +258,6 @@ public class JVMClusterUtil {
           LOG.error("Exception occurred in HMaster.shutdown()", e);
         }
       }
-
     }
     boolean wasInterrupted = false;
     final long maxTime = System.currentTimeMillis() + 30 * 1000;

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/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 fe8902a..a686e33 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
@@ -1033,7 +1033,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     t.close();
 
     getAdmin(); // create immediately the hbaseAdmin
-    LOG.info("Minicluster is up");
+    LOG.info("Minicluster is up; activeMaster=" + this.getHBaseCluster().getMaster());
 
     return (MiniHBaseCluster)this.hbaseCluster;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
index 444db64..d09b0a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -31,9 +31,11 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -182,6 +184,12 @@ public class TestJMXConnectorServer {
    */
   public static class MyAccessController extends AccessController {
     @Override
+    public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+      // Do nothing. In particular, stop the creation of the hbase:acl table. It makes the
+      // shutdown take time.
+    }
+
+    @Override
     public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c) throws IOException {
       if (!hasAccess) {
         throw new AccessDeniedException("Insufficient permissions to stop master");

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index 49e0a98..a81447b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -58,11 +58,13 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@Ignore // Depends on Master being able to host regions. Needs fixing.
 @Category({MediumTests.class, FlakeyTests.class})
 public class TestMultiParallel {
   private static final Logger LOG = LoggerFactory.getLogger(TestMultiParallel.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
index efdc53f..3478549 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
@@ -30,6 +30,7 @@ import java.util.stream.IntStream;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -43,10 +44,12 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Category({ MediumTests.class, ClientTests.class })
 public class TestZKAsyncRegistry {
-
+  private static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private static ZKAsyncRegistry REGISTRY;
@@ -96,8 +99,11 @@ public class TestZKAsyncRegistry {
 
   @Test
   public void test() throws InterruptedException, ExecutionException, IOException {
-    assertEquals(TEST_UTIL.getHBaseCluster().getClusterStatus().getClusterId(),
-      REGISTRY.getClusterId().get());
+    LOG.info("STARTED TEST");
+    String clusterId = REGISTRY.getClusterId().get();
+    String expectedClusterId = TEST_UTIL.getHBaseCluster().getMaster().getClusterId();
+    assertEquals("Expected " + expectedClusterId + ", found=" + clusterId,
+        expectedClusterId, clusterId);
     assertEquals(TEST_UTIL.getHBaseCluster().getClusterStatus().getServersSize(),
       REGISTRY.getCurrentNrHRS().get().intValue());
     assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
index 36e7456..c4a2fce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
@@ -19,8 +19,10 @@ package org.apache.hadoop.hbase.master;
 
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 
+@Ignore // Temporarily disabled. Fails 40% of the time.
 @Category({ MasterTests.class, LargeTests.class })
 public class TestDLSAsyncFSWAL extends AbstractTestDLS {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
index e83834f..a732962 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
@@ -19,8 +19,10 @@ package org.apache.hadoop.hbase.master;
 
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 
+@Ignore // Fails 33% of the time. Disabling for now.
 @Category({ MasterTests.class, LargeTests.class })
 public class TestDLSFSHLog extends AbstractTestDLS {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/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 81c1dfc..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
@@ -58,7 +58,7 @@ public class TestTableStateManager {
   @Test(timeout = 60000)
   public void testUpgradeFromZk() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    TEST_UTIL.startMiniCluster(1, 1);
+    TEST_UTIL.startMiniCluster(2, 1);
     TEST_UTIL.shutdownMiniHBaseCluster();
     ZKWatcher watcher = TEST_UTIL.getZooKeeperWatcher();
     setTableStateInZK(watcher, tableName, ZooKeeperProtos.DeprecatedTableState.State.DISABLED);

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
index 89fe35a..4c23771 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
@@ -68,6 +68,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
+@Ignore // Disabled
 @Category(MediumTests.class)
 public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java
index a8e78d7..8f06886 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -102,6 +103,8 @@ public class TestRegionsOnMasterOptions {
     checkBalance(0, rsCount);
   }
 
+  @Ignore // Fix this. The Master startup doesn't allow Master reporting as a RegionServer, not
+  // until way late after the Master startup finishes. Needs more work.
   @Test
   public void testSystemTablesOnMaster() throws Exception {
     c.setBoolean(LoadBalancer.TABLES_ON_MASTER, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/026f535a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
index 536fdb2..c5afefa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -63,11 +63,13 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@Ignore // Depends on Master being able to host regions. Needs fixing.
 @Category(MediumTests.class)
 public class TestRegionServerReadRequestMetrics {
   private static final Logger LOG =


[30/48] hbase git commit: HBASE-19642 Fix locking for peer modification procedure

Posted by zh...@apache.org.
HBASE-19642 Fix locking for peer modification procedure


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: d798b81083d62c50c50de28ada32c0eaa225ae9c
Parents: 67c319e
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 27 18:27:13 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../procedure/MasterProcedureScheduler.java     | 14 +++++++++++++
 .../master/replication/ModifyPeerProcedure.java | 21 +++++++++++++++++---
 2 files changed, 32 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d798b810/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index a25217c..4ecb3b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -610,6 +610,20 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     public boolean requireExclusiveLock(Procedure proc) {
       return requirePeerExclusiveLock((PeerProcedureInterface) proc);
     }
+
+    @Override
+    public boolean isAvailable() {
+      if (isEmpty()) {
+        return false;
+      }
+      if (getLockStatus().hasExclusiveLock()) {
+        // if we have an exclusive lock already taken
+        // only child of the lock owner can be executed
+        Procedure nextProc = peek();
+        return nextProc != null && getLockStatus().hasLockAccess(nextProc);
+      }
+      return true;
+    }
   }
 
   // ============================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/d798b810/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 279fbc7..a682606 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -46,6 +46,8 @@ public abstract class ModifyPeerProcedure
 
   protected String peerId;
 
+  private volatile boolean locked;
+
   // used to keep compatible with old client where we can only returns after updateStorage.
   protected ProcedurePrepareLatch latch;
 
@@ -145,17 +147,30 @@ public abstract class ModifyPeerProcedure
 
   @Override
   protected LockState acquireLock(MasterProcedureEnv env) {
-    return env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)
-      ? LockState.LOCK_EVENT_WAIT
-      : LockState.LOCK_ACQUIRED;
+    if (env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)) {
+      return  LockState.LOCK_EVENT_WAIT;
+    }
+    locked = true;
+    return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(MasterProcedureEnv env) {
+    locked = false;
     env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId);
   }
 
   @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+    return true;
+  }
+
+  @Override
+  protected boolean hasLock(MasterProcedureEnv env) {
+    return locked;
+  }
+
+  @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
       throws IOException, InterruptedException {
     if (state == PeerModificationState.PRE_PEER_MODIFICATION) {


[15/48] hbase git commit: HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface

Posted by zh...@apache.org.
HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 9a51a84bb687292316b7eb885c9efcd162bac0c4
Parents: 8d0b8b9
Author: huzheng <op...@gmail.com>
Authored: Tue Dec 26 16:46:10 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfigUtil.java  |  10 +-
 .../replication/VerifyReplication.java          |   9 +-
 .../hbase/replication/ReplicationFactory.java   |  10 +-
 .../hbase/replication/ReplicationPeerImpl.java  |  60 +-
 .../replication/ReplicationPeerStorage.java     |   3 +-
 .../hbase/replication/ReplicationPeers.java     | 238 ++++----
 .../replication/ReplicationPeersZKImpl.java     | 552 -------------------
 .../replication/ZKReplicationPeerStorage.java   |  12 +-
 .../replication/ZKReplicationStorageBase.java   |   3 +-
 .../replication/TestReplicationStateBasic.java  | 125 ++---
 .../replication/TestReplicationStateZKImpl.java |   2 +-
 .../TestZKReplicationPeerStorage.java           |  12 +-
 .../cleaner/ReplicationZKNodeCleaner.java       |  57 +-
 .../replication/ReplicationPeerManager.java     |   6 +-
 .../regionserver/DumpReplicationQueues.java     |   2 +-
 .../regionserver/PeerProcedureHandlerImpl.java  |  49 +-
 .../replication/regionserver/Replication.java   |   2 +-
 .../regionserver/ReplicationSource.java         |   6 +-
 .../regionserver/ReplicationSourceManager.java  |  45 +-
 .../cleaner/TestReplicationHFileCleaner.java    |   7 +-
 .../replication/TestMultiSlaveReplication.java  |   2 -
 .../TestReplicationTrackerZKImpl.java           |  36 +-
 .../TestReplicationSourceManager.java           |  17 +-
 .../hadoop/hbase/HBaseZKTestingUtility.java     |   3 +-
 24 files changed, 308 insertions(+), 960 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index 022bf64..a234a9b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -247,22 +247,22 @@ public final class ReplicationPeerConfigUtil {
   public static ReplicationPeerConfig parsePeerFrom(final byte[] bytes)
       throws DeserializationException {
     if (ProtobufUtil.isPBMagicPrefix(bytes)) {
-      int pblen = ProtobufUtil.lengthOfPBMagic();
+      int pbLen = ProtobufUtil.lengthOfPBMagic();
       ReplicationProtos.ReplicationPeer.Builder builder =
           ReplicationProtos.ReplicationPeer.newBuilder();
       ReplicationProtos.ReplicationPeer peer;
       try {
-        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+        ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
         peer = builder.build();
       } catch (IOException e) {
         throw new DeserializationException(e);
       }
       return convert(peer);
     } else {
-      if (bytes.length > 0) {
-        return ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build();
+      if (bytes == null || bytes.length <= 0) {
+        throw new DeserializationException("Bytes to deserialize should not be empty.");
       }
-      return ReplicationPeerConfig.newBuilder().setClusterKey("").build();
+      return ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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 09d4b4b..f0070f0 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
@@ -339,15 +339,10 @@ public class VerifyReplication extends Configured implements Tool {
             @Override public boolean isAborted() {return false;}
           });
 
-      ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf, localZKW);
+      ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf);
       rp.init();
 
-      Pair<ReplicationPeerConfig, Configuration> pair = rp.getPeerConf(peerId);
-      if (pair == null) {
-        throw new IOException("Couldn't get peer conf!");
-      }
-
-      return pair;
+      return Pair.newPair(rp.getPeerConfig(peerId), rp.getPeerClusterConfiguration(peerId));
     } catch (ReplicationException e) {
       throw new IOException(
           "An error occurred while trying to connect to the remove peer cluster", e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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 5e70e57..6c66aff 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
@@ -29,14 +29,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
-      Abortable abortable) {
-    return getReplicationPeers(zk, conf, null, abortable);
-  }
-
-  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
-      ReplicationQueueStorage queueStorage, Abortable abortable) {
-    return new ReplicationPeersZKImpl(zk, conf, queueStorage, abortable);
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf) {
+    return new ReplicationPeers(zk, conf);
   }
 
   public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
index 2c7ea9b..3e17025 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
@@ -18,28 +18,16 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 
 @InterfaceAudience.Private
 public class ReplicationPeerImpl implements ReplicationPeer {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerImpl.class);
-
-  private final ReplicationPeerStorage peerStorage;
-
   private final Configuration conf;
 
   private final String id;
@@ -57,21 +45,21 @@ public class ReplicationPeerImpl implements ReplicationPeer {
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerImpl(ZKWatcher zkWatcher, Configuration conf, String id,
+  public ReplicationPeerImpl(Configuration conf, String id, boolean peerState,
       ReplicationPeerConfig peerConfig) {
-    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkWatcher, conf);
     this.conf = conf;
-    this.peerConfig = peerConfig;
     this.id = id;
+    this.peerState = peerState ? PeerState.ENABLED : PeerState.DISABLED;
+    this.peerConfig = peerConfig;
     this.peerConfigListeners = new ArrayList<>();
   }
 
-  public void refreshPeerState() throws ReplicationException {
-    this.peerState = peerStorage.isPeerEnabled(id) ? PeerState.ENABLED : PeerState.DISABLED;
+  void setPeerState(boolean enabled) {
+    this.peerState = enabled ? PeerState.ENABLED : PeerState.DISABLED;
   }
 
-  public void refreshPeerConfig() throws ReplicationException {
-    this.peerConfig = peerStorage.getPeerConfig(id).orElse(peerConfig);
+  void setPeerConfig(ReplicationPeerConfig peerConfig) {
+    this.peerConfig = peerConfig;
     peerConfigListeners.forEach(listener -> listener.peerConfigUpdated(peerConfig));
   }
 
@@ -134,36 +122,4 @@ public class ReplicationPeerImpl implements ReplicationPeer {
   public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
     this.peerConfigListeners.add(listener);
   }
-
-  /**
-   * Parse the raw data from ZK to get a peer's state
-   * @param bytes raw ZK data
-   * @return True if the passed in <code>bytes</code> are those of a pb serialized ENABLED state.
-   * @throws DeserializationException
-   */
-  public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
-    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
-    return ReplicationProtos.ReplicationState.State.ENABLED == state;
-  }
-
-  /**
-   * @param bytes Content of a state znode.
-   * @return State parsed from the passed bytes.
-   * @throws DeserializationException
-   */
-  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
-      throws DeserializationException {
-    ProtobufUtil.expectPBMagicPrefix(bytes);
-    int pbLen = ProtobufUtil.lengthOfPBMagic();
-    ReplicationProtos.ReplicationState.Builder builder =
-        ReplicationProtos.ReplicationState.newBuilder();
-    ReplicationProtos.ReplicationState state;
-    try {
-      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
-      state = builder.build();
-      return state.getState();
-    } catch (IOException e) {
-      throw new DeserializationException(e);
-    }
-  }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
index e00cd0d..1adda02 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
-import java.util.Optional;
 
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -70,5 +69,5 @@ public interface ReplicationPeerStorage {
    * Get the peer config of a replication peer.
    * @throws ReplicationException if there are errors accessing the storage service.
    */
-  Optional<ReplicationPeerConfig> getPeerConfig(String peerId) throws ReplicationException;
+  ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index afc19bd..e58482e 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -18,58 +17,53 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
+import java.io.IOException;
 import java.util.Set;
-
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.CompoundConfiguration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
- * This provides an interface for maintaining a set of peer clusters. These peers are remote slave
- * clusters that data is replicated to. A peer cluster can be in three different states:
- *
- * 1. Not-Registered - There is no notion of the peer cluster.
- * 2. Registered - The peer has an id and is being tracked but there is no connection.
- * 3. Connected - There is an active connection to the remote peer.
- *
- * In the registered or connected state, a peer cluster can either be enabled or disabled.
+ * This provides an class for maintaining a set of peer clusters. These peers are remote slave
+ * clusters that data is replicated to.
  */
 @InterfaceAudience.Private
-public interface ReplicationPeers {
+public class ReplicationPeers {
 
-  /**
-   * Initialize the ReplicationPeers interface.
-   */
-  void init() throws ReplicationException;
+  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeers.class);
 
-  /**
-   * Add a new remote slave cluster for replication.
-   * @param peerId a short that identifies the cluster
-   * @param peerConfig configuration for the replication slave cluster
-   */
-  default void registerPeer(String peerId, ReplicationPeerConfig peerConfig)
-      throws ReplicationException {
-    registerPeer(peerId, peerConfig, true);
+  private final Configuration conf;
+
+  // Map of peer clusters keyed by their id
+  private final ConcurrentMap<String, ReplicationPeerImpl> peerCache;
+  private final ReplicationPeerStorage peerStorage;
+
+  protected ReplicationPeers(ZKWatcher zookeeper, Configuration conf) {
+    this.conf = conf;
+    this.peerCache = new ConcurrentHashMap<>();
+    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zookeeper, conf);
   }
 
-  /**
-   * Add a new remote slave cluster for replication.
-   * @param peerId a short that identifies the cluster
-   * @param peerConfig configuration for the replication slave cluster
-   * @param enabled peer state, true if ENABLED and false if DISABLED
-   */
-  void registerPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException;
+  public void init() throws ReplicationException {
+    // Loading all existing peerIds into peer cache.
+    for (String peerId : this.peerStorage.listPeerIds()) {
+      addPeer(peerId);
+    }
+  }
 
-  /**
-   * Removes a remote slave cluster and stops the replication to it.
-   * @param peerId a short that identifies the cluster
-   */
-  void unregisterPeer(String peerId) throws ReplicationException;
+  @VisibleForTesting
+  public ReplicationPeerStorage getPeerStorage() {
+    return this.peerStorage;
+  }
 
   /**
    * Method called after a peer has been connected. It will create a ReplicationPeer to track the
@@ -78,111 +72,115 @@ public interface ReplicationPeers {
    * @return whether a ReplicationPeer was successfully created
    * @throws ReplicationException
    */
-  boolean peerConnected(String peerId) throws ReplicationException;
-
-  /**
-   * Method called after a peer has been disconnected. It will remove the ReplicationPeer that
-   * tracked the disconnected cluster.
-   * @param peerId a short that identifies the cluster
-   */
-  void peerDisconnected(String peerId);
+  public boolean addPeer(String peerId) throws ReplicationException {
+    if (this.peerCache.containsKey(peerId)) {
+      return false;
+    }
 
-  /**
-   * Restart the replication to the specified remote slave cluster.
-   * @param peerId a short that identifies the cluster
-   */
-  void enablePeer(String peerId) throws ReplicationException;
-
-  /**
-   * Stop the replication to the specified remote slave cluster.
-   * @param peerId a short that identifies the cluster
-   */
-  void disablePeer(String peerId) throws ReplicationException;
+    peerCache.put(peerId, createPeer(peerId));
+    return true;
+  }
 
-  /**
-   * Get the table and column-family list string of the peer from the underlying storage.
-   * @param peerId a short that identifies the cluster
-   */
-  public Map<TableName, List<String>> getPeerTableCFsConfig(String peerId)
-      throws ReplicationException;
+  public void removePeer(String peerId) {
+    peerCache.remove(peerId);
+  }
 
   /**
-   * Set the table and column-family list string of the peer to the underlying storage.
+   * Get the peer state for the specified connected remote slave cluster. The value might be read
+   * from cache, so it is recommended to use {@link #peerStorage } to read storage directly if
+   * reading the state after enabling or disabling it.
    * @param peerId a short that identifies the cluster
-   * @param tableCFs the table and column-family list which will be replicated for this peer
+   * @return true if replication is enabled, false otherwise.
    */
-  public void setPeerTableCFsConfig(String peerId,
-                                    Map<TableName, ? extends Collection<String>>  tableCFs)
-      throws ReplicationException;
+  public boolean isPeerEnabled(String peerId) {
+    ReplicationPeer replicationPeer = this.peerCache.get(peerId);
+    if (replicationPeer == null) {
+      throw new IllegalArgumentException("Peer with id= " + peerId + " is not cached");
+    }
+    return replicationPeer.getPeerState() == PeerState.ENABLED;
+  }
 
   /**
-   * Returns the ReplicationPeerImpl for the specified connected peer. This ReplicationPeer will
-   * continue to track changes to the Peer's state and config. This method returns null if no
-   * peer has been connected with the given peerId.
+   * Returns the ReplicationPeerImpl for the specified cached peer. This ReplicationPeer will
+   * continue to track changes to the Peer's state and config. This method returns null if no peer
+   * has been cached with the given peerId.
    * @param peerId id for the peer
    * @return ReplicationPeer object
    */
-  ReplicationPeerImpl getConnectedPeer(String peerId);
+  public ReplicationPeerImpl getPeer(String peerId) {
+    return peerCache.get(peerId);
+  }
 
   /**
    * Returns the set of peerIds of the clusters that have been connected and have an underlying
    * ReplicationPeer.
    * @return a Set of Strings for peerIds
    */
-  public Set<String> getConnectedPeerIds();
+  public Set<String> getAllPeerIds() {
+    return peerCache.keySet();
+  }
 
-  /**
-   * Get the replication status for the specified connected remote slave cluster.
-   * The value might be read from cache, so it is recommended to
-   * use {@link #getStatusOfPeerFromBackingStore(String)}
-   * if reading the state after enabling or disabling it.
-   * @param peerId a short that identifies the cluster
-   * @return true if replication is enabled, false otherwise.
-   */
-  boolean getStatusOfPeer(String peerId);
+  public ReplicationPeerConfig getPeerConfig(String peerId) {
+    ReplicationPeer replicationPeer = this.peerCache.get(peerId);
+    if (replicationPeer == null) {
+      throw new IllegalArgumentException("Peer with id= " + peerId + " is not cached");
+    }
+    return replicationPeer.getPeerConfig();
+  }
 
-  /**
-   * Get the replication status for the specified remote slave cluster, which doesn't
-   * have to be connected. The state is read directly from the backing store.
-   * @param peerId a short that identifies the cluster
-   * @return true if replication is enabled, false otherwise.
-   * @throws ReplicationException thrown if there's an error contacting the store
-   */
-  boolean getStatusOfPeerFromBackingStore(String peerId) throws ReplicationException;
+  public Configuration getPeerClusterConfiguration(String peerId) throws ReplicationException {
+    ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
 
-  /**
-   * List the cluster replication configs of all remote slave clusters (whether they are
-   * enabled/disabled or connected/disconnected).
-   * @return A map of peer ids to peer cluster keys
-   */
-  Map<String, ReplicationPeerConfig> getAllPeerConfigs();
+    Configuration otherConf;
+    try {
+      otherConf = HBaseConfiguration.createClusterConf(this.conf, peerConfig.getClusterKey());
+    } catch (IOException e) {
+      throw new ReplicationException("Can't get peer configuration for peerId=" + peerId, e);
+    }
 
-  /**
-   * List the peer ids of all remote slave clusters (whether they are enabled/disabled or
-   * connected/disconnected).
-   * @return A list of peer ids
-   */
-  List<String> getAllPeerIds();
+    if (!peerConfig.getConfiguration().isEmpty()) {
+      CompoundConfiguration compound = new CompoundConfiguration();
+      compound.add(otherConf);
+      compound.addStringMap(peerConfig.getConfiguration());
+      return compound;
+    }
 
-  /**
-   * Returns the configured ReplicationPeerConfig for this peerId
-   * @param peerId a short name that identifies the cluster
-   * @return ReplicationPeerConfig for the peer
-   */
-  ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws ReplicationException;
+    return otherConf;
+  }
 
-  /**
-   * Returns the configuration needed to talk to the remote slave cluster.
-   * @param peerId a short that identifies the cluster
-   * @return the configuration for the peer cluster, null if it was unable to get the configuration
-   */
-  Pair<ReplicationPeerConfig, Configuration> getPeerConf(String peerId) throws ReplicationException;
+  public PeerState refreshPeerState(String peerId) throws ReplicationException {
+    ReplicationPeerImpl peer = peerCache.get(peerId);
+    if (peer == null) {
+      throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
+    }
+    peer.setPeerState(peerStorage.isPeerEnabled(peerId));
+    return peer.getPeerState();
+  }
+
+  public ReplicationPeerConfig refreshPeerConfig(String peerId) throws ReplicationException {
+    ReplicationPeerImpl peer = peerCache.get(peerId);
+    if (peer == null) {
+      throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
+    }
+    peer.setPeerConfig(peerStorage.getPeerConfig(peerId));
+    return peer.getPeerConfig();
+  }
 
   /**
+<<<<<<< 2bb2fd611d4b88c724a2b561f10433b56c6fd3dd
    * Update the peerConfig for the a given peer cluster
    * @param id a short that identifies the cluster
    * @param peerConfig new config for the peer cluster
    * @throws ReplicationException
-   */
-  void updatePeerConfig(String id, ReplicationPeerConfig peerConfig) throws ReplicationException;
+=======
+   * Helper method to connect to a peer
+   * @param peerId peer's identifier
+   * @return object representing the peer
+>>>>>>> HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface
+   */
+  private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
+    ReplicationPeerConfig peerConf = peerStorage.getPeerConfig(peerId);
+    boolean enabled = peerStorage.isPeerEnabled(peerId);
+    return new ReplicationPeerImpl(getPeerClusterConfiguration(peerId), peerId, enabled, peerConf);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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
deleted file mode 100644
index 7f6498d..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ /dev/null
@@ -1,552 +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.replication;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.CompoundConfiguration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
-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.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides an implementation of the ReplicationPeers interface using ZooKeeper. The
- * peers znode contains a list of all peer replication clusters and the current replication state of
- * those clusters. It has one child peer znode for each peer cluster. The peer znode is named with
- * the cluster id provided by the user in the HBase shell. The value of the peer znode contains the
- * peers cluster key provided by the user in the HBase Shell. The cluster key contains a list of
- * zookeeper quorum peers, the client port for the zookeeper quorum, and the base znode for HBase.
- * For example:
- *
- *  /hbase/replication/peers/1 [Value: zk1.host.com,zk2.host.com,zk3.host.com:2181:/hbase]
- *  /hbase/replication/peers/2 [Value: zk5.host.com,zk6.host.com,zk7.host.com:2181:/hbase]
- *
- * Each of these peer znodes has a child znode that indicates whether or not replication is enabled
- * on that peer cluster. These peer-state znodes do not have child znodes and simply contain a
- * boolean value (i.e. ENABLED or DISABLED). This value is read/maintained by the
- * ReplicationPeer.PeerStateTracker class. For example:
- *
- * /hbase/replication/peers/1/peer-state [Value: ENABLED]
- *
- * Each of these peer znodes has a child znode that indicates which data will be replicated
- * to the peer cluster. These peer-tableCFs znodes do not have child znodes and only have a
- * table/cf list config. This value is read/maintained by the ReplicationPeer.TableCFsTracker
- * class. For example:
- *
- * /hbase/replication/peers/1/tableCFs [Value: "table1; table2:cf1,cf3; table3:cfx,cfy"]
- */
-@InterfaceAudience.Private
-public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements ReplicationPeers {
-
-  // Map of peer clusters keyed by their id
-  private ConcurrentMap<String, ReplicationPeerImpl> peerClusters;
-  private final ReplicationQueueStorage queueStorage;
-  private Abortable abortable;
-
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeersZKImpl.class);
-
-  public ReplicationPeersZKImpl(ZKWatcher zk, Configuration conf,
-      ReplicationQueueStorage queueStorage, Abortable abortable) {
-    super(zk, conf, abortable);
-    this.abortable = abortable;
-    this.peerClusters = new ConcurrentHashMap<>();
-    this.queueStorage = queueStorage;
-  }
-
-  @Override
-  public void init() throws ReplicationException {
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, this.peersZNode) < 0) {
-        ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Could not initialize replication peers", e);
-    }
-    addExistingPeers();
-  }
-
-  @Override
-  public void registerPeer(String id, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException {
-    try {
-      if (peerExists(id)) {
-        throw new IllegalArgumentException("Cannot add a peer with id=" + id
-            + " because that id already exists.");
-      }
-
-      if(id.contains("-")){
-        throw new IllegalArgumentException("Found invalid peer name:" + id);
-      }
-
-      if (peerConfig.getClusterKey() != null) {
-        try {
-          ZKConfig.validateClusterKey(peerConfig.getClusterKey());
-        } catch (IOException ioe) {
-          throw new IllegalArgumentException(ioe.getMessage());
-        }
-      }
-
-      checkQueuesDeleted(id);
-
-      ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
-
-      List<ZKUtilOp> listOfOps = new ArrayList<>(2);
-      ZKUtilOp op1 =
-          ZKUtilOp.createAndFailSilent(getPeerNode(id),
-            ReplicationPeerConfigUtil.toByteArray(peerConfig));
-      ZKUtilOp op2 =
-          ZKUtilOp.createAndFailSilent(getPeerStateNode(id), enabled ? ENABLED_ZNODE_BYTES
-              : DISABLED_ZNODE_BYTES);
-      listOfOps.add(op1);
-      listOfOps.add(op2);
-      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
-    } catch (KeeperException e) {
-      throw new ReplicationException("Could not add peer with id=" + id + ", peerConfif=>"
-          + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e);
-    }
-  }
-
-  @Override
-  public void unregisterPeer(String id) throws ReplicationException {
-    try {
-      if (!peerExists(id)) {
-        throw new IllegalArgumentException("Cannot remove peer with id=" + id
-            + " because that id does not exist.");
-      }
-      ZKUtil.deleteNodeRecursively(this.zookeeper, ZNodePaths.joinZNode(this.peersZNode, id));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Could not remove peer with id=" + id, e);
-    }
-  }
-
-  @Override
-  public void enablePeer(String id) throws ReplicationException {
-    changePeerState(id, ReplicationProtos.ReplicationState.State.ENABLED);
-    LOG.info("peer " + id + " is enabled");
-  }
-
-  @Override
-  public void disablePeer(String id) throws ReplicationException {
-    changePeerState(id, ReplicationProtos.ReplicationState.State.DISABLED);
-    LOG.info("peer " + id + " is disabled");
-  }
-
-  @Override
-  public Map<TableName, List<String>> getPeerTableCFsConfig(String id) throws ReplicationException {
-    try {
-      if (!peerExists(id)) {
-        throw new IllegalArgumentException("peer " + id + " doesn't exist");
-      }
-      try {
-        ReplicationPeerConfig rpc = getReplicationPeerConfig(id);
-        if (rpc == null) {
-          throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id);
-        }
-        return rpc.getTableCFsMap();
-      } catch (Exception e) {
-        throw new ReplicationException(e);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id, e);
-    }
-  }
-
-  @Override
-  public void setPeerTableCFsConfig(String id,
-                                    Map<TableName, ? extends Collection<String>>  tableCFs)
-      throws ReplicationException {
-    try {
-      if (!peerExists(id)) {
-        throw new IllegalArgumentException("Cannot set peer tableCFs because id=" + id
-            + " does not exist.");
-      }
-      ReplicationPeerConfig rpc = getReplicationPeerConfig(id);
-      if (rpc == null) {
-        throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id);
-      }
-      rpc.setTableCFsMap(tableCFs);
-      ZKUtil.setData(this.zookeeper, getPeerNode(id),
-          ReplicationPeerConfigUtil.toByteArray(rpc));
-      LOG.info("Peer tableCFs with id= " + id + " is now " +
-        ReplicationPeerConfigUtil.convertToString(tableCFs));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e);
-    }
-  }
-
-  @Override
-  public boolean getStatusOfPeer(String id) {
-    ReplicationPeer replicationPeer = this.peerClusters.get(id);
-    if (replicationPeer == null) {
-      throw new IllegalArgumentException("Peer with id= " + id + " is not cached");
-    }
-    return replicationPeer.getPeerState() == PeerState.ENABLED;
-  }
-
-  @Override
-  public boolean getStatusOfPeerFromBackingStore(String id) throws ReplicationException {
-    try {
-      if (!peerExists(id)) {
-        throw new IllegalArgumentException("peer " + id + " doesn't exist");
-      }
-      String peerStateZNode = getPeerStateNode(id);
-      try {
-        return ReplicationPeerImpl.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode));
-      } catch (KeeperException e) {
-        throw new ReplicationException(e);
-      } catch (DeserializationException e) {
-        throw new ReplicationException(e);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Unable to get status of the peer with id=" + id +
-          " from backing store", e);
-    } catch (InterruptedException e) {
-      throw new ReplicationException(e);
-    }
-  }
-
-  @Override
-  public Map<String, ReplicationPeerConfig> getAllPeerConfigs() {
-    Map<String, ReplicationPeerConfig> peers = new TreeMap<>();
-    List<String> ids = null;
-    try {
-      ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
-      for (String id : ids) {
-        ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
-        if (peerConfig == null) {
-          LOG.warn("Failed to get replication peer configuration of clusterid=" + id
-            + " znode content, continuing.");
-          continue;
-        }
-        peers.put(id, peerConfig);
-      }
-    } catch (KeeperException e) {
-      this.abortable.abort("Cannot get the list of peers ", e);
-    } catch (ReplicationException e) {
-      this.abortable.abort("Cannot get the list of peers ", e);
-    }
-    return peers;
-  }
-
-  @Override
-  public ReplicationPeerImpl getConnectedPeer(String peerId) {
-    return peerClusters.get(peerId);
-  }
-
-  @Override
-  public Set<String> getConnectedPeerIds() {
-    return peerClusters.keySet(); // this is not thread-safe
-  }
-
-  /**
-   * Returns a ReplicationPeerConfig from the znode or null for the given peerId.
-   */
-  @Override
-  public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
-      throws ReplicationException {
-    String znode = getPeerNode(peerId);
-    byte[] data = null;
-    try {
-      data = ZKUtil.getData(this.zookeeper, znode);
-    } catch (InterruptedException e) {
-      LOG.warn("Could not get configuration for peer because the thread " +
-          "was interrupted. peerId=" + peerId);
-      Thread.currentThread().interrupt();
-      return null;
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error getting configuration for peer with id="
-          + peerId, e);
-    }
-    if (data == null) {
-      LOG.error("Could not get configuration for peer because it doesn't exist. peerId=" + peerId);
-      return null;
-    }
-
-    try {
-      return ReplicationPeerConfigUtil.parsePeerFrom(data);
-    } catch (DeserializationException e) {
-      LOG.warn("Failed to parse cluster key from peerId=" + peerId
-          + ", specifically the content from the following znode: " + znode);
-      return null;
-    }
-  }
-
-  @Override
-  public Pair<ReplicationPeerConfig, Configuration> getPeerConf(String peerId)
-      throws ReplicationException {
-    ReplicationPeerConfig peerConfig = getReplicationPeerConfig(peerId);
-
-    if (peerConfig == null) {
-      return null;
-    }
-
-    Configuration otherConf;
-    try {
-      otherConf = HBaseConfiguration.createClusterConf(this.conf, peerConfig.getClusterKey());
-    } catch (IOException e) {
-      LOG.error("Can't get peer configuration for peerId=" + peerId + " because:", e);
-      return null;
-    }
-
-    if (!peerConfig.getConfiguration().isEmpty()) {
-      CompoundConfiguration compound = new CompoundConfiguration();
-      compound.add(otherConf);
-      compound.addStringMap(peerConfig.getConfiguration());
-      return new Pair<>(peerConfig, compound);
-    }
-
-    return new Pair<>(peerConfig, otherConf);
-  }
-
-  @Override
-  public void updatePeerConfig(String id, ReplicationPeerConfig newConfig)
-      throws ReplicationException {
-    ReplicationPeer peer = getConnectedPeer(id);
-    if (peer == null){
-      throw new ReplicationException("Could not find peer Id " + id + " in connected peers");
-    }
-    ReplicationPeerConfig existingConfig = peer.getPeerConfig();
-    if (!isStringEquals(newConfig.getClusterKey(), existingConfig.getClusterKey())) {
-      throw new ReplicationException(
-          "Changing the cluster key on an existing peer is not allowed." + " Existing key '" +
-              existingConfig.getClusterKey() + "' does not match new key '" +
-              newConfig.getClusterKey() + "'");
-    }
-    if (!isStringEquals(newConfig.getReplicationEndpointImpl(),
-      existingConfig.getReplicationEndpointImpl())) {
-      throw new ReplicationException("Changing the replication endpoint implementation class " +
-          "on an existing peer is not allowed. Existing class '" +
-          existingConfig.getReplicationEndpointImpl() + "' does not match new class '" +
-          newConfig.getReplicationEndpointImpl() + "'");
-    }
-
-    // Update existingConfig's peer config and peer data with the new values, but don't touch config
-    // or data that weren't explicitly changed
-    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(existingConfig);
-    builder.putAllConfiguration(newConfig.getConfiguration())
-        .putAllPeerData(newConfig.getPeerData())
-        .setReplicateAllUserTables(newConfig.replicateAllUserTables())
-        .setNamespaces(newConfig.getNamespaces()).setTableCFsMap(newConfig.getTableCFsMap())
-        .setExcludeNamespaces(newConfig.getExcludeNamespaces())
-        .setExcludeTableCFsMap(newConfig.getExcludeTableCFsMap())
-        .setBandwidth(newConfig.getBandwidth());
-
-    try {
-      ZKUtil.setData(this.zookeeper, getPeerNode(id),
-          ReplicationPeerConfigUtil.toByteArray(builder.build()));
-    }
-    catch(KeeperException ke){
-      throw new ReplicationException("There was a problem trying to save changes to the " +
-          "replication peer " + id, ke);
-    }
-  }
-
-  /**
-   * List all registered peer clusters and set a watch on their znodes.
-   */
-  @Override
-  public List<String> getAllPeerIds() {
-    List<String> ids = null;
-    try {
-      ids = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Cannot get the list of peers ", e);
-    }
-    return ids;
-  }
-
-  /**
-   * A private method used during initialization. This method attempts to add all registered
-   * peer clusters. This method does not set a watch on the peer cluster znodes.
-   */
-  private void addExistingPeers() throws ReplicationException {
-    List<String> znodes = null;
-    try {
-      znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error getting the list of peer clusters.", e);
-    }
-    if (znodes != null) {
-      for (String z : znodes) {
-        createAndAddPeer(z);
-      }
-    }
-  }
-
-  @Override
-  public boolean peerConnected(String peerId) throws ReplicationException {
-    return createAndAddPeer(peerId);
-  }
-
-  @Override
-  public void peerDisconnected(String peerId) {
-    ReplicationPeer rp = this.peerClusters.get(peerId);
-    if (rp != null) {
-      peerClusters.remove(peerId, rp);
-    }
-  }
-
-  /**
-   * Attempt to connect to a new remote slave cluster.
-   * @param peerId a short that identifies the cluster
-   * @return true if a new connection was made, false if no new connection was made.
-   */
-  public boolean createAndAddPeer(String peerId) throws ReplicationException {
-    if (peerClusters == null) {
-      return false;
-    }
-    if (this.peerClusters.containsKey(peerId)) {
-      return false;
-    }
-
-    ReplicationPeerImpl peer = null;
-    try {
-      peer = createPeer(peerId);
-    } catch (Exception e) {
-      throw new ReplicationException("Error adding peer with id=" + peerId, e);
-    }
-    if (peer == null) {
-      return false;
-    }
-    ReplicationPeerImpl previous = peerClusters.putIfAbsent(peerId, peer);
-    if (previous == null) {
-      LOG.info("Added new peer cluster=" + peer.getPeerConfig().getClusterKey());
-    } else {
-      LOG.info("Peer already present, " + previous.getPeerConfig().getClusterKey() +
-        ", new cluster=" + peer.getPeerConfig().getClusterKey());
-    }
-    return true;
-  }
-
-  /**
-   * Update the state znode of a peer cluster.
-   * @param id
-   * @param state
-   */
-  private void changePeerState(String id, ReplicationProtos.ReplicationState.State state)
-      throws ReplicationException {
-    try {
-      if (!peerExists(id)) {
-        throw new IllegalArgumentException("Cannot enable/disable peer because id=" + id
-            + " does not exist.");
-      }
-      String peerStateZNode = getPeerStateNode(id);
-      byte[] stateBytes =
-          (state == ReplicationProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES
-              : DISABLED_ZNODE_BYTES;
-      if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
-        ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes);
-      } else {
-        ZKUtil.createAndWatch(this.zookeeper, peerStateZNode, stateBytes);
-      }
-      LOG.info("Peer with id= " + id + " is now " + state.name());
-    } catch (KeeperException e) {
-      throw new ReplicationException("Unable to change state of the peer with id=" + id, e);
-    }
-  }
-
-  /**
-   * Helper method to connect to a peer
-   * @param peerId peer's identifier
-   * @return object representing the peer
-   * @throws ReplicationException
-   */
-  private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
-    Pair<ReplicationPeerConfig, Configuration> pair = getPeerConf(peerId);
-    if (pair == null) {
-      return null;
-    }
-    Configuration peerConf = pair.getSecond();
-
-    ReplicationPeerImpl peer =
-        new ReplicationPeerImpl(zookeeper, peerConf, peerId, pair.getFirst());
-
-    // Load peer state and peer config by reading zookeeper directly.
-    peer.refreshPeerState();
-    peer.refreshPeerConfig();
-
-    return peer;
-  }
-
-  private void checkQueuesDeleted(String peerId) throws ReplicationException {
-    if (queueStorage == null) {
-      return;
-    }
-    try {
-      List<ServerName> replicators = queueStorage.getListOfReplicators();
-      if (replicators == null || replicators.isEmpty()) {
-        return;
-      }
-      for (ServerName replicator : replicators) {
-        List<String> queueIds = queueStorage.getAllQueues(replicator);
-        for (String queueId : queueIds) {
-          ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-          if (queueInfo.getPeerId().equals(peerId)) {
-            throw new IllegalArgumentException("undeleted queue for peerId: " + peerId
-                + ", replicator: " + replicator + ", queueId: " + queueId);
-          }
-        }
-      }
-      // Check for hfile-refs queue
-      if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode)
-          && queueStorage.getAllPeersFromHFileRefsQueue().contains(peerId)) {
-        throw new IllegalArgumentException("Undeleted queue for peerId: " + peerId
-            + ", found in hfile-refs node path " + hfileRefsZNode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Could not check queues deleted with id=" + peerId, e);
-    }
-  }
-
-  /**
-   * For replication peer cluster key or endpoint class, null and empty string is same. So here
-   * don't use {@link StringUtils#equals(CharSequence, CharSequence)} directly.
-   */
-  private boolean isStringEquals(String s1, String s2) {
-    if (StringUtils.isBlank(s1)) {
-      return StringUtils.isBlank(s2);
-    }
-    return s1.equals(s2);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
index 49af4c3..bf448e8 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication;
 
 import java.util.Arrays;
 import java.util.List;
-import java.util.Optional;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
@@ -144,7 +143,7 @@ class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements Repli
   }
 
   @Override
-  public Optional<ReplicationPeerConfig> getPeerConfig(String peerId) throws ReplicationException {
+  public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException {
     byte[] data;
     try {
       data = ZKUtil.getData(zookeeper, getPeerNode(peerId));
@@ -152,13 +151,14 @@ class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements Repli
       throw new ReplicationException("Error getting configuration for peer with id=" + peerId, e);
     }
     if (data == null || data.length == 0) {
-      return Optional.empty();
+      throw new ReplicationException(
+          "Replication peer config data shouldn't be empty, peerId=" + peerId);
     }
     try {
-      return Optional.of(ReplicationPeerConfigUtil.parsePeerFrom(data));
+      return ReplicationPeerConfigUtil.parsePeerFrom(data);
     } catch (DeserializationException e) {
-      LOG.warn("Failed to parse replication peer config for peer with id=" + peerId, e);
-      return Optional.empty();
+      throw new ReplicationException(
+          "Failed to parse replication peer config for peer with id=" + peerId, e);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
index b8a2044..d09a56b 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
@@ -48,8 +48,7 @@ class ZKReplicationStorageBase {
     String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
 
     this.replicationZNode =
-      ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode, replicationZNodeName);
-
+        ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode, replicationZNodeName);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 4afda5d..2589199 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -55,7 +55,6 @@ public abstract class TestReplicationStateBasic {
   protected static String KEY_TWO;
 
   // For testing when we try to replicate to ourself
-  protected String OUR_ID = "3";
   protected String OUR_KEY;
 
   protected static int zkTimeoutCount;
@@ -152,37 +151,6 @@ public abstract class TestReplicationStateBasic {
   }
 
   @Test
-  public void testInvalidClusterKeys() throws ReplicationException, KeeperException {
-    rp.init();
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"));
-      fail("Should throw an IllegalArgumentException because " +
-        "zookeeper.znode.parent is missing leading '/'.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"));
-      fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"));
-      fail("Should throw an IllegalArgumentException because " +
-        "hbase.zookeeper.property.clientPort is missing.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-  }
-
-  @Test
   public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
     rp.init();
 
@@ -192,7 +160,8 @@ public abstract class TestReplicationStateBasic {
     files1.add(new Pair<>(null, new Path("file_3")));
     assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
     assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rp.getPeerStorage().addPeer(ID_ONE,
+            ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true);
     rqs.addPeerToHFileRefs(ID_ONE);
     rqs.addHFileRefs(ID_ONE, files1);
     assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
@@ -208,15 +177,17 @@ public abstract class TestReplicationStateBasic {
     hfiles2.add(removedString);
     rqs.removeHFileRefs(ID_ONE, hfiles2);
     assertEquals(0, rqs.getReplicableHFiles(ID_ONE).size());
-    rp.unregisterPeer(ID_ONE);
+    rp.getPeerStorage().removePeer(ID_ONE);
   }
 
   @Test
   public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
     rp.init();
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rp.getPeerStorage().addPeer(ID_ONE,
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true);
     rqs.addPeerToHFileRefs(ID_ONE);
-    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
+    rp.getPeerStorage().addPeer(ID_TWO,
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true);
     rqs.addPeerToHFileRefs(ID_TWO);
 
     List<Pair<Path, Path>> files1 = new ArrayList<>(3);
@@ -229,13 +200,13 @@ public abstract class TestReplicationStateBasic {
     assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
     assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
 
-    rp.unregisterPeer(ID_ONE);
+    rp.getPeerStorage().removePeer(ID_ONE);
     rqs.removePeerFromHFileRefs(ID_ONE);
     assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
     assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
     assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
 
-    rp.unregisterPeer(ID_TWO);
+    rp.getPeerStorage().removePeer(ID_TWO);
     rqs.removePeerFromHFileRefs(ID_TWO);
     assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
     assertTrue(rqs.getReplicableHFiles(ID_TWO).isEmpty());
@@ -245,74 +216,77 @@ public abstract class TestReplicationStateBasic {
   public void testReplicationPeers() throws Exception {
     rp.init();
 
-    // Test methods with non-existent peer ids
     try {
-      rp.unregisterPeer("bogus");
+      rp.getPeerStorage().setPeerState("bogus", true);
       fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
+    } catch (ReplicationException e) {
     }
     try {
-      rp.enablePeer("bogus");
+      rp.getPeerStorage().setPeerState("bogus", false);
       fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
+    } catch (ReplicationException e) {
     }
     try {
-      rp.disablePeer("bogus");
+      rp.isPeerEnabled("bogus");
       fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
     } catch (IllegalArgumentException e) {
     }
+
     try {
-      rp.getStatusOfPeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
+      assertFalse(rp.addPeer("bogus"));
+      fail("Should have thrown an ReplicationException when passed a bogus peerId");
+    } catch (ReplicationException e) {
+    }
+
+    try {
+      assertNull(rp.getPeerClusterConfiguration("bogus"));
+      fail("Should have thrown an ReplicationException when passed a bogus peerId");
+    } catch (ReplicationException e) {
     }
-    assertFalse(rp.peerConnected("bogus"));
-    rp.peerDisconnected("bogus");
 
-    assertNull(rp.getPeerConf("bogus"));
     assertNumberOfPeers(0);
 
     // Add some peers
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true);
     assertNumberOfPeers(1);
-    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
+    rp.getPeerStorage().addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), true);
     assertNumberOfPeers(2);
 
     // Test methods with a peer that is added but not connected
     try {
-      rp.getStatusOfPeer(ID_ONE);
+      rp.isPeerEnabled(ID_ONE);
       fail("There are no connected peers, should have thrown an IllegalArgumentException");
     } catch (IllegalArgumentException e) {
     }
-    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond()));
-    rp.unregisterPeer(ID_ONE);
-    rp.peerDisconnected(ID_ONE);
+    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerClusterConfiguration(ID_ONE)));
+    rp.getPeerStorage().removePeer(ID_ONE);
+    rp.removePeer(ID_ONE);
     assertNumberOfPeers(1);
 
     // Add one peer
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rp.peerConnected(ID_ONE);
+    rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true);
+    rp.addPeer(ID_ONE);
     assertNumberOfPeers(2);
-    assertTrue(rp.getStatusOfPeer(ID_ONE));
-    rp.disablePeer(ID_ONE);
+    assertTrue(rp.isPeerEnabled(ID_ONE));
+    rp.getPeerStorage().setPeerState(ID_ONE, false);
     // now we do not rely on zk watcher to trigger the state change so we need to trigger it
     // manually...
-    ReplicationPeerImpl peer = rp.getConnectedPeer(ID_ONE);
-    peer.refreshPeerState();
+    ReplicationPeerImpl peer = rp.getPeer(ID_ONE);
+    rp.refreshPeerState(peer.getId());
     assertEquals(PeerState.DISABLED, peer.getPeerState());
     assertConnectedPeerStatus(false, ID_ONE);
-    rp.enablePeer(ID_ONE);
+    rp.getPeerStorage().setPeerState(ID_ONE, true);
     // now we do not rely on zk watcher to trigger the state change so we need to trigger it
     // manually...
-    peer.refreshPeerState();
+    rp.refreshPeerState(peer.getId());
     assertEquals(PeerState.ENABLED, peer.getPeerState());
     assertConnectedPeerStatus(true, ID_ONE);
 
     // Disconnect peer
-    rp.peerDisconnected(ID_ONE);
+    rp.removePeer(ID_ONE);
     assertNumberOfPeers(2);
     try {
-      rp.getStatusOfPeer(ID_ONE);
+      rp.isPeerEnabled(ID_ONE);
       fail("There are no connected peers, should have thrown an IllegalArgumentException");
     } catch (IllegalArgumentException e) {
     }
@@ -320,16 +294,16 @@ public abstract class TestReplicationStateBasic {
 
   protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
     // we can first check if the value was changed in the store, if it wasn't then fail right away
-    if (status != rp.getStatusOfPeerFromBackingStore(peerId)) {
+    if (status != rp.getPeerStorage().isPeerEnabled(peerId)) {
       fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK");
     }
     while (true) {
-      if (status == rp.getStatusOfPeer(peerId)) {
+      if (status == rp.isPeerEnabled(peerId)) {
         return;
       }
       if (zkTimeoutCount < ZK_MAX_COUNT) {
-        LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status +
-          ", sleeping and trying again.");
+        LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status
+            + ", sleeping and trying again.");
         Thread.sleep(ZK_SLEEP_INTERVAL);
       } else {
         fail("Timed out waiting for ConnectedPeerStatus to be " + status);
@@ -337,10 +311,8 @@ public abstract class TestReplicationStateBasic {
     }
   }
 
-  protected void assertNumberOfPeers(int total) {
-    assertEquals(total, rp.getAllPeerConfigs().size());
-    assertEquals(total, rp.getAllPeerIds().size());
-    assertEquals(total, rp.getAllPeerIds().size());
+  protected void assertNumberOfPeers(int total) throws ReplicationException {
+    assertEquals(total, rp.getPeerStorage().listPeerIds().size());
   }
 
   /*
@@ -359,8 +331,9 @@ public abstract class TestReplicationStateBasic {
         rqs.addWAL(server3, "qId" + i, "filename" + j);
       }
       // Add peers for the corresponding queues so they are not orphans
-      rp.registerPeer("qId" + i,
-        new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i));
+      rp.getPeerStorage().addPeer("qId" + i,
+        ReplicationPeerConfig.newBuilder().setClusterKey("localhost:2818:/bogus" + i).build(),
+        true);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index ac869d9..6825c36 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -79,7 +79,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
   public void setUp() {
     zkTimeoutCount = 0;
     rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
-    rp = ReplicationFactory.getReplicationPeers(zkw, conf, new WarnOnlyAbortable());
+    rp = ReplicationFactory.getReplicationPeers(zkw, conf);
     OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
index a3be1e6..e8098c8 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.HashMap;
@@ -143,14 +144,14 @@ public class TestZKReplicationPeerStorage {
     assertEquals(peerCount, peerIds.size());
     for (String peerId : peerIds) {
       int seed = Integer.parseInt(peerId);
-      assertConfigEquals(getConfig(seed), STORAGE.getPeerConfig(peerId).get());
+      assertConfigEquals(getConfig(seed), STORAGE.getPeerConfig(peerId));
     }
     for (int i = 0; i < peerCount; i++) {
       STORAGE.updatePeerConfig(Integer.toString(i), getConfig(i + 1));
     }
     for (String peerId : peerIds) {
       int seed = Integer.parseInt(peerId);
-      assertConfigEquals(getConfig(seed + 1), STORAGE.getPeerConfig(peerId).get());
+      assertConfigEquals(getConfig(seed + 1), STORAGE.getPeerConfig(peerId));
     }
     for (int i = 0; i < peerCount; i++) {
       assertEquals(i % 2 == 0, STORAGE.isPeerEnabled(Integer.toString(i)));
@@ -166,6 +167,11 @@ public class TestZKReplicationPeerStorage {
     peerIds = STORAGE.listPeerIds();
     assertEquals(peerCount - 1, peerIds.size());
     assertFalse(peerIds.contains(toRemove));
-    assertFalse(STORAGE.getPeerConfig(toRemove).isPresent());
+
+    try {
+      STORAGE.getPeerConfig(toRemove);
+      fail("Should throw a ReplicationException when get peer config of a peerId");
+    } catch (ReplicationException e) {
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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 af41399..f2c3ec9 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,8 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
@@ -51,20 +50,14 @@ import org.slf4j.LoggerFactory;
 public class ReplicationZKNodeCleaner {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleaner.class);
   private final ReplicationQueueStorage queueStorage;
-  private final ReplicationPeers replicationPeers;
+  private final ReplicationPeerStorage peerStorage;
   private final ReplicationQueueDeletor queueDeletor;
 
   public ReplicationZKNodeCleaner(Configuration conf, ZKWatcher zkw, Abortable abortable)
       throws IOException {
-    try {
-      this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
-      this.replicationPeers =
-          ReplicationFactory.getReplicationPeers(zkw, conf, this.queueStorage, abortable);
-      this.replicationPeers.init();
-      this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, abortable);
-    } catch (ReplicationException e) {
-      throw new IOException("failed to construct ReplicationZKNodeCleaner", e);
-    }
+    this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
+    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf);
+    this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, abortable);
   }
 
   /**
@@ -73,8 +66,8 @@ public class ReplicationZKNodeCleaner {
    */
   public Map<ServerName, List<String>> getUnDeletedQueues() throws IOException {
     Map<ServerName, List<String>> undeletedQueues = new HashMap<>();
-    Set<String> peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds());
     try {
+      Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds());
       List<ServerName> replicators = this.queueStorage.getListOfReplicators();
       if (replicators == null || replicators.isEmpty()) {
         return undeletedQueues;
@@ -84,8 +77,7 @@ public class ReplicationZKNodeCleaner {
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           if (!peerIds.contains(queueInfo.getPeerId())) {
-            undeletedQueues.computeIfAbsent(replicator, (key) -> new ArrayList<>()).add(
-              queueId);
+            undeletedQueues.computeIfAbsent(replicator, (key) -> new ArrayList<>()).add(queueId);
             if (LOG.isDebugEnabled()) {
               LOG.debug("Undeleted replication queue for removed peer found: "
                   + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]",
@@ -106,9 +98,9 @@ public class ReplicationZKNodeCleaner {
    */
   public Set<String> getUnDeletedHFileRefsQueues() throws IOException {
     Set<String> undeletedHFileRefsQueue = new HashSet<>();
-    Set<String> peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds());
     String hfileRefsZNode = queueDeletor.getHfileRefsZNode();
     try {
+      Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds());
       List<String> listOfPeers = this.queueStorage.getAllPeersFromHFileRefsQueue();
       Set<String> peers = new HashSet<>(listOfPeers);
       peers.removeAll(peerIds);
@@ -116,15 +108,15 @@ public class ReplicationZKNodeCleaner {
         undeletedHFileRefsQueue.addAll(peers);
       }
     } catch (ReplicationException e) {
-      throw new IOException(
-          "Failed to get list of all peers from hfile-refs znode " + hfileRefsZNode, e);
+      throw new IOException("Failed to get list of all peers from hfile-refs znode "
+          + hfileRefsZNode, e);
     }
     return undeletedHFileRefsQueue;
   }
 
   private class ReplicationQueueDeletor extends ReplicationStateZKBase {
 
-    public ReplicationQueueDeletor(ZKWatcher zk, Configuration conf, Abortable abortable) {
+    ReplicationQueueDeletor(ZKWatcher zk, Configuration conf, Abortable abortable) {
       super(zk, conf, abortable);
     }
 
@@ -132,19 +124,20 @@ public class ReplicationZKNodeCleaner {
      * @param replicator The regionserver which has undeleted queue
      * @param queueId The undeleted queue id
      */
-    public void removeQueue(final ServerName replicator, final String queueId) throws IOException {
-      String queueZnodePath = ZNodePaths
-          .joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator.getServerName()), queueId);
+    void removeQueue(final ServerName replicator, final String queueId) throws IOException {
+      String queueZnodePath =
+          ZNodePaths.joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator.getServerName()),
+            queueId);
       try {
         ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-        if (!replicationPeers.getAllPeerIds().contains(queueInfo.getPeerId())) {
+        if (!peerStorage.listPeerIds().contains(queueInfo.getPeerId())) {
           ZKUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath);
-          LOG.info("Successfully removed replication queue, replicator: " + replicator +
-            ", queueId: " + queueId);
+          LOG.info("Successfully removed replication queue, replicator: " + replicator
+              + ", queueId: " + queueId);
         }
-      } catch (KeeperException e) {
-        throw new IOException(
-            "Failed to delete queue, replicator: " + replicator + ", queueId: " + queueId);
+      } catch (ReplicationException | KeeperException e) {
+        throw new IOException("Failed to delete queue, replicator: " + replicator + ", queueId: "
+            + queueId);
       }
     }
 
@@ -152,17 +145,17 @@ public class ReplicationZKNodeCleaner {
      * @param hfileRefsQueueId The undeleted hfile-refs queue id
      * @throws IOException
      */
-    public void removeHFileRefsQueue(final String hfileRefsQueueId) throws IOException {
+    void removeHFileRefsQueue(final String hfileRefsQueueId) throws IOException {
       String node = ZNodePaths.joinZNode(this.hfileRefsZNode, hfileRefsQueueId);
       try {
-        if (!replicationPeers.getAllPeerIds().contains(hfileRefsQueueId)) {
+        if (!peerStorage.listPeerIds().contains(hfileRefsQueueId)) {
           ZKUtil.deleteNodeRecursively(this.zookeeper, node);
           LOG.info("Successfully removed hfile-refs queue " + hfileRefsQueueId + " from path "
               + hfileRefsZNode);
         }
-      } catch (KeeperException e) {
+      } catch (ReplicationException | KeeperException e) {
         throw new IOException("Failed to delete hfile-refs queue " + hfileRefsQueueId
-            + " from path " + hfileRefsZNode);
+            + " from path " + hfileRefsZNode, e);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index f4ccce8..b6732d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -314,12 +314,12 @@ public class ReplicationPeerManager {
   public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
       throws ReplicationException {
     ReplicationPeerStorage peerStorage =
-      ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
+        ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
     ConcurrentMap<String, ReplicationPeerDescription> peers = new ConcurrentHashMap<>();
     for (String peerId : peerStorage.listPeerIds()) {
-      Optional<ReplicationPeerConfig> peerConfig = peerStorage.getPeerConfig(peerId);
+      ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
       boolean enabled = peerStorage.isPeerEnabled(peerId);
-      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig.get()));
+      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig));
     }
     return new ReplicationPeerManager(peerStorage,
         ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/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 73e600e..27bda2d 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
@@ -310,7 +310,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
 
     queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
     replicationPeers =
-        ReplicationFactory.getReplicationPeers(zkw, getConf(), queueStorage, connection);
+        ReplicationFactory.getReplicationPeers(zkw, getConf());
     replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
       new WarnOnlyAbortable(), new WarnOnlyStoppable());
     Set<String> liveRegionServers = new HashSet<>(replicationTracker.getListOfRegionServers());

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
index 598357c..1efe180 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -19,9 +19,10 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,7 +31,8 @@ import org.slf4j.LoggerFactory;
 public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
   private static final Logger LOG = LoggerFactory.getLogger(PeerProcedureHandlerImpl.class);
 
-  private ReplicationSourceManager replicationSourceManager;
+  private final ReplicationSourceManager replicationSourceManager;
+  private final ReentrantLock peersLock = new ReentrantLock();
 
   public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) {
     this.replicationSourceManager = replicationSourceManager;
@@ -38,45 +40,40 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
   @Override
   public void addPeer(String peerId) throws ReplicationException, IOException {
-    replicationSourceManager.addPeer(peerId);
+    peersLock.lock();
+    try {
+      replicationSourceManager.addPeer(peerId);
+    } finally {
+      peersLock.unlock();
+    }
   }
 
   @Override
   public void removePeer(String peerId) throws ReplicationException, IOException {
-    replicationSourceManager.removePeer(peerId);
+    peersLock.lock();
+    try {
+      if (replicationSourceManager.getReplicationPeers().getPeer(peerId) != null) {
+        replicationSourceManager.removePeer(peerId);
+      }
+    } finally {
+      peersLock.unlock();
+    }
   }
 
   @Override
   public void disablePeer(String peerId) throws ReplicationException, IOException {
-    ReplicationPeerImpl peer =
-        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
-    if (peer != null) {
-      peer.refreshPeerState();
-      LOG.info("disable replication peer, id: " + peerId + ", new state: " + peer.getPeerState());
-    } else {
-      throw new ReplicationException("No connected peer found, peerId=" + peerId);
-    }
+    PeerState newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+    LOG.info("disable replication peer, id: " + peerId + ", new state: " + newState);
   }
 
   @Override
   public void enablePeer(String peerId) throws ReplicationException, IOException {
-    ReplicationPeerImpl peer =
-        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
-    if (peer != null) {
-      peer.refreshPeerState();
-      LOG.info("enable replication peer, id: " + peerId + ", new state: " + peer.getPeerState());
-    } else {
-      throw new ReplicationException("No connected peer found, peerId=" + peerId);
-    }
+    PeerState newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+    LOG.info("enable replication peer, id: " + peerId + ", new state: " + newState);
   }
 
   @Override
   public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
-    ReplicationPeerImpl peer =
-        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
-    if (peer == null) {
-      throw new ReplicationException("No connected peer found, peerId=" + peerId);
-    }
-    peer.refreshPeerConfig();
+    replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 72f0fe7..f985f90 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -129,7 +129,7 @@ public class Replication implements
       this.queueStorage =
           ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
       this.replicationPeers =
-          ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server);
+          ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf);
       this.replicationPeers.init();
       this.replicationTracker =
           ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.replicationPeers,

http://git-wip-us.apache.org/repos/asf/hbase/blob/9a51a84b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 271eea7..1f4729b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -223,7 +223,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       // A peerId will not have "-" in its name, see HBASE-11394
       peerId = peerClusterZnode.split("-")[0];
     }
-    Map<TableName, List<String>> tableCFMap = replicationPeers.getConnectedPeer(peerId).getTableCFs();
+    Map<TableName, List<String>> tableCFMap = replicationPeers.getPeer(peerId).getTableCFs();
     if (tableCFMap != null) {
       List<String> tableCfs = tableCFMap.get(tableName);
       if (tableCFMap.containsKey(tableName)
@@ -371,7 +371,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   }
 
   private long getCurrentBandwidth() {
-    ReplicationPeer replicationPeer = this.replicationPeers.getConnectedPeer(peerId);
+    ReplicationPeer replicationPeer = this.replicationPeers.getPeer(peerId);
     long peerBandwidth = replicationPeer != null ? replicationPeer.getPeerBandwidth() : 0;
     // user can set peer bandwidth to 0 to use default bandwidth
     return peerBandwidth != 0 ? peerBandwidth : defaultBandwidth;
@@ -416,7 +416,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
    */
   @Override
   public boolean isPeerEnabled() {
-    return this.replicationPeers.getStatusOfPeer(this.peerId);
+    return this.replicationPeers.isPeerEnabled(this.peerId);
   }
 
   @Override


[36/48] hbase git commit: HBASE-19633 Clean up the replication queues in the postPeerModification stage when removing a peer

Posted by zh...@apache.org.
HBASE-19633 Clean up the replication queues in the postPeerModification stage when removing a peer


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 6b28956f0abdbe80dee3fe0557694c1ac2ddf1a3
Parents: 9a51a84
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 2 09:57:23 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfig.java      |  2 +-
 .../replication/VerifyReplication.java          | 34 ++++++++++-------
 .../hbase/replication/ReplicationPeers.java     | 32 ++++++----------
 .../replication/ZKReplicationQueueStorage.java  |  3 +-
 .../replication/ZKReplicationStorageBase.java   |  4 +-
 .../replication/TestReplicationStateBasic.java  | 10 +----
 .../master/replication/AddPeerProcedure.java    |  5 +--
 .../replication/DisablePeerProcedure.java       |  3 +-
 .../master/replication/EnablePeerProcedure.java |  3 +-
 .../master/replication/ModifyPeerProcedure.java | 34 +++++++++--------
 .../replication/RefreshPeerProcedure.java       | 17 ++++-----
 .../master/replication/RemovePeerProcedure.java |  7 ++--
 .../replication/ReplicationPeerManager.java     | 31 +++++++++++++++-
 .../replication/UpdatePeerConfigProcedure.java  |  3 +-
 .../RemoteProcedureResultReporter.java          |  3 +-
 .../regionserver/RefreshPeerCallable.java       |  5 ++-
 .../regionserver/ReplicationSourceManager.java  | 39 +++++++-------------
 .../TestReplicationAdminUsingProcedure.java     |  7 ++--
 18 files changed, 124 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index b80ee16..fdae288 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -27,8 +27,8 @@ import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A configuration for the replication peer cluster.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/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 f0070f0..fe45762 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
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce.replication;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,13 +44,14 @@ import org.apache.hadoop.hbase.filter.PrefixFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableMapper;
+import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableSplit;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -66,6 +66,7 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -333,19 +334,24 @@ public class VerifyReplication extends Configured implements Tool {
       final Configuration conf, String peerId) throws IOException {
     ZKWatcher localZKW = null;
     try {
-      localZKW = new ZKWatcher(conf, "VerifyReplication",
-          new Abortable() {
-            @Override public void abort(String why, Throwable e) {}
-            @Override public boolean isAborted() {return false;}
-          });
-
-      ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf);
-      rp.init();
+      localZKW = new ZKWatcher(conf, "VerifyReplication", new Abortable() {
+        @Override
+        public void abort(String why, Throwable e) {
+        }
 
-      return Pair.newPair(rp.getPeerConfig(peerId), rp.getPeerClusterConfiguration(peerId));
+        @Override
+        public boolean isAborted() {
+          return false;
+        }
+      });
+      ReplicationPeerStorage storage =
+          ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf);
+      ReplicationPeerConfig peerConfig = storage.getPeerConfig(peerId);
+      return Pair.newPair(peerConfig,
+        ReplicationPeers.getPeerClusterConfiguration(peerConfig, conf));
     } catch (ReplicationException e) {
-      throw new IOException(
-          "An error occurred while trying to connect to the remove peer cluster", e);
+      throw new IOException("An error occurred while trying to connect to the remove peer cluster",
+          e);
     } finally {
       if (localZKW != null) {
         localZKW.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index e58482e..422801b 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
@@ -39,20 +37,22 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 @InterfaceAudience.Private
 public class ReplicationPeers {
 
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeers.class);
-
   private final Configuration conf;
 
   // Map of peer clusters keyed by their id
   private final ConcurrentMap<String, ReplicationPeerImpl> peerCache;
   private final ReplicationPeerStorage peerStorage;
 
-  protected ReplicationPeers(ZKWatcher zookeeper, Configuration conf) {
+  ReplicationPeers(ZKWatcher zookeeper, Configuration conf) {
     this.conf = conf;
     this.peerCache = new ConcurrentHashMap<>();
     this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zookeeper, conf);
   }
 
+  public Configuration getConf() {
+    return conf;
+  }
+
   public void init() throws ReplicationException {
     // Loading all existing peerIds into peer cache.
     for (String peerId : this.peerStorage.listPeerIds()) {
@@ -120,22 +120,13 @@ public class ReplicationPeers {
     return peerCache.keySet();
   }
 
-  public ReplicationPeerConfig getPeerConfig(String peerId) {
-    ReplicationPeer replicationPeer = this.peerCache.get(peerId);
-    if (replicationPeer == null) {
-      throw new IllegalArgumentException("Peer with id= " + peerId + " is not cached");
-    }
-    return replicationPeer.getPeerConfig();
-  }
-
-  public Configuration getPeerClusterConfiguration(String peerId) throws ReplicationException {
-    ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
-
+  public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,
+      Configuration baseConf) throws ReplicationException {
     Configuration otherConf;
     try {
-      otherConf = HBaseConfiguration.createClusterConf(this.conf, peerConfig.getClusterKey());
+      otherConf = HBaseConfiguration.createClusterConf(baseConf, peerConfig.getClusterKey());
     } catch (IOException e) {
-      throw new ReplicationException("Can't get peer configuration for peerId=" + peerId, e);
+      throw new ReplicationException("Can't get peer configuration for peer " + peerConfig, e);
     }
 
     if (!peerConfig.getConfiguration().isEmpty()) {
@@ -179,8 +170,9 @@ public class ReplicationPeers {
 >>>>>>> HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface
    */
   private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
-    ReplicationPeerConfig peerConf = peerStorage.getPeerConfig(peerId);
+    ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
     boolean enabled = peerStorage.isPeerEnabled(peerId);
-    return new ReplicationPeerImpl(getPeerClusterConfiguration(peerId), peerId, enabled, peerConf);
+    return new ReplicationPeerImpl(getPeerClusterConfiguration(peerConfig, conf), peerId, enabled,
+        peerConfig);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 41f50d8..ee237f2 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
@@ -50,7 +49,7 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * ZK based replication queue storage.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
index d09a56b..2321e4f 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
@@ -19,13 +19,13 @@ package org.apache.hadoop.hbase.replication;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 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.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 2589199..07c6c15 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -238,12 +237,6 @@ public abstract class TestReplicationStateBasic {
     } catch (ReplicationException e) {
     }
 
-    try {
-      assertNull(rp.getPeerClusterConfiguration("bogus"));
-      fail("Should have thrown an ReplicationException when passed a bogus peerId");
-    } catch (ReplicationException e) {
-    }
-
     assertNumberOfPeers(0);
 
     // Add some peers
@@ -258,7 +251,8 @@ public abstract class TestReplicationStateBasic {
       fail("There are no connected peers, should have thrown an IllegalArgumentException");
     } catch (IllegalArgumentException e) {
     }
-    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerClusterConfiguration(ID_ONE)));
+    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationPeers
+        .getPeerClusterConfiguration(rp.getPeerStorage().getPeerConfig(ID_ONE), rp.getConf())));
     rp.getPeerStorage().removePeer(ID_ONE);
     rp.removePeer(ID_ONE);
     assertNumberOfPeers(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index a4f9b32..f0f7704 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -74,8 +73,8 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
 
   @Override
   protected void postPeerModification(MasterProcedureEnv env) throws IOException {
-    LOG.info("Successfully added " + (enabled ? "ENABLED" : "DISABLED") + " peer " + peerId +
-      ", config " + peerConfig);
+    LOG.info("Successfully added {} peer {}, config {}", enabled ? "ENABLED" : "DISABLED", peerId,
+      peerConfig);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       env.getMasterCoprocessorHost().postAddReplicationPeer(peerId, peerConfig);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
index 10e35a8..0871575 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -62,7 +61,7 @@ public class DisablePeerProcedure extends ModifyPeerProcedure {
 
   @Override
   protected void postPeerModification(MasterProcedureEnv env) throws IOException {
-    LOG.info("Successfully disabled peer " + peerId);
+    LOG.info("Successfully disabled peer {}", peerId);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.postDisableReplicationPeer(peerId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
index f2a9f01..890462f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -62,7 +61,7 @@ public class EnablePeerProcedure extends ModifyPeerProcedure {
 
   @Override
   protected void postPeerModification(MasterProcedureEnv env) throws IOException {
-    LOG.info("Successfully enabled peer " + peerId);
+    LOG.info("Successfully enabled peer {}", peerId);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.postEnableReplicationPeer(peerId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index a682606..c225619 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@@ -84,10 +83,13 @@ public abstract class ModifyPeerProcedure
    * Called before we finish the procedure. The implementation can do some logging work, and also
    * call the coprocessor hook if any.
    * <p>
-   * Notice that, since we have already done the actual work, throwing exception here will not fail
-   * this procedure, we will just ignore it and finish the procedure as suceeded.
+   * Notice that, since we have already done the actual work, throwing {@code IOException} here will
+   * not fail this procedure, we will just ignore it and finish the procedure as suceeded. If
+   * {@code ReplicationException} is thrown we will retry since this usually means we fails to
+   * update the peer storage.
    */
-  protected abstract void postPeerModification(MasterProcedureEnv env) throws IOException;
+  protected abstract void postPeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException;
 
   private void releaseLatch() {
     ProcedurePrepareLatch.releaseLatch(latch, this);
@@ -101,16 +103,14 @@ public abstract class ModifyPeerProcedure
         try {
           prePeerModification(env);
         } catch (IOException e) {
-          LOG.warn(
-            getClass().getName() + " failed to call CP hook or the pre check is failed for peer " +
-              peerId + ", mark the procedure as failure and give up",
-            e);
+          LOG.warn("{} failed to call pre CP hook or the pre check is failed for peer {}, " +
+            "mark the procedure as failure and give up", getClass().getName(), peerId, e);
           setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer", e);
           releaseLatch();
           return Flow.NO_MORE_STATE;
         } catch (ReplicationException e) {
-          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
-            ", retry", e);
+          LOG.warn("{} failed to call prePeerModification for peer {}, retry", getClass().getName(),
+            peerId, e);
           throw new ProcedureYieldException();
         }
         setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
@@ -119,8 +119,8 @@ public abstract class ModifyPeerProcedure
         try {
           updatePeerStorage(env);
         } catch (ReplicationException e) {
-          LOG.warn(
-            getClass().getName() + " update peer storage for peer " + peerId + " failed, retry", e);
+          LOG.warn("{} update peer storage for peer {} failed, retry", getClass().getName(), peerId,
+            e);
           throw new ProcedureYieldException();
         }
         setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
@@ -134,9 +134,13 @@ public abstract class ModifyPeerProcedure
       case POST_PEER_MODIFICATION:
         try {
           postPeerModification(env);
+        } catch (ReplicationException e) {
+          LOG.warn("{} failed to call postPeerModification for peer {}, retry",
+            getClass().getName(), peerId, e);
+          throw new ProcedureYieldException();
         } catch (IOException e) {
-          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
-            ", ignore since the procedure has already done", e);
+          LOG.warn("{} failed to call post CP hook for peer {}, " +
+            "ignore since the procedure has already done", getClass().getName(), peerId, e);
         }
         releaseLatch();
         return Flow.NO_MORE_STATE;
@@ -175,7 +179,7 @@ public abstract class ModifyPeerProcedure
       throws IOException, InterruptedException {
     if (state == PeerModificationState.PRE_PEER_MODIFICATION) {
       // actually the peer related operations has no rollback, but if we haven't done any
-      // modifications on the peer storage, we can just return.
+      // modifications on the peer storage yet, we can just return.
       return;
     }
     throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
index ba4285f..1253ef9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
@@ -122,17 +121,15 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
 
   private void complete(MasterProcedureEnv env, Throwable error) {
     if (event == null) {
-      LOG.warn("procedure event for " + getProcId() +
-          " is null, maybe the procedure is created when recovery",
-        new Exception());
+      LOG.warn("procedure event for {} is null, maybe the procedure is created when recovery",
+        getProcId());
       return;
     }
     if (error != null) {
-      LOG.warn("Refresh peer " + peerId + " for " + type + " on " + targetServer + " failed",
-        error);
+      LOG.warn("Refresh peer {} for {} on {} failed", peerId, type, targetServer, error);
       this.succ = false;
     } else {
-      LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer + " suceeded");
+      LOG.info("Refresh peer {} for {} on {} suceeded", peerId, type, targetServer);
       this.succ = true;
     }
 
@@ -168,9 +165,9 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
       dispatched = false;
     }
     if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) {
-      LOG.info("Can not add remote operation for refreshing peer " + peerId + " for " + type +
-          " to " + targetServer + ", this usually because the server is already dead," +
-          " give up and mark the procedure as complete");
+      LOG.info("Can not add remote operation for refreshing peer {} for {} to {}, " +
+        "this usually because the server is already dead, " +
+        "give up and mark the procedure as complete", peerId, type, targetServer);
       return null;
     }
     dispatched = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index 6e9c384..64faf2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -61,8 +60,10 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
-  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
-    LOG.info("Successfully removed peer " + peerId);
+  protected void postPeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
+    env.getReplicationPeerManager().removeAllQueuesAndHFileRefs(peerId);
+    LOG.info("Successfully removed peer {}", peerId);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.postRemoveReplicationPeer(peerId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index b6732d7..1414d22 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -217,6 +216,36 @@ public class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
+  private void removeAllQueues0(String peerId) throws ReplicationException {
+    for (ServerName replicator : queueStorage.getListOfReplicators()) {
+      List<String> queueIds = queueStorage.getAllQueues(replicator);
+      for (String queueId : queueIds) {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        if (queueInfo.getPeerId().equals(peerId)) {
+          queueStorage.removeQueue(replicator, queueId);
+        }
+      }
+      queueStorage.removeReplicatorIfQueueIsEmpty(replicator);
+    }
+  }
+
+  public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
+    // Here we need two passes to address the problem of claimQueue. Maybe a claimQueue is still
+    // on-going when the refresh peer config procedure is done, if a RS which has already been
+    // scanned claims the queue of a RS which has not been scanned yet, we will miss that queue in
+    // the scan here, and if the RS who has claimed the queue crashed before creating recovered
+    // source, then the queue will leave there until the another RS detects the crash and helps
+    // removing the queue.
+    // A two pass scan can solve the problem. Anyway, the queue will not disappear during the
+    // claiming, it will either under the old RS or under the new RS, and a queue can only be
+    // claimed once after the refresh peer procedure done(as the next claim queue will just delete
+    // it), so we can make sure that a two pass scan will finally find the queue and remove it,
+    // unless it has already been removed by others.
+    removeAllQueues0(peerId);
+    removeAllQueues0(peerId);
+    queueStorage.removePeerFromHFileRefs(peerId);
+  }
+
   private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetryIOException {
     checkClusterKey(peerConfig.getClusterKey());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index a43532d..3497447 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -70,7 +69,7 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
 
   @Override
   protected void postPeerModification(MasterProcedureEnv env) throws IOException {
-    LOG.info("Successfully updated peer config of " + peerId + " to " + peerConfig);
+    LOG.info("Successfully updated peer config of {} to {}", peerId, peerConfig);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
index e4be422..ac3e95a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
@@ -28,7 +28,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
+import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
index c3f33aa..7ada24b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
@@ -20,11 +20,12 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
 import org.apache.log4j.Logger;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 24a4f30..0e50bea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationListener;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@@ -306,9 +305,8 @@ public class ReplicationSourceManager implements ReplicationListener {
    */
   @VisibleForTesting
   ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException {
-    ReplicationPeerConfig peerConfig = replicationPeers.getPeerConfig(id);
     ReplicationPeer peer = replicationPeers.getPeer(id);
-    ReplicationSourceInterface src = getReplicationSource(id, peerConfig, peer);
+    ReplicationSourceInterface src = getReplicationSource(id, peer);
     synchronized (this.walsById) {
       this.sources.add(src);
       Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
@@ -499,8 +497,8 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param peerId the id of the peer cluster
    * @return the created source
    */
-  private ReplicationSourceInterface getReplicationSource(String peerId,
-      ReplicationPeerConfig peerConfig, ReplicationPeer replicationPeer) throws IOException {
+  private ReplicationSourceInterface getReplicationSource(String peerId, ReplicationPeer peer)
+      throws IOException {
     RegionServerCoprocessorHost rsServerHost = null;
     TableDescriptors tableDescriptors = null;
     if (server instanceof HRegionServer) {
@@ -512,24 +510,24 @@ public class ReplicationSourceManager implements ReplicationListener {
 
     ReplicationEndpoint replicationEndpoint = null;
     try {
-      String replicationEndpointImpl = peerConfig.getReplicationEndpointImpl();
+      String replicationEndpointImpl = peer.getPeerConfig().getReplicationEndpointImpl();
       if (replicationEndpointImpl == null) {
         // Default to HBase inter-cluster replication endpoint
         replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
       }
       replicationEndpoint = Class.forName(replicationEndpointImpl)
           .asSubclass(ReplicationEndpoint.class).newInstance();
-      if(rsServerHost != null) {
-        ReplicationEndpoint newReplicationEndPoint = rsServerHost
-            .postCreateReplicationEndPoint(replicationEndpoint);
-        if(newReplicationEndPoint != null) {
+      if (rsServerHost != null) {
+        ReplicationEndpoint newReplicationEndPoint =
+            rsServerHost.postCreateReplicationEndPoint(replicationEndpoint);
+        if (newReplicationEndPoint != null) {
           // Override the newly created endpoint from the hook with configured end point
           replicationEndpoint = newReplicationEndPoint;
         }
       }
     } catch (Exception e) {
-      LOG.warn("Passed replication endpoint implementation throws errors"
-          + " while initializing ReplicationSource for peer: " + peerId, e);
+      LOG.warn("Passed replication endpoint implementation throws errors" +
+        " while initializing ReplicationSource for peer: " + peerId, e);
       throw new IOException(e);
     }
 
@@ -539,8 +537,8 @@ public class ReplicationSourceManager implements ReplicationListener {
       replicationEndpoint, walFileLengthProvider, metrics);
 
     // init replication endpoint
-    replicationEndpoint.init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(),
-      fs, peerId, clusterId, replicationPeer, metrics, tableDescriptors, server));
+    replicationEndpoint.init(new ReplicationEndpoint.Context(conf, peer.getConfiguration(), fs,
+        peerId, clusterId, peer, metrics, tableDescriptors, server));
 
     return src;
   }
@@ -736,17 +734,6 @@ public class ReplicationSourceManager implements ReplicationListener {
             abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
             continue;
           }
-
-          ReplicationPeerConfig peerConfig = null;
-          try {
-            peerConfig = replicationPeers.getPeerConfig(actualPeerId);
-          } catch (Exception e) {
-            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS
-                + ", failed to read peer config", e);
-            abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
-            continue;
-          }
-
           // track sources in walsByIdRecoveredQueues
           Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
           walsByIdRecoveredQueues.put(peerId, walsByGroup);
@@ -761,7 +748,7 @@ public class ReplicationSourceManager implements ReplicationListener {
           }
 
           // enqueue sources
-          ReplicationSourceInterface src = getReplicationSource(peerId, peerConfig, peer);
+          ReplicationSourceInterface src = getReplicationSource(peerId, peer);
           // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
           // see removePeer
           synchronized (oldsources) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b28956f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
index b09a8a7..1300376 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
@@ -15,19 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.client.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.TestReplicationBase;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -39,6 +35,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
 @Category({ MediumTests.class, ClientTests.class })
 public class TestReplicationAdminUsingProcedure extends TestReplicationBase {
 


[24/48] hbase git commit: HBASE-19579 Add peer lock test for shell command list_locks

Posted by zh...@apache.org.
HBASE-19579 Add peer lock test for shell command list_locks

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 80a36ad5ed7214571657bdfcbdd3c45b92f61823
Parents: 050617d
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Dec 23 21:04:27 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/LockService.proto          |  1 +
 .../src/test/ruby/shell/list_locks_test.rb       | 19 +++++++++++++++++++
 2 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/80a36ad5/hbase-protocol-shaded/src/main/protobuf/LockService.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/LockService.proto b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
index b8d180c..0675070 100644
--- a/hbase-protocol-shaded/src/main/protobuf/LockService.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
@@ -77,6 +77,7 @@ enum LockedResourceType {
   NAMESPACE = 2;
   TABLE = 3;
   REGION = 4;
+  PEER = 5;
 }
 
 message LockedResource {

http://git-wip-us.apache.org/repos/asf/hbase/blob/80a36ad5/hbase-shell/src/test/ruby/shell/list_locks_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/list_locks_test.rb b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
index f465a6b..ef1c0ce 100644
--- a/hbase-shell/src/test/ruby/shell/list_locks_test.rb
+++ b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
@@ -67,6 +67,25 @@ module Hbase
         proc_id)
     end
 
+    define_test 'list peer locks' do
+      lock = create_exclusive_lock(0)
+      peer_id = '1'
+
+      @scheduler.waitPeerExclusiveLock(lock, peer_id)
+      output = capture_stdout { @list_locks.command }
+      @scheduler.wakePeerExclusiveLock(lock, peer_id)
+
+      assert_equal(
+        "PEER(1)\n" \
+        "Lock type: EXCLUSIVE, procedure: {" \
+          "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+          "\"procId\"=>\"0\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \
+          "\"lastUpdate\"=>\"0\", " \
+          "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \
+        "}\n\n",
+        output)
+    end
+
     define_test 'list server locks' do
       lock = create_exclusive_lock(0)
 


[12/48] hbase git commit: HBASE-19797 Operator priority leads to wrong logic in ReplicationSourceWALReader

Posted by zh...@apache.org.
HBASE-19797 Operator priority leads to wrong logic in ReplicationSourceWALReader


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: e014e5f8efc44bfeab17c76da6ddfe5e54fa3513
Parents: 204a453
Author: zhangduo <zh...@apache.org>
Authored: Mon Jan 15 15:20:13 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:10:56 2018 +0800

----------------------------------------------------------------------
 .../replication/regionserver/ReplicationSourceWALReader.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e014e5f8/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index 1ec797f..4643a22 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -204,8 +204,8 @@ public class ReplicationSourceWALReader extends Thread {
   // (highly likely we've closed the current log), we've hit the max retries, and autorecovery is
   // enabled, then dump the log
   private void handleEofException(IOException e) {
-    if (e instanceof EOFException ||
-        e.getCause() instanceof EOFException && logQueue.size() > 1 && this.eofAutoRecovery) {
+    if ((e instanceof EOFException || e.getCause() instanceof EOFException) &&
+      logQueue.size() > 1 && this.eofAutoRecovery) {
       try {
         if (fs.getFileStatus(logQueue.peek()).getLen() == 0) {
           LOG.warn("Forcing removal of 0 length log in queue: " + logQueue.peek());


[48/48] hbase git commit: HBASE-19661 Replace ReplicationStateZKBase with ZKReplicationStorageBase

Posted by zh...@apache.org.
HBASE-19661 Replace ReplicationStateZKBase with ZKReplicationStorageBase


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 72dbeedceb311bd3dc8fa3db8b10073017ffda2a
Parents: eb0bb20
Author: huzheng <op...@gmail.com>
Authored: Fri Dec 29 15:55:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |   5 +-
 .../replication/ReplicationStateZKBase.java     | 153 -------------------
 .../replication/ReplicationTrackerZKImpl.java   |  21 +--
 .../replication/ZKReplicationPeerStorage.java   |  24 ++-
 .../replication/ZKReplicationStorageBase.java   |  13 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   4 +-
 .../master/ReplicationPeerConfigUpgrader.java   | 128 ++++++++--------
 .../regionserver/DumpReplicationQueues.java     |  18 +--
 .../replication/regionserver/Replication.java   |   3 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   3 +-
 .../TestReplicationTrackerZKImpl.java           |   3 +-
 .../replication/master/TestTableCFsUpdater.java |  41 ++---
 .../TestReplicationSourceManager.java           |   6 +-
 13 files changed, 136 insertions(+), 286 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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 6c66aff..2a970ba 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
@@ -33,9 +33,8 @@ public class ReplicationFactory {
     return new ReplicationPeers(zk, conf);
   }
 
-  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,
-      final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
+  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, Abortable abortable,
       Stoppable stopper) {
-    return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, abortable, stopper);
+    return new ReplicationTrackerZKImpl(zookeeper, abortable, stopper);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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
deleted file mode 100644
index f49537c..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ /dev/null
@@ -1,153 +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.replication;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.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.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * This is a base class for maintaining replication state in zookeeper.
- */
-@InterfaceAudience.Private
-public abstract class ReplicationStateZKBase {
-
-  /**
-   * The name of the znode that contains the replication status of a remote slave (i.e. peer)
-   * cluster.
-   */
-  protected final String peerStateNodeName;
-  /** The name of the base znode that contains all replication state. */
-  protected final String replicationZNode;
-  /** The name of the znode that contains a list of all remote slave (i.e. peer) clusters. */
-  protected final String peersZNode;
-  /** The name of the znode that contains all replication queues */
-  protected final String queuesZNode;
-  /** The name of the znode that contains queues of hfile references to be replicated */
-  protected final String hfileRefsZNode;
-  /** The cluster key of the local cluster */
-  protected final String ourClusterKey;
-  /** The name of the znode that contains tableCFs */
-  protected final String tableCFsNodeName;
-
-  protected final ZKWatcher zookeeper;
-  protected final Configuration conf;
-  protected final Abortable abortable;
-
-  public static final byte[] ENABLED_ZNODE_BYTES =
-      toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
-  public static final byte[] DISABLED_ZNODE_BYTES =
-      toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
-  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
-      "zookeeper.znode.replication.hfile.refs";
-  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
-
-  public ReplicationStateZKBase(ZKWatcher zookeeper, Configuration conf,
-                                Abortable abortable) {
-    this.zookeeper = zookeeper;
-    this.conf = conf;
-    this.abortable = abortable;
-
-    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
-    String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
-    String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
-    String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
-      ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
-    this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
-    this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
-    this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
-    this.replicationZNode = ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode,
-      replicationZNodeName);
-    this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);
-    this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName);
-    this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName);
-  }
-
-  public List<String> getListOfReplicators() {
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of replicators", e);
-    }
-    return result;
-  }
-
-  /**
-   * @param state
-   * @return Serialized protobuf of <code>state</code> with pb magic prefix prepended suitable for
-   *         use as content of a peer-state znode under a peer cluster id as in
-   *         /hbase/replication/peers/PEER_ID/peer-state.
-   */
-  protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) {
-    ReplicationProtos.ReplicationState msg =
-        ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
-    // There is no toByteArray on this pb Message?
-    // 32 bytes is default which seems fair enough here.
-    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
-      CodedOutputStream cos = CodedOutputStream.newInstance(baos, 16);
-      msg.writeTo(cos);
-      cos.flush();
-      baos.flush();
-      return ProtobufUtil.prependPBMagic(baos.toByteArray());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  protected boolean peerExists(String id) throws KeeperException {
-    return ZKUtil.checkExists(this.zookeeper, ZNodePaths.joinZNode(this.peersZNode, id)) >= 0;
-  }
-
-  /**
-   * Determine if a ZK path points to a peer node.
-   * @param path path to be checked
-   * @return true if the path points to a peer node, otherwise false
-   */
-  protected boolean isPeerPath(String path) {
-    return path.split("/").length == peersZNode.split("/").length + 1;
-  }
-
-  @VisibleForTesting
-  protected String getTableCFsNode(String id) {
-    return ZNodePaths.joinZNode(this.peersZNode, ZNodePaths.joinZNode(id, this.tableCFsNodeName));
-  }
-
-  @VisibleForTesting
-  protected String getPeerStateNode(String id) {
-    return ZNodePaths.joinZNode(this.peersZNode, ZNodePaths.joinZNode(id, this.peerStateNodeName));
-  }
-  @VisibleForTesting
-  protected String getPeerNode(String id) {
-    return ZNodePaths.joinZNode(this.peersZNode, id);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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 5659e4b..16a1668 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
@@ -20,14 +20,12 @@ package org.apache.hadoop.hbase.replication;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
-
-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.ZKListener;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,9 +36,14 @@ import org.slf4j.LoggerFactory;
  * interface.
  */
 @InterfaceAudience.Private
-public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements ReplicationTracker {
+public class ReplicationTrackerZKImpl implements ReplicationTracker {
 
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationTrackerZKImpl.class);
+
+  // Zookeeper
+  private final ZKWatcher zookeeper;
+  // Server to abort.
+  private final Abortable abortable;
   // All about stopping
   private final Stoppable stopper;
   // listeners to be notified
@@ -48,9 +51,9 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   // List of all the other region servers in this cluster
   private final ArrayList<String> otherRegionServers = new ArrayList<>();
 
-  public ReplicationTrackerZKImpl(ZKWatcher zookeeper, final ReplicationPeers replicationPeers,
-      Configuration conf, Abortable abortable, Stoppable stopper) {
-    super(zookeeper, conf, abortable);
+  public ReplicationTrackerZKImpl(ZKWatcher zookeeper, Abortable abortable, Stoppable stopper) {
+    this.zookeeper = zookeeper;
+    this.abortable = abortable;
     this.stopper = stopper;
     this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper));
     // watch the changes

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
index 42d4b3f..a53500a 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 
@@ -36,7 +37,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
  * ZK based replication peer storage.
  */
 @InterfaceAudience.Private
-class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements ReplicationPeerStorage {
+public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
+    implements ReplicationPeerStorage {
+
+  public static final String PEERS_ZNODE = "zookeeper.znode.replication.peers";
+  public static final String PEERS_ZNODE_DEFAULT = "peers";
+
+  public static final String PEERS_STATE_ZNODE = "zookeeper.znode.replication.peers.state";
+  public static final String PEERS_STATE_ZNODE_DEFAULT = "peer-state";
 
   public static final byte[] ENABLED_ZNODE_BYTES =
     toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
@@ -56,16 +64,18 @@ class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements Repli
 
   public ZKReplicationPeerStorage(ZKWatcher zookeeper, Configuration conf) {
     super(zookeeper, conf);
-    this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
-    String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
+    this.peerStateNodeName = conf.get(PEERS_STATE_ZNODE, PEERS_STATE_ZNODE_DEFAULT);
+    String peersZNodeName = conf.get(PEERS_ZNODE, PEERS_ZNODE_DEFAULT);
     this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);
   }
 
-  private String getPeerStateNode(String peerId) {
+  @VisibleForTesting
+  public String getPeerStateNode(String peerId) {
     return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName);
   }
 
-  private String getPeerNode(String peerId) {
+  @VisibleForTesting
+  public String getPeerNode(String peerId) {
     return ZNodePaths.joinZNode(peersZNode, peerId);
   }
 
@@ -82,8 +92,8 @@ class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements Repli
             enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES)),
         false);
     } catch (KeeperException e) {
-      throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>" +
-        peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e);
+      throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>"
+          + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
index 2321e4f..7190aeb 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.replication;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
@@ -34,7 +35,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
  * zookeeper.
  */
 @InterfaceAudience.Private
-class ZKReplicationStorageBase {
+public class ZKReplicationStorageBase {
+
+  public static final String REPLICATION_ZNODE = "zookeeper.znode.replication";
+  public static final String REPLICATION_ZNODE_DEFAULT = "replication";
 
   /** The name of the base znode that contains all replication state. */
   protected final String replicationZNode;
@@ -45,10 +49,9 @@ class ZKReplicationStorageBase {
   protected ZKReplicationStorageBase(ZKWatcher zookeeper, Configuration conf) {
     this.zookeeper = zookeeper;
     this.conf = conf;
-    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
 
-    this.replicationZNode =
-        ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode, replicationZNodeName);
+    this.replicationZNode = ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode,
+      conf.get(REPLICATION_ZNODE, REPLICATION_ZNODE_DEFAULT));
   }
 
   /**
@@ -58,7 +61,7 @@ class ZKReplicationStorageBase {
    */
   protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) {
     ReplicationProtos.ReplicationState msg =
-      ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
+        ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
     // There is no toByteArray on this pb Message?
     // 32 bytes is default which seems fair enough here.
     try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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 df5e466..58a0f4b 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
@@ -829,8 +829,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     // This is for backwards compatibility
     // See HBASE-11393
     status.setStatus("Update TableCFs node in ZNode");
-    ReplicationPeerConfigUpgrader tableCFsUpdater = new ReplicationPeerConfigUpgrader(zooKeeper,
-            conf, this.clusterConnection);
+    ReplicationPeerConfigUpgrader tableCFsUpdater =
+        new ReplicationPeerConfigUpgrader(zooKeeper, conf);
     tableCFsUpdater.copyTableCFs();
 
     // Add the Observer to delete space quotas on table deletion before starting all CPs by

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
index ea5509f..b6e8862 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
@@ -18,96 +18,107 @@
  */
 package org.apache.hadoop.hbase.replication.master;
 
+import static org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage.PEERS_ZNODE;
+import static org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage.PEERS_ZNODE_DEFAULT;
+import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLICATION_ZNODE;
+import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLICATION_ZNODE_DEFAULT;
+
 import java.io.IOException;
-import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 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.yetus.audience.InterfaceStability;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
- * This class is used to upgrade TableCFs from HBase 1.0, 1.1, 1.2, 1.3 to HBase 1.4 or 2.x.
- * It will be removed in HBase 3.x. See HBASE-11393
+ * This class is used to upgrade TableCFs from HBase 1.0, 1.1, 1.2, 1.3 to HBase 1.4 or 2.x. It will
+ * be removed in HBase 3.x. See HBASE-11393
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class ReplicationPeerConfigUpgrader extends ReplicationStateZKBase {
+public class ReplicationPeerConfigUpgrader{
+
+  private static final String TABLE_CFS_ZNODE = "zookeeper.znode.replication.peers.tableCFs";
+  private static final String TABLE_CFS_ZNODE_DEFAULT = "tableCFs";
 
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerConfigUpgrader.class);
+  private final Configuration conf;
+  private final ZKWatcher zookeeper;
+  private final ReplicationPeerStorage peerStorage;
 
-  public ReplicationPeerConfigUpgrader(ZKWatcher zookeeper,
-                         Configuration conf, Abortable abortable) {
-    super(zookeeper, conf, abortable);
+  public ReplicationPeerConfigUpgrader(ZKWatcher zookeeper, Configuration conf) {
+    this.zookeeper = zookeeper;
+    this.conf = conf;
+    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zookeeper, conf);
   }
 
   public void upgrade() throws Exception {
     try (Connection conn = ConnectionFactory.createConnection(conf)) {
       Admin admin = conn.getAdmin();
-      admin.listReplicationPeers().forEach(
-        (peerDesc) -> {
-          String peerId = peerDesc.getPeerId();
-          ReplicationPeerConfig peerConfig = peerDesc.getPeerConfig();
-          if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty())
-              || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-            peerConfig.setReplicateAllUserTables(false);
-            try {
-              admin.updateReplicationPeerConfig(peerId, peerConfig);
-            } catch (Exception e) {
-              LOG.error("Failed to upgrade replication peer config for peerId=" + peerId, e);
-            }
+      admin.listReplicationPeers().forEach((peerDesc) -> {
+        String peerId = peerDesc.getPeerId();
+        ReplicationPeerConfig peerConfig = peerDesc.getPeerConfig();
+        if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty())
+            || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+          peerConfig.setReplicateAllUserTables(false);
+          try {
+            admin.updateReplicationPeerConfig(peerId, peerConfig);
+          } catch (Exception e) {
+            LOG.error("Failed to upgrade replication peer config for peerId=" + peerId, e);
           }
-        });
+        }
+      });
     }
   }
 
-  public void copyTableCFs() {
-    List<String> znodes = null;
-    try {
-      znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
-    } catch (KeeperException e) {
-      LOG.error("Failed to get peers znode", e);
-    }
-    if (znodes != null) {
-      for (String peerId : znodes) {
-        if (!copyTableCFs(peerId)) {
-          LOG.error("upgrade tableCFs failed for peerId=" + peerId);
-        }
+  public void copyTableCFs() throws ReplicationException {
+    for (String peerId : peerStorage.listPeerIds()) {
+      if (!copyTableCFs(peerId)) {
+        LOG.error("upgrade tableCFs failed for peerId=" + peerId);
       }
     }
   }
 
-  public boolean copyTableCFs(String peerId) {
+  @VisibleForTesting
+  protected String getTableCFsNode(String peerId) {
+    String replicationZNode = ZNodePaths.joinZNode(zookeeper.znodePaths.baseZNode,
+      conf.get(REPLICATION_ZNODE, REPLICATION_ZNODE_DEFAULT));
+    String peersZNode =
+        ZNodePaths.joinZNode(replicationZNode, conf.get(PEERS_ZNODE, PEERS_ZNODE_DEFAULT));
+    return ZNodePaths.joinZNode(peersZNode,
+      ZNodePaths.joinZNode(peerId, conf.get(TABLE_CFS_ZNODE, TABLE_CFS_ZNODE_DEFAULT)));
+  }
+
+  public boolean copyTableCFs(String peerId) throws ReplicationException {
     String tableCFsNode = getTableCFsNode(peerId);
     try {
       if (ZKUtil.checkExists(zookeeper, tableCFsNode) != -1) {
-        String peerNode = getPeerNode(peerId);
-        ReplicationPeerConfig rpc = getReplicationPeerConig(peerNode);
+        ReplicationPeerConfig rpc = peerStorage.getPeerConfig(peerId);
         // We only need to copy data from tableCFs node to rpc Node the first time hmaster start.
         if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().isEmpty()) {
           // we copy TableCFs node into PeerNode
           LOG.info("copy tableCFs into peerNode:" + peerId);
           ReplicationProtos.TableCF[] tableCFs =
-                  ReplicationPeerConfigUtil.parseTableCFs(
-                          ZKUtil.getData(this.zookeeper, tableCFsNode));
+              ReplicationPeerConfigUtil.parseTableCFs(ZKUtil.getData(this.zookeeper, tableCFsNode));
           if (tableCFs != null && tableCFs.length > 0) {
             rpc.setTableCFsMap(ReplicationPeerConfigUtil.convert2Map(tableCFs));
-            ZKUtil.setData(this.zookeeper, peerNode,
-              ReplicationPeerConfigUtil.toByteArray(rpc));
+            peerStorage.updatePeerConfig(peerId, rpc);
           }
         } else {
           LOG.info("No tableCFs in peerNode:" + peerId);
@@ -126,23 +137,6 @@ public class ReplicationPeerConfigUpgrader extends ReplicationStateZKBase {
     return true;
   }
 
-  private ReplicationPeerConfig getReplicationPeerConig(String peerNode)
-          throws KeeperException, InterruptedException {
-    byte[] data = null;
-    data = ZKUtil.getData(this.zookeeper, peerNode);
-    if (data == null) {
-      LOG.error("Could not get configuration for " +
-              "peer because it doesn't exist. peer=" + peerNode);
-      return null;
-    }
-    try {
-      return ReplicationPeerConfigUtil.parsePeerFrom(data);
-    } catch (DeserializationException e) {
-      LOG.warn("Failed to parse cluster key from peer=" + peerNode);
-      return null;
-    }
-  }
-
   private static void printUsageAndExit() {
     System.err.printf(
       "Usage: hbase org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader"
@@ -163,19 +157,17 @@ public class ReplicationPeerConfigUpgrader extends ReplicationStateZKBase {
       printUsageAndExit();
     } else if (args[0].equals("copyTableCFs")) {
       Configuration conf = HBaseConfiguration.create();
-      ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null);
-      try {
-        ReplicationPeerConfigUpgrader tableCFsUpdater = new ReplicationPeerConfigUpgrader(zkw,
-            conf, null);
+      try (ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null)) {
+        ReplicationPeerConfigUpgrader tableCFsUpdater =
+            new ReplicationPeerConfigUpgrader(zkw, conf);
         tableCFsUpdater.copyTableCFs();
-      } finally {
-        zkw.close();
       }
     } else if (args[0].equals("upgrade")) {
       Configuration conf = HBaseConfiguration.create();
-      ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null);
-      ReplicationPeerConfigUpgrader upgrader = new ReplicationPeerConfigUpgrader(zkw, conf, null);
-      upgrader.upgrade();
+      try (ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null)) {
+        ReplicationPeerConfigUpgrader upgrader = new ReplicationPeerConfigUpgrader(zkw, conf);
+        upgrader.upgrade();
+      }
     } else {
       printUsageAndExit();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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 27bda2d..22e8628 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
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
@@ -237,7 +236,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
         LOG.info("Found [--distributed], will poll each RegionServer.");
         Set<String> peerIds = peers.stream().map((peer) -> peer.getPeerId())
             .collect(Collectors.toSet());
-        System.out.println(dumpQueues(connection, zkw, peerIds, opts.isHdfs()));
+        System.out.println(dumpQueues(zkw, peerIds, opts.isHdfs()));
         System.out.println(dumpReplicationSummary());
       } else {
         // use ZK instead
@@ -301,18 +300,15 @@ public class DumpReplicationQueues extends Configured implements Tool {
     return sb.toString();
   }
 
-  public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set<String> peerIds,
+  public String dumpQueues(ZKWatcher zkw, Set<String> peerIds,
       boolean hdfs) throws Exception {
     ReplicationQueueStorage queueStorage;
-    ReplicationPeers replicationPeers;
     ReplicationTracker replicationTracker;
     StringBuilder sb = new StringBuilder();
 
     queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
-    replicationPeers =
-        ReplicationFactory.getReplicationPeers(zkw, getConf());
-    replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
-      new WarnOnlyAbortable(), new WarnOnlyStoppable());
+    replicationTracker = ReplicationFactory.getReplicationTracker(zkw, new WarnOnlyAbortable(),
+      new WarnOnlyStoppable());
     Set<String> liveRegionServers = new HashSet<>(replicationTracker.getListOfRegionServers());
 
     // Loops each peer on each RS and dumps the queues
@@ -330,11 +326,9 @@ public class DumpReplicationQueues extends Configured implements Tool {
         List<String> wals = queueStorage.getWALsInQueue(regionserver, queueId);
         if (!peerIds.contains(queueInfo.getPeerId())) {
           deletedQueues.add(regionserver + "/" + queueId);
-          sb.append(
-            formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, true, hdfs));
+          sb.append(formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, true, hdfs));
         } else {
-          sb.append(
-            formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, false, hdfs));
+          sb.append(formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, false, hdfs));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index f985f90..dca2439 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -132,8 +132,7 @@ public class Replication implements
           ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf);
       this.replicationPeers.init();
       this.replicationTracker =
-          ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.replicationPeers,
-            this.conf, this.server, this.server);
+          ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.server, this.server);
     } catch (Exception e) {
       throw new IOException("Failed replication handler create", e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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 8b5a4e6..40ad2bc 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
@@ -330,8 +330,7 @@ public class HBaseFsck extends Configured implements Closeable {
    * @throws MasterNotRunningException if the master is not running
    * @throws ZooKeeperConnectionException if unable to connect to ZooKeeper
    */
-  public HBaseFsck(Configuration conf) throws MasterNotRunningException,
-      ZooKeeperConnectionException, IOException, ClassNotFoundException {
+  public HBaseFsck(Configuration conf) throws IOException, ClassNotFoundException {
     this(conf, createThreadPool(conf));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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 fdfa6b7..757d9a9 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
@@ -90,7 +90,8 @@ public class TestReplicationTrackerZKImpl {
       ZKClusterId.setClusterId(zkw, new ClusterId());
       rp = ReplicationFactory.getReplicationPeers(zkw, conf);
       rp.init();
-      rt = ReplicationFactory.getReplicationTracker(zkw, rp, conf, zkw, new DummyServer(fakeRs1));
+      rt = ReplicationFactory.getReplicationTracker(zkw, new DummyServer(fakeRs1),
+        new DummyServer(fakeRs1));
     } catch (Exception e) {
       fail("Exception during test setup: " + e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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 2993043..19acc75 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
@@ -25,14 +25,13 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage;
 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.ZKWatcher;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -57,12 +56,19 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
 
   private static ZKWatcher zkw = null;
   private static Abortable abortable = null;
+  private static ZKStorageUtil zkStorageUtil = null;
+
+  private static class ZKStorageUtil extends ZKReplicationPeerStorage {
+    public ZKStorageUtil(ZKWatcher zookeeper, Configuration conf) {
+      super(zookeeper, conf);
+    }
+  }
 
   @Rule
   public TestName name = new TestName();
 
   public TestTableCFsUpdater() {
-    super(zkw, TEST_UTIL.getConfiguration(), abortable);
+    super(zkw, TEST_UTIL.getConfiguration());
   }
 
   @BeforeClass
@@ -81,6 +87,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
       }
     };
     zkw = new ZKWatcher(conf, "TableCFs", abortable, true);
+    zkStorageUtil = new ZKStorageUtil(zkw, conf);
   }
 
   @AfterClass
@@ -89,8 +96,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
   }
 
   @Test
-  public void testUpgrade() throws KeeperException, InterruptedException,
-      DeserializationException {
+  public void testUpgrade() throws Exception {
     String peerId = "1";
     final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1");
     final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2");
@@ -98,13 +104,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
 
     ReplicationPeerConfig rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(zkw.getQuorum());
-    String peerNode = getPeerNode(peerId);
+    String peerNode = zkStorageUtil.getPeerNode(peerId);
     ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc));
 
     String tableCFs = tableName1 + ":cf1,cf2;" + tableName2 + ":cf3;" + tableName3;
     String tableCFsNode = getTableCFsNode(peerId);
     LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
-    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+    ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs));
 
     ReplicationPeerConfig actualRpc =
         ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
@@ -117,13 +123,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     peerId = "2";
     rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(zkw.getQuorum());
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc));
 
     tableCFs = tableName1 + ":cf1,cf3;" + tableName2 + ":cf2";
     tableCFsNode = getTableCFsNode(peerId);
     LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
-    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+    ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs));
 
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
@@ -135,13 +141,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     peerId = "3";
     rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(zkw.getQuorum());
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc));
 
     tableCFs = "";
     tableCFsNode = getTableCFsNode(peerId);
     LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
-    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+    ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs));
 
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
@@ -153,7 +159,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     peerId = "4";
     rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(zkw.getQuorum());
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc));
 
     tableCFsNode = getTableCFsNode(peerId);
@@ -167,7 +173,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     copyTableCFs();
 
     peerId = "1";
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
     Map<TableName, List<String>> tableNameListMap = actualRpc.getTableCFsMap();
@@ -182,9 +188,8 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     assertEquals("cf3", tableNameListMap.get(tableName2).get(0));
     assertNull(tableNameListMap.get(tableName3));
 
-
     peerId = "2";
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
     tableNameListMap = actualRpc.getTableCFsMap();
@@ -198,19 +203,17 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     assertEquals("cf2", tableNameListMap.get(tableName2).get(0));
 
     peerId = "3";
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
     tableNameListMap = actualRpc.getTableCFsMap();
     assertNull(tableNameListMap);
 
     peerId = "4";
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
     tableNameListMap = actualRpc.getTableCFsMap();
     assertNull(tableNameListMap);
   }
-
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72dbeedc/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 f4d3901..1001aa5 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
@@ -68,8 +68,8 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
-import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -163,9 +163,9 @@ public abstract class TestReplicationSourceManager {
             + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":/1"));
     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state");
     ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state",
-      ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
+      ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
     ZKUtil.createWithParents(zkw, "/hbase/replication/state");
-    ZKUtil.setData(zkw, "/hbase/replication/state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
+    ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
 
     ZKClusterId.setClusterId(zkw, new ClusterId());
     FSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir());


[29/48] hbase git commit: HBASE-19592 Add UTs to test retry on update zk failure

Posted by zh...@apache.org.
HBASE-19592 Add UTs to test retry on update zk failure


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 67c319e0e64b8f91268ed6d96d13cb346c931339
Parents: b8af6a4
Author: zhangduo <zh...@apache.org>
Authored: Tue Dec 26 20:39:00 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerManager.java     |   5 +-
 .../TestReplicationProcedureRetry.java          | 200 +++++++++++++++++++
 2 files changed, 202 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/67c319e0/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index b78cbce..f4ccce8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -53,7 +53,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Used to add/remove a replication peer.
  */
 @InterfaceAudience.Private
-public final class ReplicationPeerManager {
+public class ReplicationPeerManager {
 
   private final ReplicationPeerStorage peerStorage;
 
@@ -61,8 +61,7 @@ public final class ReplicationPeerManager {
 
   private final ConcurrentMap<String, ReplicationPeerDescription> peers;
 
-  private ReplicationPeerManager(ReplicationPeerStorage peerStorage,
-      ReplicationQueueStorage queueStorage,
+  ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage,
       ConcurrentMap<String, ReplicationPeerDescription> peers) {
     this.peerStorage = peerStorage;
     this.queueStorage = queueStorage;

http://git-wip-us.apache.org/repos/asf/hbase/blob/67c319e0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
new file mode 100644
index 0000000..ab35b46
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
@@ -0,0 +1,200 @@
+/**
+ * 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.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+
+/**
+ * All the modification method will fail once in the test and should finally succeed.
+ */
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestReplicationProcedureRetry {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setClass(HConstants.MASTER_IMPL, MockHMaster.class, HMaster.class);
+    UTIL.startMiniCluster(3);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @After
+  public void tearDownAfterTest() throws IOException {
+    for (ReplicationPeerDescription desc : UTIL.getAdmin().listReplicationPeers()) {
+      UTIL.getAdmin().removeReplicationPeer(desc.getPeerId());
+    }
+  }
+
+  private void doTest() throws IOException {
+    Admin admin = UTIL.getAdmin();
+    String peerId = "1";
+    ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
+        .setClusterKey("localhost:" + UTIL.getZkCluster().getClientPort() + ":/hbase2").build();
+    admin.addReplicationPeer(peerId, peerConfig, true);
+
+    assertEquals(peerConfig.getClusterKey(),
+      admin.getReplicationPeerConfig(peerId).getClusterKey());
+    ReplicationPeerConfig newPeerConfig =
+        ReplicationPeerConfig.newBuilder(peerConfig).setBandwidth(123456).build();
+    admin.updateReplicationPeerConfig(peerId, newPeerConfig);
+    assertEquals(newPeerConfig.getBandwidth(),
+      admin.getReplicationPeerConfig(peerId).getBandwidth());
+
+    admin.disableReplicationPeer(peerId);
+    assertFalse(admin.listReplicationPeers().get(0).isEnabled());
+
+    admin.enableReplicationPeer(peerId);
+    assertTrue(admin.listReplicationPeers().get(0).isEnabled());
+
+    admin.removeReplicationPeer(peerId);
+    assertTrue(admin.listReplicationPeers().isEmpty());
+
+    // make sure that we have run into the mocked method
+    MockHMaster master = (MockHMaster) UTIL.getHBaseCluster().getMaster();
+    assertTrue(master.addPeerCalled);
+    assertTrue(master.removePeerCalled);
+    assertTrue(master.updatePeerConfigCalled);
+    assertTrue(master.enablePeerCalled);
+    assertTrue(master.disablePeerCalled);
+  }
+
+  @Test
+  public void testErrorBeforeUpdate() throws IOException, ReplicationException {
+    ((MockHMaster) UTIL.getHBaseCluster().getMaster()).reset(true);
+    doTest();
+  }
+
+  @Test
+  public void testErrorAfterUpdate() throws IOException, ReplicationException {
+    ((MockHMaster) UTIL.getHBaseCluster().getMaster()).reset(false);
+    doTest();
+  }
+
+  public static final class MockHMaster extends HMaster {
+
+    volatile boolean addPeerCalled;
+
+    volatile boolean removePeerCalled;
+
+    volatile boolean updatePeerConfigCalled;
+
+    volatile boolean enablePeerCalled;
+
+    volatile boolean disablePeerCalled;
+
+    private ReplicationPeerManager manager;
+
+    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+      super(conf);
+    }
+
+    private Object invokeWithError(InvocationOnMock invocation, boolean errorBeforeUpdate)
+        throws Throwable {
+      if (errorBeforeUpdate) {
+        throw new ReplicationException("mock error before update");
+      }
+      invocation.callRealMethod();
+      throw new ReplicationException("mock error after update");
+    }
+
+    public void reset(boolean errorBeforeUpdate) throws ReplicationException {
+      addPeerCalled = false;
+      removePeerCalled = false;
+      updatePeerConfigCalled = false;
+      enablePeerCalled = false;
+      disablePeerCalled = false;
+      ReplicationPeerManager m = super.getReplicationPeerManager();
+      manager = spy(m);
+      doAnswer(invocation -> {
+        if (!addPeerCalled) {
+          addPeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).addPeer(anyString(), any(ReplicationPeerConfig.class), anyBoolean());
+      doAnswer(invocation -> {
+        if (!removePeerCalled) {
+          removePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).removePeer(anyString());
+      doAnswer(invocation -> {
+        if (!updatePeerConfigCalled) {
+          updatePeerConfigCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).updatePeerConfig(anyString(), any(ReplicationPeerConfig.class));
+      doAnswer(invocation -> {
+        if (!enablePeerCalled) {
+          enablePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).enablePeer(anyString());
+      doAnswer(invocation -> {
+        if (!disablePeerCalled) {
+          disablePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).disablePeer(anyString());
+    }
+
+    @Override
+    public ReplicationPeerManager getReplicationPeerManager() {
+      return manager;
+    }
+  }
+}


[17/48] hbase git commit: HBASE-19543 Abstract a replication storage interface to extract the zk specific code

Posted by zh...@apache.org.
HBASE-19543 Abstract a replication storage interface to extract the zk specific code


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: ef8c27c36e0cb36fe82dbfff132aac4cb2953fa3
Parents: 1b2ce99
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 22 14:37:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/util/CollectionUtils.java      |   3 +
 hbase-replication/pom.xml                       |  12 +
 .../replication/ReplicationPeerStorage.java     |  74 ++++
 .../replication/ReplicationQueueStorage.java    | 164 +++++++
 .../replication/ReplicationStateZKBase.java     |   1 -
 .../replication/ReplicationStorageFactory.java  |  49 +++
 .../replication/ZKReplicationPeerStorage.java   | 164 +++++++
 .../replication/ZKReplicationQueueStorage.java  | 425 +++++++++++++++++++
 .../replication/ZKReplicationStorageBase.java   |  75 ++++
 .../TestZKReplicationPeerStorage.java           | 171 ++++++++
 .../TestZKReplicationQueueStorage.java          | 171 ++++++++
 .../org/apache/hadoop/hbase/master/HMaster.java |  36 +-
 .../hadoop/hbase/master/MasterServices.java     |   6 +-
 .../master/procedure/MasterProcedureEnv.java    |  24 +-
 .../master/replication/AddPeerProcedure.java    |   6 +-
 .../replication/DisablePeerProcedure.java       |   7 +-
 .../master/replication/EnablePeerProcedure.java |   6 +-
 .../master/replication/ModifyPeerProcedure.java |  41 +-
 .../master/replication/RemovePeerProcedure.java |   6 +-
 .../master/replication/ReplicationManager.java  | 199 ---------
 .../replication/ReplicationPeerManager.java     | 331 +++++++++++++++
 .../replication/UpdatePeerConfigProcedure.java  |   7 +-
 .../replication/TestReplicationAdmin.java       |  63 ++-
 .../hbase/master/MockNoopMasterServices.java    |  12 +-
 .../hbase/master/TestMasterNoCluster.java       |   4 +-
 .../TestReplicationDisableInactivePeer.java     |   6 +-
 26 files changed, 1750 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
index 875b124..8bbb6f1 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
@@ -107,6 +107,9 @@ public class CollectionUtils {
     return list.get(list.size() - 1);
   }
 
+  public static <T> List<T> nullToEmpty(List<T> list) {
+    return list != null ? list : Collections.emptyList();
+  }
   /**
    * In HBASE-16648 we found that ConcurrentHashMap.get is much faster than computeIfAbsent if the
    * value already exists. Notice that the implementation does not guarantee that the supplier will

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index bd593d3..b28e852 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -121,6 +121,18 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-zookeeper</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-zookeeper</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
     <!-- General dependencies -->
     <dependency>
       <groupId>org.apache.commons</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
new file mode 100644
index 0000000..e00cd0d
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
@@ -0,0 +1,74 @@
+/**
+ * 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.replication;
+
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Perform read/write to the replication peer storage.
+ */
+@InterfaceAudience.Private
+public interface ReplicationPeerStorage {
+
+  /**
+   * Add a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException;
+
+  /**
+   * Remove a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void removePeer(String peerId) throws ReplicationException;
+
+  /**
+   * Set the state of peer, {@code true} to {@code ENABLED}, otherwise to {@code DISABLED}.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void setPeerState(String peerId, boolean enabled) throws ReplicationException;
+
+  /**
+   * Update the config a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException;
+
+  /**
+   * Return the peer ids of all replication peers.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  List<String> listPeerIds() throws ReplicationException;
+
+  /**
+   * Test whether a replication peer is enabled.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  boolean isPeerEnabled(String peerId) throws ReplicationException;
+
+  /**
+   * Get the peer config of a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  Optional<ReplicationPeerConfig> getPeerConfig(String peerId) throws ReplicationException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
new file mode 100644
index 0000000..7210d9a
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.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.replication;
+
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Perform read/write to the replication queue storage.
+ */
+@InterfaceAudience.Private
+public interface ReplicationQueueStorage {
+
+  /**
+   * Remove a replication queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   */
+  void removeQueue(ServerName serverName, String queueId) throws ReplicationException;
+
+  /**
+   * Add a new WAL file to the given queue for a given regionserver. If the queue does not exist it
+   * is created.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   * @param fileName name of the WAL
+   */
+  void addWAL(ServerName serverName, String queueId, String fileName) throws ReplicationException;
+
+  /**
+   * Remove an WAL file from the given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   * @param fileName name of the WAL
+   */
+  void removeWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException;
+
+  /**
+   * Set the current position for a specific WAL in a given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue
+   * @param fileName name of the WAL
+   * @param position the current position in the file
+   */
+  void setWALPosition(ServerName serverName, String queueId, String fileName, long position)
+      throws ReplicationException;
+
+  /**
+   * Get the current position for a specific WAL in a given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue
+   * @param fileName name of the WAL
+   * @return the current position in the file
+   */
+  long getWALPosition(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException;
+
+  /**
+   * Get a list of all queues for the specified region server.
+   * @param serverName the server name of the region server that owns the set of queues
+   * @return a list of queueIds
+   */
+  List<String> getAllQueues(ServerName serverName) throws ReplicationException;
+
+  /**
+   * Change ownership for the queue identified by queueId and belongs to a dead region server.
+   * @param sourceServerName the name of the dead region server
+   * @param destServerName the name of the target region server
+   * @param queueId the id of the queue
+   * @return the new PeerId and A SortedSet of WALs in its queue
+   */
+  Pair<String, SortedSet<String>> claimQueue(ServerName sourceServerName, String queueId,
+      ServerName destServerName) throws ReplicationException;
+
+  /**
+   * Remove the record of region server if the queue is empty.
+   */
+  void removeReplicatorIfQueueIsEmpty(ServerName serverName) throws ReplicationException;
+
+  /**
+   * Get a list of all region servers that have outstanding replication queues. These servers could
+   * be alive, dead or from a previous run of the cluster.
+   * @return a list of server names
+   */
+  List<ServerName> getListOfReplicators() throws ReplicationException;
+
+  /**
+   * Load all wals in all replication queues. This method guarantees to return a snapshot which
+   * contains all WALs in the zookeeper at the start of this call even there is concurrent queue
+   * failover. However, some newly created WALs during the call may not be included.
+   */
+  Set<String> getAllWALs() throws ReplicationException;
+
+  /**
+   * Add a peer to hfile reference queue if peer does not exist.
+   * @param peerId peer cluster id to be added
+   * @throws ReplicationException if fails to add a peer id to hfile reference queue
+   */
+  void addPeerToHFileRefs(String peerId) throws ReplicationException;
+
+  /**
+   * Remove a peer from hfile reference queue.
+   * @param peerId peer cluster id to be removed
+   */
+  void removePeerFromHFileRefs(String peerId) throws ReplicationException;
+
+  /**
+   * Add new hfile references to the queue.
+   * @param peerId peer cluster id to which the hfiles need to be replicated
+   * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which
+   *          will be added in the queue }
+   * @throws ReplicationException if fails to add a hfile reference
+   */
+  void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs) throws ReplicationException;
+
+  /**
+   * Remove hfile references from the queue.
+   * @param peerId peer cluster id from which this hfile references needs to be removed
+   * @param files list of hfile references to be removed
+   */
+  void removeHFileRefs(String peerId, List<String> files) throws ReplicationException;
+
+  /**
+   * Get the change version number of replication hfile references node. This can be used as
+   * optimistic locking to get a consistent snapshot of the replication queues of hfile references.
+   * @return change version number of hfile references node
+   */
+  int getHFileRefsNodeChangeVersion() throws ReplicationException;
+
+  /**
+   * Get list of all peers from hfile reference queue.
+   * @return a list of peer ids
+   */
+  List<String> getAllPeersFromHFileRefsQueue() throws ReplicationException;
+
+  /**
+   * Get a list of all hfile references in the given peer.
+   * @param peerId a String that identifies the peer
+   * @return a list of hfile references
+   */
+  List<String> getReplicableHFiles(String peerId) throws ReplicationException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/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 05bbc84..f49537c 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
@@ -63,7 +63,6 @@ public abstract class ReplicationStateZKBase {
   protected final Configuration conf;
   protected final Abortable abortable;
 
-  // Public for testing
   public static final byte[] ENABLED_ZNODE_BYTES =
       toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
   public static final byte[] DISABLED_ZNODE_BYTES =

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
new file mode 100644
index 0000000..60d0749
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.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.replication;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Used to create replication storage(peer, queue) classes.
+ * <p>
+ * For now we only have zk based implementation.
+ */
+@InterfaceAudience.Private
+public class ReplicationStorageFactory {
+
+  private ReplicationStorageFactory() {
+  }
+
+  /**
+   * Create a new {@link ReplicationPeerStorage}.
+   */
+  public static ReplicationPeerStorage getReplicationPeerStorage(ZKWatcher zk, Configuration conf) {
+    return new ZKReplicationPeerStorage(zk, conf);
+  }
+
+  /**
+   * Create a new {@link ReplicationQueueStorage}.
+   */
+  public static ReplicationQueueStorage getReplicationQueueStorage(ZKWatcher zk,
+      Configuration conf) {
+    return new ZKReplicationQueueStorage(zk, conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
new file mode 100644
index 0000000..49af4c3
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.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.replication;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+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.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+/**
+ * ZK based replication peer storage.
+ */
+@InterfaceAudience.Private
+class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements ReplicationPeerStorage {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationPeerStorage.class);
+
+  public static final byte[] ENABLED_ZNODE_BYTES =
+    toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
+  public static final byte[] DISABLED_ZNODE_BYTES =
+    toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
+
+  /**
+   * The name of the znode that contains the replication status of a remote slave (i.e. peer)
+   * cluster.
+   */
+  private final String peerStateNodeName;
+
+  /**
+   * The name of the znode that contains a list of all remote slave (i.e. peer) clusters.
+   */
+  private final String peersZNode;
+
+  public ZKReplicationPeerStorage(ZKWatcher zookeeper, Configuration conf) {
+    super(zookeeper, conf);
+    this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
+    String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
+    this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);
+  }
+
+  private String getPeerStateNode(String peerId) {
+    return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName);
+  }
+
+  private String getPeerNode(String peerId) {
+    return ZNodePaths.joinZNode(peersZNode, peerId);
+  }
+
+  @Override
+  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException {
+    try {
+      ZKUtil.createWithParents(zookeeper, peersZNode);
+      ZKUtil.multiOrSequential(zookeeper,
+        Arrays.asList(
+          ZKUtilOp.createAndFailSilent(getPeerNode(peerId),
+            ReplicationPeerConfigUtil.toByteArray(peerConfig)),
+          ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
+            enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES)),
+        false);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>" +
+        peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e);
+    }
+  }
+
+  @Override
+  public void removePeer(String peerId) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeRecursively(zookeeper, getPeerNode(peerId));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Could not remove peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public void setPeerState(String peerId, boolean enabled) throws ReplicationException {
+    byte[] stateBytes = enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES;
+    try {
+      ZKUtil.setData(zookeeper, getPeerStateNode(peerId), stateBytes);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Unable to change state of the peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException {
+    try {
+      ZKUtil.setData(this.zookeeper, getPeerNode(peerId),
+        ReplicationPeerConfigUtil.toByteArray(peerConfig));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "There was a problem trying to save changes to the " + "replication peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public List<String> listPeerIds() throws ReplicationException {
+    try {
+      return CollectionUtils.nullToEmpty(ZKUtil.listChildrenAndWatchThem(zookeeper, peersZNode));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Cannot get the list of peers", e);
+    }
+  }
+
+  @Override
+  public boolean isPeerEnabled(String peerId) throws ReplicationException {
+    try {
+      return Arrays.equals(ENABLED_ZNODE_BYTES,
+        ZKUtil.getData(zookeeper, getPeerStateNode(peerId)));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Unable to get status of the peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public Optional<ReplicationPeerConfig> getPeerConfig(String peerId) throws ReplicationException {
+    byte[] data;
+    try {
+      data = ZKUtil.getData(zookeeper, getPeerNode(peerId));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Error getting configuration for peer with id=" + peerId, e);
+    }
+    if (data == null || data.length == 0) {
+      return Optional.empty();
+    }
+    try {
+      return Optional.of(ReplicationPeerConfigUtil.parsePeerFrom(data));
+    } catch (DeserializationException e) {
+      LOG.warn("Failed to parse replication peer config for peer with id=" + peerId, e);
+      return Optional.empty();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
new file mode 100644
index 0000000..7015d7f
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -0,0 +1,425 @@
+/**
+ * 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.replication;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.hadoop.hbase.util.CollectionUtils.nullToEmpty;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+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.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.BadVersionException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
+import org.apache.zookeeper.KeeperException.NotEmptyException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+
+/**
+ * ZK based replication queue storage.
+ */
+@InterfaceAudience.Private
+class ZKReplicationQueueStorage extends ZKReplicationStorageBase
+    implements ReplicationQueueStorage {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationQueueStorage.class);
+
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
+    "zookeeper.znode.replication.hfile.refs";
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
+
+  /**
+   * The name of the znode that contains all replication queues
+   */
+  private final String queuesZNode;
+
+  /**
+   * The name of the znode that contains queues of hfile references to be replicated
+   */
+  private final String hfileRefsZNode;
+
+  public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) {
+    super(zookeeper, conf);
+
+    String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
+    String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
+      ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
+    this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName);
+    this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName);
+  }
+
+  private String getRsNode(ServerName serverName) {
+    return ZNodePaths.joinZNode(queuesZNode, serverName.getServerName());
+  }
+
+  private String getQueueNode(ServerName serverName, String queueId) {
+    return ZNodePaths.joinZNode(getRsNode(serverName), queueId);
+  }
+
+  private String getFileNode(String queueNode, String fileName) {
+    return ZNodePaths.joinZNode(queueNode, fileName);
+  }
+
+  private String getFileNode(ServerName serverName, String queueId, String fileName) {
+    return getFileNode(getQueueNode(serverName, queueId), fileName);
+  }
+
+  @Override
+  public void removeQueue(ServerName serverName, String queueId) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeRecursively(zookeeper, getQueueNode(serverName, queueId));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Failed to delete queue (serverName=" + serverName + ", queueId=" + queueId + ")", e);
+    }
+  }
+
+  @Override
+  public void addWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    try {
+      ZKUtil.createWithParents(zookeeper, getFileNode(serverName, queueId, fileName));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add wal to queue (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+  }
+
+  @Override
+  public void removeWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    String fileNode = getFileNode(serverName, queueId, fileName);
+    try {
+      ZKUtil.deleteNode(zookeeper, fileNode);
+    } catch (NoNodeException e) {
+      LOG.warn(fileNode + " has already been deleted when removing log");
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove wal from queue (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+  }
+
+  @Override
+  public void setWALPosition(ServerName serverName, String queueId, String fileName, long position)
+      throws ReplicationException {
+    try {
+      ZKUtil.setData(zookeeper, getFileNode(serverName, queueId, fileName),
+        ZKUtil.positionToByteArray(position));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to set log position (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);
+    }
+  }
+
+  @Override
+  public long getWALPosition(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    byte[] bytes;
+    try {
+      bytes = ZKUtil.getData(zookeeper, getFileNode(serverName, queueId, fileName));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Failed to get log position (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+    try {
+      return ZKUtil.parseWALPositionFrom(bytes);
+    } catch (DeserializationException de) {
+      LOG.warn("Failed to parse log position (serverName=" + serverName + ", queueId=" + queueId +
+        ", fileName=" + fileName + ")");
+    }
+    // if we can not parse the position, start at the beginning of the wal file again
+    return 0;
+  }
+
+  @Override
+  public Pair<String, SortedSet<String>> claimQueue(ServerName sourceServerName, String queueId,
+      ServerName destServerName) throws ReplicationException {
+    LOG.info(
+      "Atomically moving " + sourceServerName + "/" + queueId + "'s WALs to " + destServerName);
+    try {
+      ZKUtil.createWithParents(zookeeper, getRsNode(destServerName));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Claim queue queueId=" + queueId + " from " + sourceServerName + " to " + destServerName +
+            " failed when creating the node for " + destServerName,
+          e);
+    }
+    try {
+      String oldQueueNode = getQueueNode(sourceServerName, queueId);
+      List<String> wals = ZKUtil.listChildrenNoWatch(zookeeper, oldQueueNode);
+      String newQueueId = queueId + "-" + sourceServerName;
+      if (CollectionUtils.isEmpty(wals)) {
+        ZKUtil.deleteNodeFailSilent(zookeeper, oldQueueNode);
+        LOG.info("Removed " + sourceServerName + "/" + queueId + " since it's empty");
+        return new Pair<>(newQueueId, Collections.emptySortedSet());
+      }
+      String newQueueNode = getQueueNode(destServerName, newQueueId);
+      List<ZKUtilOp> listOfOps = new ArrayList<>();
+      SortedSet<String> logQueue = new TreeSet<>();
+      // create the new cluster znode
+      listOfOps.add(ZKUtilOp.createAndFailSilent(newQueueNode, HConstants.EMPTY_BYTE_ARRAY));
+      // get the offset of the logs and set it to new znodes
+      for (String wal : wals) {
+        String oldWalNode = getFileNode(oldQueueNode, wal);
+        byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalNode);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Creating " + wal + " with data " + Bytes.toStringBinary(logOffset));
+        }
+        String newWalNode = getFileNode(newQueueNode, wal);
+        listOfOps.add(ZKUtilOp.createAndFailSilent(newWalNode, logOffset));
+        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalNode));
+        logQueue.add(wal);
+      }
+      // add delete op for peer
+      listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldQueueNode));
+
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("The multi list size is: " + listOfOps.size());
+      }
+      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
+
+      LOG.info(
+        "Atomically moved " + sourceServerName + "/" + queueId + "'s WALs to " + destServerName);
+      return new Pair<>(newQueueId, logQueue);
+    } catch (NoNodeException | NodeExistsException | NotEmptyException | BadVersionException e) {
+      // Multi call failed; it looks like some other regionserver took away the logs.
+      // These exceptions mean that zk tells us the request can not be execute so it is safe to just
+      // return a null. For other types of exception should be thrown out to notify the upper layer.
+      LOG.info(
+        "Claim queue queueId=" + queueId + " from " + sourceServerName + " to " + destServerName +
+          " failed with " + e.toString() + ", maybe someone else has already took away the logs");
+      return null;
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Claim queue queueId=" + queueId + " from " +
+        sourceServerName + " to " + destServerName + " failed", e);
+    }
+  }
+
+  @Override
+  public void removeReplicatorIfQueueIsEmpty(ServerName serverName) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeFailSilent(zookeeper, getRsNode(serverName));
+    } catch (NotEmptyException e) {
+      // keep silence to avoid logging too much.
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove replicator for " + serverName, e);
+    }
+  }
+
+  private List<ServerName> getListOfReplicators0() throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, queuesZNode)).stream()
+        .map(ServerName::parseServerName).collect(toList());
+  }
+
+  @Override
+  public List<ServerName> getListOfReplicators() throws ReplicationException {
+    try {
+      return getListOfReplicators0();
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of replicators", e);
+    }
+  }
+
+  private List<String> getLogsInQueue0(ServerName serverName, String queueId)
+      throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getQueueNode(serverName, queueId)));
+  }
+
+  private List<String> getAllQueues0(ServerName serverName) throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getRsNode(serverName)));
+  }
+
+  @Override
+  public List<String> getAllQueues(ServerName serverName) throws ReplicationException {
+    try {
+      return getAllQueues0(serverName);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get all queues (serverName=" + serverName + ")", e);
+    }
+  }
+
+  private int getQueuesZNodeCversion() throws KeeperException {
+    Stat stat = new Stat();
+    ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
+    return stat.getCversion();
+  }
+
+  @Override
+  public Set<String> getAllWALs() throws ReplicationException {
+    try {
+      for (int retry = 0;; retry++) {
+        int v0 = getQueuesZNodeCversion();
+        List<ServerName> rss = getListOfReplicators0();
+        if (rss.isEmpty()) {
+          LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
+          return Collections.emptySet();
+        }
+        Set<String> wals = Sets.newHashSet();
+        for (ServerName rs : rss) {
+          for (String queueId : getAllQueues0(rs)) {
+            wals.addAll(getLogsInQueue0(rs, queueId));
+          }
+        }
+        int v1 = getQueuesZNodeCversion();
+        if (v0 == v1) {
+          return wals;
+        }
+        LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
+          v0, v1, retry));
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get all wals", e);
+    }
+  }
+
+  private String getHFileRefsPeerNode(String peerId) {
+    return ZNodePaths.joinZNode(hfileRefsZNode, peerId);
+  }
+
+  private String getHFileNode(String peerNode, String fileName) {
+    return ZNodePaths.joinZNode(peerNode, fileName);
+  }
+
+  @Override
+  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    try {
+      if (ZKUtil.checkExists(zookeeper, peerNode) == -1) {
+        LOG.info("Adding peer " + peerId + " to hfile reference queue.");
+        ZKUtil.createWithParents(zookeeper, peerNode);
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
+          e);
+    }
+  }
+
+  @Override
+  public void removePeerFromHFileRefs(String peerId) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    try {
+      if (ZKUtil.checkExists(zookeeper, peerNode) == -1) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Peer " + peerNode + " not found in hfile reference queue.");
+        }
+      } else {
+        LOG.info("Removing peer " + peerNode + " from hfile reference queue.");
+        ZKUtil.deleteNodeRecursively(zookeeper, peerNode);
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Failed to remove peer " + peerId + " from hfile reference queue.", e);
+    }
+  }
+
+  @Override
+  public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
+      throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    boolean debugEnabled = LOG.isDebugEnabled();
+    if (debugEnabled) {
+      LOG.debug("Adding hfile references " + pairs + " in queue " + peerNode);
+    }
+    List<ZKUtilOp> listOfOps =
+      pairs.stream().map(p -> p.getSecond().getName()).map(n -> getHFileNode(peerNode, n))
+          .map(f -> ZKUtilOp.createAndFailSilent(f, HConstants.EMPTY_BYTE_ARRAY)).collect(toList());
+    if (debugEnabled) {
+      LOG.debug("The multi list size for adding hfile references in zk for node " + peerNode +
+        " is " + listOfOps.size());
+    }
+    try {
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add hfile reference to peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public void removeHFileRefs(String peerId, List<String> files) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    boolean debugEnabled = LOG.isDebugEnabled();
+    if (debugEnabled) {
+      LOG.debug("Removing hfile references " + files + " from queue " + peerNode);
+    }
+
+    List<ZKUtilOp> listOfOps = files.stream().map(n -> getHFileNode(peerNode, n))
+        .map(ZKUtilOp::deleteNodeFailSilent).collect(toList());
+    if (debugEnabled) {
+      LOG.debug("The multi list size for removing hfile references in zk for node " + peerNode +
+        " is " + listOfOps.size());
+    }
+    try {
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove hfile reference from peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public int getHFileRefsNodeChangeVersion() throws ReplicationException {
+    Stat stat = new Stat();
+    try {
+      ZKUtil.getDataNoWatch(zookeeper, hfileRefsZNode, stat);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get stat of replication hfile references node.", e);
+    }
+    return stat.getCversion();
+  }
+
+  @Override
+  public List<String> getAllPeersFromHFileRefsQueue() throws ReplicationException {
+    try {
+      return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of all peers in hfile references node.",
+          e);
+    }
+  }
+
+  @Override
+  public List<String> getReplicableHFiles(String peerId) throws ReplicationException {
+    try {
+      return nullToEmpty(ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId)));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of hfile references for peer " + peerId,
+          e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
new file mode 100644
index 0000000..b8a2044
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
@@ -0,0 +1,75 @@
+/**
+ * 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.replication;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+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.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+/**
+ * This is a base class for maintaining replication related data,for example, peer, queue, etc, in
+ * zookeeper.
+ */
+@InterfaceAudience.Private
+class ZKReplicationStorageBase {
+
+  /** The name of the base znode that contains all replication state. */
+  protected final String replicationZNode;
+
+  protected final ZKWatcher zookeeper;
+  protected final Configuration conf;
+
+  protected ZKReplicationStorageBase(ZKWatcher zookeeper, Configuration conf) {
+    this.zookeeper = zookeeper;
+    this.conf = conf;
+    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
+
+    this.replicationZNode =
+      ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode, replicationZNodeName);
+
+  }
+
+  /**
+   * Serialized protobuf of <code>state</code> with pb magic prefix prepended suitable for use as
+   * content of a peer-state znode under a peer cluster id as in
+   * /hbase/replication/peers/PEER_ID/peer-state.
+   */
+  protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) {
+    ReplicationProtos.ReplicationState msg =
+      ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
+    // There is no toByteArray on this pb Message?
+    // 32 bytes is default which seems fair enough here.
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+      CodedOutputStream cos = CodedOutputStream.newInstance(baos, 16);
+      msg.writeTo(cos);
+      cos.flush();
+      baos.flush();
+      return ProtobufUtil.prependPBMagic(baos.toByteArray());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
new file mode 100644
index 0000000..a3be1e6
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
@@ -0,0 +1,171 @@
+/**
+ * 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.replication;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestZKReplicationPeerStorage {
+
+  private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
+
+  private static ZKReplicationPeerStorage STORAGE;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniZKCluster();
+    STORAGE = new ZKReplicationPeerStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    UTIL.shutdownMiniZKCluster();
+  }
+
+  private Set<String> randNamespaces(Random rand) {
+    return Stream.generate(() -> Long.toHexString(rand.nextLong())).limit(rand.nextInt(5))
+        .collect(toSet());
+  }
+
+  private Map<TableName, List<String>> randTableCFs(Random rand) {
+    int size = rand.nextInt(5);
+    Map<TableName, List<String>> map = new HashMap<>();
+    for (int i = 0; i < size; i++) {
+      TableName tn = TableName.valueOf(Long.toHexString(rand.nextLong()));
+      List<String> cfs = Stream.generate(() -> Long.toHexString(rand.nextLong()))
+          .limit(rand.nextInt(5)).collect(toList());
+      map.put(tn, cfs);
+    }
+    return map;
+  }
+
+  private ReplicationPeerConfig getConfig(int seed) {
+    Random rand = new Random(seed);
+    ReplicationPeerConfig config = new ReplicationPeerConfig();
+    config.setClusterKey(Long.toHexString(rand.nextLong()));
+    config.setReplicationEndpointImpl(Long.toHexString(rand.nextLong()));
+    config.setNamespaces(randNamespaces(rand));
+    config.setExcludeNamespaces(randNamespaces(rand));
+    config.setTableCFsMap(randTableCFs(rand));
+    config.setReplicateAllUserTables(rand.nextBoolean());
+    config.setBandwidth(rand.nextInt(1000));
+    return config;
+  }
+
+  private void assertSetEquals(Set<String> expected, Set<String> actual) {
+    if (expected == null || expected.size() == 0) {
+      assertTrue(actual == null || actual.size() == 0);
+      return;
+    }
+    assertEquals(expected.size(), actual.size());
+    expected.forEach(s -> assertTrue(actual.contains(s)));
+  }
+
+  private void assertMapEquals(Map<TableName, List<String>> expected,
+      Map<TableName, List<String>> actual) {
+    if (expected == null || expected.size() == 0) {
+      assertTrue(actual == null || actual.size() == 0);
+      return;
+    }
+    assertEquals(expected.size(), actual.size());
+    expected.forEach((expectedTn, expectedCFs) -> {
+      List<String> actualCFs = actual.get(expectedTn);
+      if (expectedCFs == null || expectedCFs.size() == 0) {
+        assertTrue(actual.containsKey(expectedTn));
+        assertTrue(actualCFs == null || actualCFs.size() == 0);
+      } else {
+        assertNotNull(actualCFs);
+        assertEquals(expectedCFs.size(), actualCFs.size());
+        for (Iterator<String> expectedIt = expectedCFs.iterator(), actualIt = actualCFs.iterator();
+          expectedIt.hasNext();) {
+          assertEquals(expectedIt.next(), actualIt.next());
+        }
+      }
+    });
+  }
+
+  private void assertConfigEquals(ReplicationPeerConfig expected, ReplicationPeerConfig actual) {
+    assertEquals(expected.getClusterKey(), actual.getClusterKey());
+    assertEquals(expected.getReplicationEndpointImpl(), actual.getReplicationEndpointImpl());
+    assertSetEquals(expected.getNamespaces(), actual.getNamespaces());
+    assertSetEquals(expected.getExcludeNamespaces(), actual.getExcludeNamespaces());
+    assertMapEquals(expected.getTableCFsMap(), actual.getTableCFsMap());
+    assertMapEquals(expected.getExcludeTableCFsMap(), actual.getExcludeTableCFsMap());
+    assertEquals(expected.replicateAllUserTables(), actual.replicateAllUserTables());
+    assertEquals(expected.getBandwidth(), actual.getBandwidth());
+  }
+
+  @Test
+  public void test() throws ReplicationException {
+    int peerCount = 10;
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.addPeer(Integer.toString(i), getConfig(i), i % 2 == 0);
+    }
+    List<String> peerIds = STORAGE.listPeerIds();
+    assertEquals(peerCount, peerIds.size());
+    for (String peerId : peerIds) {
+      int seed = Integer.parseInt(peerId);
+      assertConfigEquals(getConfig(seed), STORAGE.getPeerConfig(peerId).get());
+    }
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.updatePeerConfig(Integer.toString(i), getConfig(i + 1));
+    }
+    for (String peerId : peerIds) {
+      int seed = Integer.parseInt(peerId);
+      assertConfigEquals(getConfig(seed + 1), STORAGE.getPeerConfig(peerId).get());
+    }
+    for (int i = 0; i < peerCount; i++) {
+      assertEquals(i % 2 == 0, STORAGE.isPeerEnabled(Integer.toString(i)));
+    }
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.setPeerState(Integer.toString(i), i % 2 != 0);
+    }
+    for (int i = 0; i < peerCount; i++) {
+      assertEquals(i % 2 != 0, STORAGE.isPeerEnabled(Integer.toString(i)));
+    }
+    String toRemove = Integer.toString(peerCount / 2);
+    STORAGE.removePeer(toRemove);
+    peerIds = STORAGE.listPeerIds();
+    assertEquals(peerCount - 1, peerIds.size());
+    assertFalse(peerIds.contains(toRemove));
+    assertFalse(STORAGE.getPeerConfig(toRemove).isPresent());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
new file mode 100644
index 0000000..d5bba0d
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -0,0 +1,171 @@
+/**
+ * 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.replication;
+
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestZKReplicationQueueStorage {
+  private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
+
+  private static ZKReplicationQueueStorage STORAGE;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniZKCluster();
+    STORAGE = new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    UTIL.shutdownMiniZKCluster();
+  }
+
+  @After
+  public void tearDownAfterTest() throws ReplicationException {
+    for (ServerName serverName : STORAGE.getListOfReplicators()) {
+      for (String queue : STORAGE.getAllQueues(serverName)) {
+        STORAGE.removeQueue(serverName, queue);
+      }
+      STORAGE.removeReplicatorIfQueueIsEmpty(serverName);
+    }
+    for (String peerId : STORAGE.getAllPeersFromHFileRefsQueue()) {
+      STORAGE.removePeerFromHFileRefs(peerId);
+    }
+  }
+
+  private ServerName getServerName(int i) {
+    return ServerName.valueOf("127.0.0.1", 8000 + i, 10000 + i);
+  }
+
+  @Test
+  public void testReplicator() throws ReplicationException {
+    assertTrue(STORAGE.getListOfReplicators().isEmpty());
+    String queueId = "1";
+    for (int i = 0; i < 10; i++) {
+      STORAGE.addWAL(getServerName(i), queueId, "file" + i);
+    }
+    List<ServerName> replicators = STORAGE.getListOfReplicators();
+    assertEquals(10, replicators.size());
+    for (int i = 0; i < 10; i++) {
+      assertThat(replicators, hasItems(getServerName(i)));
+    }
+    for (int i = 0; i < 5; i++) {
+      STORAGE.removeQueue(getServerName(i), queueId);
+    }
+    for (int i = 0; i < 10; i++) {
+      STORAGE.removeReplicatorIfQueueIsEmpty(getServerName(i));
+    }
+    replicators = STORAGE.getListOfReplicators();
+    assertEquals(5, replicators.size());
+    for (int i = 5; i < 10; i++) {
+      assertThat(replicators, hasItems(getServerName(i)));
+    }
+  }
+
+  private String getFileName(String base, int i) {
+    return String.format(base + "-%04d", i);
+  }
+
+  @Test
+  public void testAddRemoveLog() throws ReplicationException {
+    ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
+    assertTrue(STORAGE.getAllQueues(serverName1).isEmpty());
+    String queue1 = "1";
+    String queue2 = "2";
+    for (int i = 0; i < 10; i++) {
+      STORAGE.addWAL(serverName1, queue1, getFileName("file1", i));
+      STORAGE.addWAL(serverName1, queue2, getFileName("file2", i));
+    }
+    List<String> queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(2, queueIds.size());
+    assertThat(queueIds, hasItems("1", "2"));
+
+    for (int i = 0; i < 10; i++) {
+      assertEquals(0, STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
+      assertEquals(0, STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
+      STORAGE.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100);
+      STORAGE.setWALPosition(serverName1, queue2, getFileName("file2", i), (i + 1) * 100 + 10);
+    }
+
+    for (int i = 0; i < 10; i++) {
+      assertEquals((i + 1) * 100,
+        STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
+      assertEquals((i + 1) * 100 + 10,
+        STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
+    }
+
+    for (int i = 0; i < 10; i++) {
+      if (i % 2 == 0) {
+        STORAGE.removeWAL(serverName1, queue1, getFileName("file1", i));
+      } else {
+        STORAGE.removeWAL(serverName1, queue2, getFileName("file2", i));
+      }
+    }
+
+    queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(2, queueIds.size());
+    assertThat(queueIds, hasItems("1", "2"));
+
+    ServerName serverName2 = ServerName.valueOf("127.0.0.1", 8001, 10001);
+    Pair<String, SortedSet<String>> peer1 = STORAGE.claimQueue(serverName1, "1", serverName2);
+
+    assertEquals("1-" + serverName1.getServerName(), peer1.getFirst());
+    assertEquals(5, peer1.getSecond().size());
+    int i = 1;
+    for (String wal : peer1.getSecond()) {
+      assertEquals(getFileName("file1", i), wal);
+      assertEquals((i + 1) * 100,
+        STORAGE.getWALPosition(serverName2, peer1.getFirst(), getFileName("file1", i)));
+      i += 2;
+    }
+
+    queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(1, queueIds.size());
+    assertThat(queueIds, hasItems("2"));
+
+    queueIds = STORAGE.getAllQueues(serverName2);
+    assertEquals(1, queueIds.size());
+    assertThat(queueIds, hasItems(peer1.getFirst()));
+
+    Set<String> allWals = STORAGE.getAllWALs();
+    assertEquals(10, allWals.size());
+    for (i = 0; i < 10; i++) {
+      assertThat(allWals, hasItems(i % 2 == 0 ? getFileName("file2", i) : getFileName("file1", i)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/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 ad81cc6..0e8ef3b 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
@@ -38,6 +38,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -133,7 +134,7 @@ import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
 import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
@@ -326,7 +327,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   private AssignmentManager assignmentManager;
 
   // manager of replication
-  private ReplicationManager replicationManager;
+  private ReplicationPeerManager replicationPeerManager;
 
   // buffer for "fatal error" notices from region servers
   // in the cluster. This is only used for assisting
@@ -717,8 +718,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   /**
    * Initialize all ZK based system trackers.
    */
-  void initializeZKBasedSystemTrackers() throws IOException,
-      InterruptedException, KeeperException, CoordinatedStateException {
+  void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, KeeperException,
+      CoordinatedStateException, ReplicationException {
     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
     this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
     this.normalizer.setMasterServices(this);
@@ -736,7 +737,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.assignmentManager = new AssignmentManager(this);
     this.assignmentManager.start();
 
-    this.replicationManager = new ReplicationManager(conf, zooKeeper, this);
+    this.replicationPeerManager = ReplicationPeerManager.create(zooKeeper, conf);
 
     this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
     this.regionServerTracker.start();
@@ -781,9 +782,8 @@ public class HMaster extends HRegionServer implements MasterServices {
    * <li>Handle either fresh cluster start or master failover</li>
    * </ol>
    */
-  private void finishActiveMasterInitialization(MonitoredTask status)
-      throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
-
+  private void finishActiveMasterInitialization(MonitoredTask status) throws IOException,
+      InterruptedException, KeeperException, CoordinatedStateException, ReplicationException {
     Thread zombieDetector = new Thread(new InitializationMonitor(this),
         "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
     zombieDetector.setDaemon(true);
@@ -3369,18 +3369,19 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws ReplicationException,
-      IOException {
+  public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
+      throws ReplicationException, IOException {
     if (cpHost != null) {
       cpHost.preGetReplicationPeerConfig(peerId);
     }
-    final ReplicationPeerConfig peerConfig = this.replicationManager.getPeerConfig(peerId);
-    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId + ", config="
-        + peerConfig);
+    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId);
+    Optional<ReplicationPeerConfig> peerConfig =
+      this.replicationPeerManager.getPeerConfig(peerId);
+
     if (cpHost != null) {
       cpHost.postGetReplicationPeerConfig(peerId);
     }
-    return peerConfig;
+    return peerConfig.orElse(null);
   }
 
   @Override
@@ -3399,7 +3400,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
     LOG.info(getClientIdAuditPrefix() + " list replication peers, regex=" + regex);
     Pattern pattern = regex == null ? null : Pattern.compile(regex);
-    List<ReplicationPeerDescription> peers = this.replicationManager.listReplicationPeers(pattern);
+    List<ReplicationPeerDescription> peers =
+      this.replicationPeerManager.listPeers(pattern);
     if (cpHost != null) {
       cpHost.postListReplicationPeers(regex);
     }
@@ -3549,7 +3551,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ReplicationManager getReplicationManager() {
-    return replicationManager;
+  public ReplicationPeerManager getReplicationPeerManager() {
+    return replicationPeerManager;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index e798455..9d371bd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -459,9 +459,9 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
-   * Returns the {@link ReplicationManager}.
+   * Returns the {@link ReplicationPeerManager}.
    */
-  ReplicationManager getReplicationManager();
+  ReplicationPeerManager getReplicationPeerManager();
 
   /**
    * Update the peerConfig for the specified peer

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index fa4d371..ae038a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -24,24 +24,24 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -138,8 +138,8 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return remoteDispatcher;
   }
 
-  public ReplicationManager getReplicationManager() {
-    return master.getReplicationManager();
+  public ReplicationPeerManager getReplicationPeerManager() {
+    return master.getReplicationPeerManager();
   }
 
   public boolean isRunning() {
@@ -151,22 +151,22 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return master.isInitialized();
   }
 
-  public boolean waitInitialized(Procedure proc) {
+  public boolean waitInitialized(Procedure<?> proc) {
     return master.getInitializedEvent().suspendIfNotReady(proc);
   }
 
-  public boolean waitServerCrashProcessingEnabled(Procedure proc) {
+  public boolean waitServerCrashProcessingEnabled(Procedure<?> proc) {
     if (master instanceof HMaster) {
       return ((HMaster)master).getServerCrashProcessingEnabledEvent().suspendIfNotReady(proc);
     }
     return false;
   }
 
-  public boolean waitFailoverCleanup(Procedure proc) {
+  public boolean waitFailoverCleanup(Procedure<?> proc) {
     return master.getAssignmentManager().getFailoverCleanupEvent().suspendIfNotReady(proc);
   }
 
-  public void setEventReady(ProcedureEvent event, boolean isReady) {
+  public void setEventReady(ProcedureEvent<?> event, boolean isReady) {
     if (isReady) {
       event.wake(procSched);
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index 066c3e7..a4f9b32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -58,16 +58,18 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
-  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+  protected void prePeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.preAddReplicationPeer(peerId, peerConfig);
     }
+    env.getReplicationPeerManager().preAddPeer(peerId, peerConfig);
   }
 
   @Override
   protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
-    env.getReplicationManager().addReplicationPeer(peerId, peerConfig, enabled);
+    env.getReplicationPeerManager().addPeer(peerId, peerConfig, enabled);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
index 9a28de6..10e35a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,12 +52,12 @@ public class DisablePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preDisableReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preDisablePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception {
-    env.getReplicationManager().disableReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().disablePeer(peerId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
index 4855901..f2a9f01 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,11 +52,12 @@ public class EnablePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preEnableReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preEnablePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
-    env.getReplicationManager().enableReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().enablePeer(peerId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index c4552ed..279fbc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@@ -27,6 +26,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -67,17 +67,16 @@ public abstract class ModifyPeerProcedure
   }
 
   /**
-   * Called before we start the actual processing. If an exception is thrown then we will give up
-   * and mark the procedure as failed directly.
+   * Called before we start the actual processing. The implementation should call the pre CP hook,
+   * and also the pre-check for the peer modification.
+   * <p>
+   * If an IOException is thrown then we will give up and mark the procedure as failed directly. If
+   * all checks passes then the procedure can not be rolled back any more.
    */
-  protected abstract void prePeerModification(MasterProcedureEnv env) throws IOException;
+  protected abstract void prePeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException;
 
-  /**
-   * We will give up and mark the procedure as failure if {@link IllegalArgumentException} is
-   * thrown, for other type of Exception we will retry.
-   */
-  protected abstract void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception;
+  protected abstract void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException;
 
   /**
    * Called before we finish the procedure. The implementation can do some logging work, and also
@@ -100,23 +99,24 @@ public abstract class ModifyPeerProcedure
         try {
           prePeerModification(env);
         } catch (IOException e) {
-          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
-            ", mark the procedure as failure and give up", e);
-          setFailure("prePeerModification", e);
+          LOG.warn(
+            getClass().getName() + " failed to call CP hook or the pre check is failed for peer " +
+              peerId + ", mark the procedure as failure and give up",
+            e);
+          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer", e);
           releaseLatch();
           return Flow.NO_MORE_STATE;
+        } catch (ReplicationException e) {
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", retry", e);
+          throw new ProcedureYieldException();
         }
         setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
         return Flow.HAS_MORE_STATE;
       case UPDATE_PEER_STORAGE:
         try {
           updatePeerStorage(env);
-        } catch (IllegalArgumentException e) {
-          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer",
-            new DoNotRetryIOException(e));
-          releaseLatch();
-          return Flow.NO_MORE_STATE;
-        } catch (Exception e) {
+        } catch (ReplicationException e) {
           LOG.warn(
             getClass().getName() + " update peer storage for peer " + peerId + " failed, retry", e);
           throw new ProcedureYieldException();
@@ -158,8 +158,7 @@ public abstract class ModifyPeerProcedure
   @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
       throws IOException, InterruptedException {
-    if (state == PeerModificationState.PRE_PEER_MODIFICATION ||
-      state == PeerModificationState.UPDATE_PEER_STORAGE) {
+    if (state == PeerModificationState.PRE_PEER_MODIFICATION) {
       // actually the peer related operations has no rollback, but if we haven't done any
       // modifications on the peer storage, we can just return.
       return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index d40df02..6e9c384 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,11 +52,12 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preRemoveReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preRemovePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
-    env.getReplicationManager().removeReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().removePeer(peerId);
   }
 
   @Override


[13/48] hbase git commit: HBASE-19216 Implement a general framework to execute remote procedure on RS

Posted by zh...@apache.org.
HBASE-19216 Implement a general framework to execute remote procedure on RS


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 12fd193a57b4a88e8ea6a241666a15dd23b1dacd
Parents: e014e5f
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 15 21:06:44 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:25:49 2018 +0800

----------------------------------------------------------------------
 .../hbase/procedure2/LockedResourceType.java    |   4 +-
 .../procedure2/RemoteProcedureDispatcher.java   |  23 +-
 .../src/main/protobuf/Admin.proto               |   9 +-
 .../src/main/protobuf/MasterProcedure.proto     |  30 +++
 .../src/main/protobuf/RegionServerStatus.proto  |  15 ++
 .../apache/hadoop/hbase/executor/EventType.java |  26 ++-
 .../hadoop/hbase/executor/ExecutorType.java     |   3 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  33 ++-
 .../hadoop/hbase/master/MasterRpcServices.java  |  13 ++
 .../assignment/RegionTransitionProcedure.java   |  18 +-
 .../procedure/MasterProcedureScheduler.java     | 224 +++++++++++++------
 .../procedure/PeerProcedureInterface.java       |  34 +++
 .../master/procedure/RSProcedureDispatcher.java | 101 +++++----
 .../master/replication/ModifyPeerProcedure.java | 127 +++++++++++
 .../master/replication/RefreshPeerCallable.java |  67 ++++++
 .../replication/RefreshPeerProcedure.java       | 197 ++++++++++++++++
 .../hbase/procedure2/RSProcedureCallable.java   |  43 ++++
 .../hbase/regionserver/HRegionServer.java       |  75 ++++++-
 .../hbase/regionserver/RSRpcServices.java       |  56 +++--
 .../handler/RSProcedureHandler.java             |  51 +++++
 .../assignment/TestAssignmentManager.java       |  20 +-
 .../replication/DummyModifyPeerProcedure.java   |  41 ++++
 .../TestDummyModifyPeerProcedure.java           |  80 +++++++
 .../security/access/TestAccessController.java   |   1 +
 24 files changed, 1114 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
index c5fe62b..dc9b5d4 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -22,5 +22,5 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 public enum LockedResourceType {
-  SERVER, NAMESPACE, TABLE, REGION
+  SERVER, NAMESPACE, TABLE, REGION, PEER
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
index 71932b8..78c49fb 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -226,13 +226,30 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
 
   /**
    * Remote procedure reference.
-   * @param <TEnv>
-   * @param <TRemote>
    */
   public interface RemoteProcedure<TEnv, TRemote> {
+    /**
+     * For building the remote operation.
+     */
     RemoteOperation remoteCallBuild(TEnv env, TRemote remote);
-    void remoteCallCompleted(TEnv env, TRemote remote, RemoteOperation response);
+
+    /**
+     * Called when the executeProcedure call is failed.
+     */
     void remoteCallFailed(TEnv env, TRemote remote, IOException exception);
+
+    /**
+     * Called when RS tells the remote procedure is succeeded through the
+     * {@code reportProcedureDone} method.
+     */
+    void remoteOperationCompleted(TEnv env);
+
+    /**
+     * Called when RS tells the remote procedure is failed through the {@code reportProcedureDone}
+     * method.
+     * @param error the error message
+     */
+    void remoteOperationFailed(TEnv env, String error);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 118c79b..ddcc266 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -256,14 +256,19 @@ message ClearRegionBlockCacheResponse {
   required CacheEvictionStats stats = 1;
 }
 
+message RemoteProcedureRequest {
+  required uint64 proc_id = 1;
+  required string proc_class = 2;
+  optional bytes proc_data = 3;
+}
+
 message ExecuteProceduresRequest {
   repeated OpenRegionRequest open_region = 1;
   repeated CloseRegionRequest close_region = 2;
+  repeated RemoteProcedureRequest proc = 3;
 }
 
 message ExecuteProceduresResponse {
-  repeated OpenRegionResponse open_region = 1;
-  repeated CloseRegionResponse close_region = 2;
 }
 
 service AdminService {

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index f9b8807..0e2bdba 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -365,3 +365,33 @@ message GCMergedRegionsStateData {
   required RegionInfo parent_b = 2;
   required RegionInfo merged_child = 3;
 }
+
+enum PeerModificationState {
+  UPDATE_PEER_STORAGE = 1;
+  REFRESH_PEER_ON_RS = 2;
+  POST_PEER_MODIFICATION = 3;
+}
+
+message PeerModificationStateData {
+  required string peer_id = 1;
+}
+
+enum PeerModificationType {
+  ADD_PEER = 1;
+  REMOVE_PEER = 2;
+  ENABLE_PEER = 3;
+  DISABLE_PEER = 4;
+  UPDATE_PEER_CONFIG = 5;
+}
+
+message RefreshPeerStateData {
+  required string peer_id = 1;
+  required PeerModificationType type = 2;
+  required ServerName target_server = 3;
+}
+
+message RefreshPeerParameter {
+  required string peer_id = 1;
+  required PeerModificationType type = 2;
+  required ServerName target_server = 3;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index f83bb20..eb396ac 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -143,7 +143,19 @@ message RegionSpaceUseReportRequest {
 }
 
 message RegionSpaceUseReportResponse {
+}
 
+message ReportProcedureDoneRequest {
+  required uint64 proc_id = 1;
+  enum Status {
+    SUCCESS = 1;
+    ERROR = 2;
+  }
+  required Status status = 2;
+  optional string error = 3;
+}
+
+message ReportProcedureDoneResponse {
 }
 
 service RegionServerStatusService {
@@ -181,4 +193,7 @@ service RegionServerStatusService {
    */
   rpc ReportRegionSpaceUse(RegionSpaceUseReportRequest)
     returns(RegionSpaceUseReportResponse);
+
+  rpc ReportProcedureDone(ReportProcedureDoneRequest)
+    returns(ReportProcedureDoneResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
index 26fb63a..922deb8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
@@ -20,15 +20,14 @@ package org.apache.hadoop.hbase.executor;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * List of all HBase event handler types.  Event types are named by a
- * convention: event type names specify the component from which the event
- * originated and then where its destined -- e.g. RS2ZK_ prefix means the
- * event came from a regionserver destined for zookeeper -- and then what
- * the even is; e.g. REGION_OPENING.
- *
- * <p>We give the enums indices so we can add types later and keep them
- * grouped together rather than have to add them always to the end as we
- * would have to if we used raw enum ordinals.
+ * List of all HBase event handler types.
+ * <p>
+ * Event types are named by a convention: event type names specify the component from which the
+ * event originated and then where its destined -- e.g. RS_ZK_ prefix means the event came from a
+ * regionserver destined for zookeeper -- and then what the even is; e.g. REGION_OPENING.
+ * <p>
+ * We give the enums indices so we can add types later and keep them grouped together rather than
+ * have to add them always to the end as we would have to if we used raw enum ordinals.
  */
 @InterfaceAudience.Private
 public enum EventType {
@@ -275,7 +274,14 @@ public enum EventType {
    *
    * RS_COMPACTED_FILES_DISCHARGER
    */
-  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER);
+  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER),
+
+  /**
+   * RS refresh peer.<br>
+   *
+   * RS_REFRESH_PEER
+   */
+  RS_REFRESH_PEER (84, ExecutorType.RS_REFRESH_PEER);
 
   private final int code;
   private final ExecutorType executor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
index c75a0a9..7f130d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
@@ -46,7 +46,8 @@ public enum ExecutorType {
   RS_LOG_REPLAY_OPS          (27),
   RS_REGION_REPLICA_FLUSH_OPS  (28),
   RS_COMPACTED_FILES_DISCHARGER (29),
-  RS_OPEN_PRIORITY_REGION    (30);
+  RS_OPEN_PRIORITY_REGION    (30),
+  RS_REFRESH_PEER               (31);
 
   ExecutorType(int value) {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/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 2683a6a..9487946 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
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -141,6 +140,7 @@ import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
@@ -329,8 +329,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   // flag set after we become the active master (used for testing)
   private volatile boolean activeMaster = false;
 
-  // flag set after we complete initialization once active,
-  // it is not private since it's used in unit tests
+  // flag set after we complete initialization once active
   private final ProcedureEvent initialized = new ProcedureEvent("master initialized");
 
   // flag set after master services are started,
@@ -3545,4 +3544,28 @@ public class HMaster extends HRegionServer implements MasterServices {
   public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
     return this.spaceQuotaSnapshotNotifier;
   }
-}
+
+  @SuppressWarnings("unchecked")
+  private RemoteProcedure<MasterProcedureEnv, ?> getRemoteProcedure(long procId) {
+    Procedure<?> procedure = procedureExecutor.getProcedure(procId);
+    if (procedure == null) {
+      return null;
+    }
+    assert procedure instanceof RemoteProcedure;
+    return (RemoteProcedure<MasterProcedureEnv, ?>) procedure;
+  }
+
+  public void remoteProcedureCompleted(long procId) {
+    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
+    if (procedure != null) {
+      procedure.remoteOperationCompleted(procedureExecutor.getEnvironment());
+    }
+  }
+
+  public void remoteProcedureFailed(long procId, String error) {
+    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
+    if (procedure != null) {
+      procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 907ca9b..f875e20 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -264,6 +264,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
@@ -2247,4 +2249,15 @@ public class MasterRpcServices extends RSRpcServices
     }
     return response.build();
   }
+
+  @Override
+  public ReportProcedureDoneResponse reportProcedureDone(RpcController controller,
+      ReportProcedureDoneRequest request) throws ServiceException {
+    if (request.getStatus() == ReportProcedureDoneRequest.Status.SUCCESS) {
+      master.remoteProcedureCompleted(request.getProcId());
+    } else {
+      master.remoteProcedureFailed(request.getProcId(), request.getError());
+    }
+    return ReportProcedureDoneResponse.getDefaultInstance();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 4a88e3b..04dccc4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -173,12 +173,6 @@ public abstract class RegionTransitionProcedure
       RegionStateNode regionNode, IOException exception);
 
   @Override
-  public void remoteCallCompleted(final MasterProcedureEnv env,
-      final ServerName serverName, final RemoteOperation response) {
-    // Ignore the response? reportTransition() is the one that count?
-  }
-
-  @Override
   public void remoteCallFailed(final MasterProcedureEnv env,
       final ServerName serverName, final IOException exception) {
     final RegionStateNode regionNode = getRegionState(env);
@@ -413,4 +407,16 @@ public abstract class RegionTransitionProcedure
    * @return ServerName the Assign or Unassign is going against.
    */
   public abstract ServerName getServer(final MasterProcedureEnv env);
+
+  @Override
+  public void remoteOperationCompleted(MasterProcedureEnv env) {
+    // should not be called for region operation until we modified the open/close region procedure
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void remoteOperationFailed(MasterProcedureEnv env, String error) {
+    // should not be called for region operation until we modified the open/close region procedure
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index dc9c69d..8ff2d12 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -24,7 +24,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+import java.util.function.Function;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface.PeerOperationType;
 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
 import org.apache.hadoop.hbase.procedure2.AbstractProcedureScheduler;
 import org.apache.hadoop.hbase.procedure2.LockAndQueue;
@@ -109,12 +110,17 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       new ServerQueueKeyComparator();
   private final static TableQueueKeyComparator TABLE_QUEUE_KEY_COMPARATOR =
       new TableQueueKeyComparator();
+  private final static PeerQueueKeyComparator PEER_QUEUE_KEY_COMPARATOR =
+      new PeerQueueKeyComparator();
 
   private final FairQueue<ServerName> serverRunQueue = new FairQueue<>();
   private final FairQueue<TableName> tableRunQueue = new FairQueue<>();
+  private final FairQueue<String> peerRunQueue = new FairQueue<>();
 
   private final ServerQueue[] serverBuckets = new ServerQueue[128];
   private TableQueue tableMap = null;
+  private PeerQueue peerMap = null;
+
   private final SchemaLocking locking = new SchemaLocking();
 
   /**
@@ -161,6 +167,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       doAdd(tableRunQueue, getTableQueue(getTableName(proc)), proc, addFront);
     } else if (isServerProcedure(proc)) {
       doAdd(serverRunQueue, getServerQueue(getServerName(proc)), proc, addFront);
+    } else if (isPeerProcedure(proc)) {
+      doAdd(peerRunQueue, getPeerQueue(getPeerId(proc)), proc, addFront);
     } else {
       // TODO: at the moment we only have Table and Server procedures
       // if you are implementing a non-table/non-server procedure, you have two options: create
@@ -172,7 +180,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   private <T extends Comparable<T>> void doAdd(final FairQueue<T> fairq,
-      final Queue<T> queue, final Procedure proc, final boolean addFront) {
+      final Queue<T> queue, final Procedure<?> proc, final boolean addFront) {
     queue.add(proc, addFront);
     if (!queue.getLockStatus().hasExclusiveLock() || queue.getLockStatus().isLockOwner(proc.getProcId())) {
       // if the queue was not remove for an xlock execution
@@ -189,7 +197,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
 
   @Override
   protected boolean queueHasRunnables() {
-    return tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables();
+    return tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables() ||
+        peerRunQueue.hasRunnables();
   }
 
   @Override
@@ -197,7 +206,10 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     // For now, let server handling have precedence over table handling; presumption is that it
     // is more important handling crashed servers than it is running the
     // enabling/disabling tables, etc.
-    Procedure pollResult = doPoll(serverRunQueue);
+    Procedure<?> pollResult = doPoll(serverRunQueue);
+    if (pollResult == null) {
+      pollResult = doPoll(peerRunQueue);
+    }
     if (pollResult == null) {
       pollResult = doPoll(tableRunQueue);
     }
@@ -267,60 +279,30 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         exclusiveLockOwnerProcedure, sharedLockCount, waitingProcedures);
   }
 
+  private <T> void addToLockedResources(List<LockedResource> lockedResources,
+      Map<T, LockAndQueue> locks, Function<T, String> keyTransformer,
+      LockedResourceType resourcesType) {
+    locks.entrySet().stream().filter(e -> e.getValue().isLocked())
+        .map(
+          e -> createLockedResource(resourcesType, keyTransformer.apply(e.getKey()), e.getValue()))
+        .forEachOrdered(lockedResources::add);
+  }
+
   @Override
   public List<LockedResource> getLocks() {
     schedLock();
-
     try {
       List<LockedResource> lockedResources = new ArrayList<>();
-
-      for (Entry<ServerName, LockAndQueue> entry : locking.serverLocks
-          .entrySet()) {
-        String serverName = entry.getKey().getServerName();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.SERVER, serverName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<String, LockAndQueue> entry : locking.namespaceLocks
-          .entrySet()) {
-        String namespaceName = entry.getKey();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.NAMESPACE, namespaceName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<TableName, LockAndQueue> entry : locking.tableLocks
-          .entrySet()) {
-        String tableName = entry.getKey().getNameAsString();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.TABLE, tableName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<String, LockAndQueue> entry : locking.regionLocks.entrySet()) {
-        String regionName = entry.getKey();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.REGION, regionName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
+      addToLockedResources(lockedResources, locking.serverLocks, sn -> sn.getServerName(),
+        LockedResourceType.SERVER);
+      addToLockedResources(lockedResources, locking.namespaceLocks, Function.identity(),
+        LockedResourceType.NAMESPACE);
+      addToLockedResources(lockedResources, locking.tableLocks, tn -> tn.getNameAsString(),
+        LockedResourceType.TABLE);
+      addToLockedResources(lockedResources, locking.regionLocks, Function.identity(),
+        LockedResourceType.REGION);
+      addToLockedResources(lockedResources, locking.peerLocks, Function.identity(),
+        LockedResourceType.PEER);
       return lockedResources;
     } finally {
       schedUnlock();
@@ -328,8 +310,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   @Override
-  public LockedResource getLockResource(LockedResourceType resourceType,
-      String resourceName) {
+  public LockedResource getLockResource(LockedResourceType resourceType, String resourceName) {
     LockAndQueue queue = null;
     schedLock();
     try {
@@ -346,8 +327,10 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         case REGION:
           queue = locking.regionLocks.get(resourceName);
           break;
+        case PEER:
+          queue = locking.peerLocks.get(resourceName);
+          break;
       }
-
       return queue != null ? createLockedResource(resourceType, resourceName, queue) : null;
     } finally {
       schedUnlock();
@@ -431,6 +414,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         markTableAsDeleted(iProcTable.getTableName(), proc);
         return;
       }
+    } else if (proc instanceof PeerProcedureInterface) {
+      PeerProcedureInterface iProcPeer = (PeerProcedureInterface) proc;
+      if (iProcPeer.getPeerOperationType() == PeerOperationType.REMOVE) {
+        removePeerQueue(iProcPeer.getPeerId());
+      }
     } else {
       // No cleanup for ServerProcedureInterface types, yet.
       return;
@@ -468,12 +456,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     locking.removeTableLock(tableName);
   }
 
-
-  private static boolean isTableProcedure(Procedure proc) {
+  private static boolean isTableProcedure(Procedure<?> proc) {
     return proc instanceof TableProcedureInterface;
   }
 
-  private static TableName getTableName(Procedure proc) {
+  private static TableName getTableName(Procedure<?> proc) {
     return ((TableProcedureInterface)proc).getTableName();
   }
 
@@ -494,15 +481,42 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     return Math.abs(hashCode) % buckets.length;
   }
 
-  private static boolean isServerProcedure(Procedure proc) {
+  private static boolean isServerProcedure(Procedure<?> proc) {
     return proc instanceof ServerProcedureInterface;
   }
 
-  private static ServerName getServerName(Procedure proc) {
+  private static ServerName getServerName(Procedure<?> proc) {
     return ((ServerProcedureInterface)proc).getServerName();
   }
 
   // ============================================================================
+  //  Peer Queue Lookup Helpers
+  // ============================================================================
+  private PeerQueue getPeerQueue(String peerId) {
+    PeerQueue node = AvlTree.get(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR);
+    if (node != null) {
+      return node;
+    }
+    node = new PeerQueue(peerId, locking.getPeerLock(peerId));
+    peerMap = AvlTree.insert(peerMap, node);
+    return node;
+  }
+
+  private void removePeerQueue(String peerId) {
+    peerMap = AvlTree.remove(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR);
+    locking.removePeerLock(peerId);
+  }
+
+
+  private static boolean isPeerProcedure(Procedure<?> proc) {
+    return proc instanceof PeerProcedureInterface;
+  }
+
+  private static String getPeerId(Procedure<?> proc) {
+    return ((PeerProcedureInterface) proc).getPeerId();
+  }
+
+  // ============================================================================
   //  Table and Server Queue Implementation
   // ============================================================================
   private static class ServerQueueKeyComparator implements AvlKeyComparator<ServerQueue> {
@@ -571,6 +585,26 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     }
   }
 
+  private static class PeerQueueKeyComparator implements AvlKeyComparator<PeerQueue> {
+
+    @Override
+    public int compareKey(PeerQueue node, Object key) {
+      return node.compareKey((String) key);
+    }
+  }
+
+  public static class PeerQueue extends Queue<String> {
+
+    public PeerQueue(String peerId, LockStatus lockStatus) {
+      super(peerId, lockStatus);
+    }
+
+    @Override
+    public boolean requireExclusiveLock(Procedure proc) {
+      return requirePeerExclusiveLock((PeerProcedureInterface) proc);
+    }
+  }
+
   // ============================================================================
   //  Table Locking Helpers
   // ============================================================================
@@ -958,7 +992,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @param serverName Server to lock
    * @return true if the procedure has to wait for the server to be available
    */
-  public boolean waitServerExclusiveLock(final Procedure procedure, final ServerName serverName) {
+  public boolean waitServerExclusiveLock(final Procedure<?> procedure,
+      final ServerName serverName) {
     schedLock();
     try {
       final LockAndQueue lock = locking.getServerLock(serverName);
@@ -980,7 +1015,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @param procedure the procedure releasing the lock
    * @param serverName the server that has the exclusive lock
    */
-  public void wakeServerExclusiveLock(final Procedure procedure, final ServerName serverName) {
+  public void wakeServerExclusiveLock(final Procedure<?> procedure, final ServerName serverName) {
     schedLock();
     try {
       final LockAndQueue lock = locking.getServerLock(serverName);
@@ -994,6 +1029,56 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   // ============================================================================
+  //  Peer Locking Helpers
+  // ============================================================================
+
+  private static boolean requirePeerExclusiveLock(PeerProcedureInterface proc) {
+    return proc.getPeerOperationType() != PeerOperationType.REFRESH;
+  }
+
+  /**
+   * Try to acquire the exclusive lock on the specified peer.
+   * @see #wakePeerExclusiveLock(Procedure, String)
+   * @param procedure the procedure trying to acquire the lock
+   * @param peerId peer to lock
+   * @return true if the procedure has to wait for the per to be available
+   */
+  public boolean waitPeerExclusiveLock(Procedure<?> procedure, String peerId) {
+    schedLock();
+    try {
+      final LockAndQueue lock = locking.getPeerLock(peerId);
+      if (lock.tryExclusiveLock(procedure)) {
+        removeFromRunQueue(peerRunQueue, getPeerQueue(peerId));
+        return false;
+      }
+      waitProcedure(lock, procedure);
+      logLockedResource(LockedResourceType.PEER, peerId);
+      return true;
+    } finally {
+      schedUnlock();
+    }
+  }
+
+  /**
+   * Wake the procedures waiting for the specified peer
+   * @see #waitPeerExclusiveLock(Procedure, String)
+   * @param procedure the procedure releasing the lock
+   * @param peerId the peer that has the exclusive lock
+   */
+  public void wakePeerExclusiveLock(Procedure<?> procedure, String peerId) {
+    schedLock();
+    try {
+      final LockAndQueue lock = locking.getPeerLock(peerId);
+      lock.releaseExclusiveLock(procedure);
+      addToRunQueue(peerRunQueue, getPeerQueue(peerId));
+      int waitingCount = wakeWaitingProcedures(lock);
+      wakePollIfNeeded(waitingCount);
+    } finally {
+      schedUnlock();
+    }
+  }
+
+  // ============================================================================
   //  Generic Helpers
   // ============================================================================
   private static abstract class Queue<TKey extends Comparable<TKey>>
@@ -1098,6 +1183,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     final Map<TableName, LockAndQueue> tableLocks = new HashMap<>();
     // Single map for all regions irrespective of tables. Key is encoded region name.
     final Map<String, LockAndQueue> regionLocks = new HashMap<>();
+    final Map<String, LockAndQueue> peerLocks = new HashMap<>();
 
     private <T> LockAndQueue getLock(Map<T, LockAndQueue> map, T key) {
       LockAndQueue lock = map.get(key);
@@ -1132,6 +1218,14 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       return getLock(serverLocks, serverName);
     }
 
+    LockAndQueue getPeerLock(String peerId) {
+      return getLock(peerLocks, peerId);
+    }
+
+    LockAndQueue removePeerLock(String peerId) {
+      return peerLocks.remove(peerId);
+    }
+
     /**
      * Removes all locks by clearing the maps.
      * Used when procedure executor is stopped for failure and recovery testing.
@@ -1142,6 +1236,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       namespaceLocks.clear();
       tableLocks.clear();
       regionLocks.clear();
+      peerLocks.clear();
     }
 
     @Override
@@ -1149,7 +1244,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       return "serverLocks=" + filterUnlocked(this.serverLocks) +
         ", namespaceLocks=" + filterUnlocked(this.namespaceLocks) +
         ", tableLocks=" + filterUnlocked(this.tableLocks) +
-        ", regionLocks=" + filterUnlocked(this.regionLocks);
+        ", regionLocks=" + filterUnlocked(this.regionLocks) +
+        ", peerLocks=" + filterUnlocked(this.peerLocks);
     }
 
     private String filterUnlocked(Map<?, LockAndQueue> locks) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java
new file mode 100644
index 0000000..4abc9ad
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.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.master.procedure;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface PeerProcedureInterface {
+
+  enum PeerOperationType {
+    ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH
+  }
+
+  String getPeerId();
+
+  PeerOperationType getPeerOperationType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 65c4d08..0f68f31 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -36,10 +35,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.ipc.RemoteException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
@@ -49,6 +45,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProc
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
 /**
  * A remote procecdure dispatcher for regionservers.
@@ -222,7 +225,10 @@ public class RSProcedureDispatcher
 
   private interface RemoteProcedureResolver {
     void dispatchOpenRequests(MasterProcedureEnv env, List<RegionOpenOperation> operations);
+
     void dispatchCloseRequests(MasterProcedureEnv env, List<RegionCloseOperation> operations);
+
+    void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations);
   }
 
   /**
@@ -231,22 +237,28 @@ public class RSProcedureDispatcher
    * Then {@code resolver} is used to dispatch {@link RegionOpenOperation}s and
    * {@link RegionCloseOperation}s.
    * @param serverName RegionServer to which the remote operations are sent
-   * @param remoteProcedures Remote procedures which are dispatched to the given server
+   * @param operations Remote procedures which are dispatched to the given server
    * @param resolver Used to dispatch remote procedures to given server.
    */
-  public void splitAndResolveOperation(final ServerName serverName,
-      final Set<RemoteProcedure> remoteProcedures, final RemoteProcedureResolver resolver) {
-    final ArrayListMultimap<Class<?>, RemoteOperation> reqsByType =
-      buildAndGroupRequestByType(procedureEnv, serverName, remoteProcedures);
+  public void splitAndResolveOperation(ServerName serverName, Set<RemoteProcedure> operations,
+      RemoteProcedureResolver resolver) {
+    MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+    ArrayListMultimap<Class<?>, RemoteOperation> reqsByType =
+      buildAndGroupRequestByType(env, serverName, operations);
 
-    final List<RegionOpenOperation> openOps = fetchType(reqsByType, RegionOpenOperation.class);
+    List<RegionOpenOperation> openOps = fetchType(reqsByType, RegionOpenOperation.class);
     if (!openOps.isEmpty()) {
-      resolver.dispatchOpenRequests(procedureEnv, openOps);
+      resolver.dispatchOpenRequests(env, openOps);
     }
 
-    final List<RegionCloseOperation> closeOps = fetchType(reqsByType, RegionCloseOperation.class);
+    List<RegionCloseOperation> closeOps = fetchType(reqsByType, RegionCloseOperation.class);
     if (!closeOps.isEmpty()) {
-      resolver.dispatchCloseRequests(procedureEnv, closeOps);
+      resolver.dispatchCloseRequests(env, closeOps);
+    }
+
+    List<ServerOperation> refreshOps = fetchType(reqsByType, ServerOperation.class);
+    if (!refreshOps.isEmpty()) {
+      resolver.dispatchServerOperations(env, refreshOps);
     }
 
     if (!reqsByType.isEmpty()) {
@@ -277,8 +289,7 @@ public class RSProcedureDispatcher
       splitAndResolveOperation(getServerName(), remoteProcedures, this);
 
       try {
-        final ExecuteProceduresResponse response = sendRequest(getServerName(), request.build());
-        remoteCallCompleted(procedureEnv, response);
+        sendRequest(getServerName(), request.build());
       } catch (IOException e) {
         e = unwrapException(e);
         // TODO: In the future some operation may want to bail out early.
@@ -302,6 +313,11 @@ public class RSProcedureDispatcher
       }
     }
 
+    @Override
+    public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) {
+      operations.stream().map(o -> o.buildRequest()).forEachOrdered(request::addProc);
+    }
+
     protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
         final ExecuteProceduresRequest request) throws IOException {
       try {
@@ -311,17 +327,8 @@ public class RSProcedureDispatcher
       }
     }
 
-
-    private void remoteCallCompleted(final MasterProcedureEnv env,
-        final ExecuteProceduresResponse response) {
-      /*
-      for (RemoteProcedure proc: operations) {
-        proc.remoteCallCompleted(env, getServerName(), response);
-      }*/
-    }
-
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
-      for (RemoteProcedure proc: remoteProcedures) {
+      for (RemoteProcedure proc : remoteProcedures) {
         proc.remoteCallFailed(env, getServerName(), e);
       }
     }
@@ -362,8 +369,7 @@ public class RSProcedureDispatcher
           buildOpenRegionRequest(procedureEnv, getServerName(), operations);
 
       try {
-        OpenRegionResponse response = sendRequest(getServerName(), request);
-        remoteCallCompleted(procedureEnv, response);
+        sendRequest(getServerName(), request);
       } catch (IOException e) {
         e = unwrapException(e);
         // TODO: In the future some operation may want to bail out early.
@@ -384,16 +390,6 @@ public class RSProcedureDispatcher
       }
     }
 
-    private void remoteCallCompleted(final MasterProcedureEnv env,
-        final OpenRegionResponse response) {
-      int index = 0;
-      for (RegionOpenOperation op: operations) {
-        OpenRegionResponse.RegionOpeningState state = response.getOpeningState(index++);
-        op.setFailedOpen(state == OpenRegionResponse.RegionOpeningState.FAILED_OPENING);
-        op.getRemoteProcedure().remoteCallCompleted(env, getServerName(), op);
-      }
-    }
-
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
       for (RegionOpenOperation op: operations) {
         op.getRemoteProcedure().remoteCallFailed(env, getServerName(), e);
@@ -443,7 +439,6 @@ public class RSProcedureDispatcher
     private void remoteCallCompleted(final MasterProcedureEnv env,
         final CloseRegionResponse response) {
       operation.setClosed(response.getClosed());
-      operation.getRemoteProcedure().remoteCallCompleted(env, getServerName(), operation);
     }
 
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
@@ -482,6 +477,11 @@ public class RSProcedureDispatcher
         submitTask(new CloseRegionRemoteCall(serverName, op));
       }
     }
+
+    @Override
+    public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) {
+      throw new UnsupportedOperationException();
+    }
   }
 
   // ==========================================================================
@@ -489,13 +489,28 @@ public class RSProcedureDispatcher
   //  - ServerOperation: refreshConfig, grant, revoke, ... (TODO)
   //  - RegionOperation: open, close, flush, snapshot, ...
   // ==========================================================================
-  /* Currently unused
-  public static abstract class ServerOperation extends RemoteOperation {
-    protected ServerOperation(final RemoteProcedure remoteProcedure) {
+
+  public static final class ServerOperation extends RemoteOperation {
+
+    private final long procId;
+
+    private final Class<?> rsProcClass;
+
+    private final byte[] rsProcData;
+
+    public ServerOperation(RemoteProcedure remoteProcedure, long procId, Class<?> rsProcClass,
+        byte[] rsProcData) {
       super(remoteProcedure);
+      this.procId = procId;
+      this.rsProcClass = rsProcClass;
+      this.rsProcData = rsProcData;
+    }
+
+    public RemoteProcedureRequest buildRequest() {
+      return RemoteProcedureRequest.newBuilder().setProcId(procId)
+          .setProcClass(rsProcClass.getName()).setProcData(ByteString.copyFrom(rsProcData)).build();
     }
   }
-  */
 
   public static abstract class RegionOperation extends RemoteOperation {
     private final RegionInfo regionInfo;

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
new file mode 100644
index 0000000..fca05a7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -0,0 +1,127 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
+
+@InterfaceAudience.Private
+public abstract class ModifyPeerProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
+    implements PeerProcedureInterface {
+
+  private static final Log LOG = LogFactory.getLog(ModifyPeerProcedure.class);
+
+  protected String peerId;
+
+  protected ModifyPeerProcedure() {
+  }
+
+  protected ModifyPeerProcedure(String peerId) {
+    this.peerId = peerId;
+  }
+
+  @Override
+  public String getPeerId() {
+    return peerId;
+  }
+
+  /**
+   * Return {@code false} means that the operation is invalid and we should give up, otherwise
+   * {@code true}.
+   * <p>
+   * You need to call {@link #setFailure(String, Throwable)} to give the detail failure information.
+   */
+  protected abstract boolean updatePeerStorage() throws IOException;
+
+  protected void postPeerModification() {
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    switch (state) {
+      case UPDATE_PEER_STORAGE:
+        try {
+          if (!updatePeerStorage()) {
+            assert isFailed() : "setFailure is not called";
+            return Flow.NO_MORE_STATE;
+          }
+        } catch (IOException e) {
+          LOG.warn("update peer storage failed, retry", e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
+        return Flow.HAS_MORE_STATE;
+      case REFRESH_PEER_ON_RS:
+        addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
+            .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn))
+            .toArray(RefreshPeerProcedure[]::new));
+        setNextState(PeerModificationState.POST_PEER_MODIFICATION);
+        return Flow.HAS_MORE_STATE;
+      case POST_PEER_MODIFICATION:
+        postPeerModification();
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+  }
+
+  @Override
+  protected LockState acquireLock(MasterProcedureEnv env) {
+    return env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)
+      ? LockState.LOCK_EVENT_WAIT
+      : LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void releaseLock(MasterProcedureEnv env) {
+    env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId);
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
+      throws IOException, InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected PeerModificationState getState(int stateId) {
+    return PeerModificationState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(PeerModificationState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected PeerModificationState getInitialState() {
+    return PeerModificationState.UPDATE_PEER_STORAGE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
new file mode 100644
index 0000000..4e09107
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
@@ -0,0 +1,67 @@
+/**
+ * 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.replication;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+
+/**
+ * The callable executed at RS side to refresh the peer config/state.
+ * <p>
+ * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ */
+@InterfaceAudience.Private
+public class RefreshPeerCallable implements RSProcedureCallable {
+
+  private HRegionServer rs;
+
+  private String peerId;
+
+  private Exception initError;
+
+  @Override
+  public Void call() throws Exception {
+    if (initError != null) {
+      throw initError;
+    }
+    rs.getFileSystem().create(new Path("/" + peerId + "/" + rs.getServerName().toString())).close();
+    return null;
+  }
+
+  @Override
+  public void init(byte[] parameter, HRegionServer rs) {
+    this.rs = rs;
+    try {
+      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+    } catch (InvalidProtocolBufferException e) {
+      initError = e;
+      return;
+    }
+  }
+
+  @Override
+  public EventType getEventType() {
+    return EventType.RS_REFRESH_PEER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
new file mode 100644
index 0000000..18da487
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -0,0 +1,197 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.ServerOperation;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerStateData;
+
+@InterfaceAudience.Private
+public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
+    implements PeerProcedureInterface, RemoteProcedure<MasterProcedureEnv, ServerName> {
+
+  private static final Log LOG = LogFactory.getLog(RefreshPeerProcedure.class);
+
+  private String peerId;
+
+  private PeerOperationType type;
+
+  private ServerName targetServer;
+
+  private boolean dispatched;
+
+  private ProcedureEvent<?> event;
+
+  private boolean succ;
+
+  public RefreshPeerProcedure() {
+  }
+
+  public RefreshPeerProcedure(String peerId, PeerOperationType type, ServerName targetServer) {
+    this.peerId = peerId;
+    this.type = type;
+    this.targetServer = targetServer;
+  }
+
+  @Override
+  public String getPeerId() {
+    return peerId;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.REFRESH;
+  }
+
+  private static PeerModificationType toPeerModificationType(PeerOperationType type) {
+    switch (type) {
+      case ADD:
+        return PeerModificationType.ADD_PEER;
+      case REMOVE:
+        return PeerModificationType.REMOVE_PEER;
+      case ENABLE:
+        return PeerModificationType.ENABLE_PEER;
+      case DISABLE:
+        return PeerModificationType.DISABLE_PEER;
+      case UPDATE_CONFIG:
+        return PeerModificationType.UPDATE_PEER_CONFIG;
+      default:
+        throw new IllegalArgumentException("Unknown type: " + type);
+    }
+  }
+
+  private static PeerOperationType toPeerOperationType(PeerModificationType type) {
+    switch (type) {
+      case ADD_PEER:
+        return PeerOperationType.ADD;
+      case REMOVE_PEER:
+        return PeerOperationType.REMOVE;
+      case ENABLE_PEER:
+        return PeerOperationType.ENABLE;
+      case DISABLE_PEER:
+        return PeerOperationType.DISABLE;
+      case UPDATE_PEER_CONFIG:
+        return PeerOperationType.UPDATE_CONFIG;
+      default:
+        throw new IllegalArgumentException("Unknown type: " + type);
+    }
+  }
+
+  @Override
+  public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
+    assert targetServer.equals(remote);
+    return new ServerOperation(this, getProcId(), RefreshPeerCallable.class,
+        RefreshPeerParameter.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type))
+            .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray());
+  }
+
+  private void complete(MasterProcedureEnv env, boolean succ) {
+    if (event == null) {
+      LOG.warn("procedure event for " + getProcId() +
+          " is null, maybe the procedure is created when recovery", new Exception());
+      return;
+    }
+    LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer +
+        (succ ? " suceeded" : " failed"));
+    this.succ = succ;
+    event.wake(env.getProcedureScheduler());
+    event = null;
+  }
+
+  @Override
+  public synchronized void remoteCallFailed(MasterProcedureEnv env, ServerName remote,
+      IOException exception) {
+    complete(env, false);
+  }
+
+  @Override
+  public synchronized void remoteOperationCompleted(MasterProcedureEnv env) {
+    complete(env, true);
+  }
+
+  @Override
+  public synchronized void remoteOperationFailed(MasterProcedureEnv env, String error) {
+    complete(env, false);
+  }
+
+  @Override
+  protected synchronized Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
+      throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
+    if (dispatched) {
+      if (succ) {
+        return null;
+      }
+      // retry
+      dispatched = false;
+    }
+    if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) {
+      LOG.info("Can not add remote operation for refreshing peer " + peerId + " for " + type +
+          " to " + targetServer + ", this usually because the server is already dead," +
+          " give up and mark the procedure as complete");
+      return null;
+    }
+    dispatched = true;
+    event = new ProcedureEvent<>(this);
+    event.suspendIfNotReady(this);
+    throw new ProcedureSuspendedException();
+  }
+
+  @Override
+  protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    // TODO: no correctness problem if we just ignore this, implement later.
+    return false;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    serializer.serialize(
+      RefreshPeerStateData.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type))
+          .setTargetServer(ProtobufUtil.toServerName(targetServer)).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    RefreshPeerStateData data = serializer.deserialize(RefreshPeerStateData.class);
+    peerId = data.getPeerId();
+    type = toPeerOperationType(data.getType());
+    targetServer = ProtobufUtil.toServerName(data.getTargetServer());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
new file mode 100644
index 0000000..62c2e36
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
@@ -0,0 +1,43 @@
+/**
+ * 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.procedure2;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A general interface for a sub procedure runs at RS side.
+ */
+@InterfaceAudience.Private
+public interface RSProcedureCallable extends Callable<Void> {
+
+  /**
+   * Initialize the callable
+   * @param parameter the parameter passed from master.
+   * @param rs the regionserver instance
+   */
+  void init(byte[] parameter, HRegionServer rs);
+
+  /**
+   * Event type used to select thread pool.
+   */
+  EventType getEventType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/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 37ec595..65cfd18 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
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.servlet.http.HttpServlet;
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.lang.management.MemoryType;
@@ -49,7 +46,9 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
-
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.servlet.http.HttpServlet;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.SystemUtils;
@@ -117,6 +116,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
@@ -127,6 +127,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
+import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler;
 import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler;
 import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
@@ -172,6 +173,9 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import sun.misc.Signal;
+import sun.misc.SignalHandler;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
@@ -181,6 +185,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
@@ -206,13 +211,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 
-import sun.misc.Signal;
-import sun.misc.SignalHandler;
-
 /**
  * HRegionServer makes a set of HRegions available to clients. It checks in with
  * the HMaster. There are many HRegionServers in a single HBase deployment.
@@ -1917,6 +1920,8 @@ public class HRegionServer extends HasThread implements
           conf.getInt("hbase.regionserver.region.replica.flusher.threads",
               conf.getInt("hbase.regionserver.executor.openregion.threads", 3)));
     }
+    this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER,
+      conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2));
 
     Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
     uncaughtExceptionHandler);
@@ -3698,4 +3703,60 @@ public class HRegionServer extends HasThread implements
     return ConnectionUtils.createShortCircuitConnection(conf, null, user, this.serverName,
         this.rpcServices, this.rpcServices);
   }
+
+  public void executeProcedure(long procId, RSProcedureCallable callable) {
+    executorService.submit(new RSProcedureHandler(this, procId, callable));
+  }
+
+  public void reportProcedureDone(long procId, Throwable error) {
+    ReportProcedureDoneRequest.Builder builder =
+      ReportProcedureDoneRequest.newBuilder().setProcId(procId);
+    if (error != null) {
+      builder.setStatus(ReportProcedureDoneRequest.Status.ERROR)
+          .setError(Throwables.getStackTraceAsString(error));
+    } else {
+      builder.setStatus(ReportProcedureDoneRequest.Status.SUCCESS);
+    }
+    ReportProcedureDoneRequest request = builder.build();
+    int tries = 0;
+    long pauseTime = INIT_PAUSE_TIME_MS;
+    while (keepLooping()) {
+      RegionServerStatusService.BlockingInterface rss = rssStub;
+      try {
+        if (rss == null) {
+          createRegionServerStatusStub();
+          continue;
+        }
+        rss.reportProcedureDone(null, request);
+        // Log if we had to retry else don't log unless TRACE. We want to
+        // know if were successful after an attempt showed in logs as failed.
+        if (tries > 0 || LOG.isTraceEnabled()) {
+          LOG.info("PROCEDURE REPORTED " + request);
+        }
+        return;
+      } catch (ServiceException se) {
+        IOException ioe = ProtobufUtil.getRemoteException(se);
+        boolean pause =
+          ioe instanceof ServerNotRunningYetException || ioe instanceof PleaseHoldException;
+        if (pause) {
+          // Do backoff else we flood the Master with requests.
+          pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
+        } else {
+          pauseTime = INIT_PAUSE_TIME_MS; // Reset.
+        }
+        LOG.info(
+          "Failed to report transition " + TextFormat.shortDebugString(request) + "; retry (#" +
+            tries + ")" + (pause ? " after " + pauseTime + "ms delay (Master is coming online...)."
+              : " immediately."),
+          ioe);
+        if (pause) {
+          Threads.sleep(pauseTime);
+        }
+        tries++;
+        if (rssStub == rss) {
+          rssStub = null;
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index b6c0ebe..e88f70e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -100,6 +99,7 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.OperationQuota;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
@@ -172,6 +172,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
@@ -3435,23 +3436,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   }
 
   @Override
-  public ExecuteProceduresResponse executeProcedures(RpcController controller,
-       ExecuteProceduresRequest request) throws ServiceException {
-    ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
-    if (request.getOpenRegionCount() > 0) {
-      for (OpenRegionRequest req : request.getOpenRegionList()) {
-        builder.addOpenRegion(openRegion(controller, req));
-      }
-    }
-    if (request.getCloseRegionCount() > 0) {
-      for (CloseRegionRequest req : request.getCloseRegionList()) {
-        builder.addCloseRegion(closeRegion(controller, req));
-      }
-    }
-    return builder.build();
-  }
-
-  @Override
   public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,
       ClearRegionBlockCacheRequest request) {
     ClearRegionBlockCacheResponse.Builder builder =
@@ -3468,4 +3452,38 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     stats.withMaxCacheSize(regionServer.getCacheConfig().getBlockCache().getMaxSize());
     return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build();
   }
+
+  @Override
+  public ExecuteProceduresResponse executeProcedures(RpcController controller,
+      ExecuteProceduresRequest request) throws ServiceException {
+    if (request.getOpenRegionCount() > 0) {
+      for (OpenRegionRequest req : request.getOpenRegionList()) {
+        openRegion(controller, req);
+      }
+    }
+    if (request.getCloseRegionCount() > 0) {
+      for (CloseRegionRequest req : request.getCloseRegionList()) {
+        closeRegion(controller, req);
+      }
+    }
+    if (request.getProcCount() > 0) {
+      for (RemoteProcedureRequest req : request.getProcList()) {
+        RSProcedureCallable callable;
+        try {
+          callable =
+            Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance();
+        } catch (Exception e) {
+          // here we just ignore the error as this should not happen and we do not provide a general
+          // way to report errors for all types of remote procedure. The procedure will hang at
+          // master side but after you solve the problem and restart master it will be executed
+          // again and pass.
+          LOG.warn("create procedure of type " + req.getProcClass() + " failed, give up", e);
+          continue;
+        }
+        callable.init(req.getProcData().toByteArray(), regionServer);
+        regionServer.executeProcedure(req.getProcId(), callable);
+      }
+    }
+    return ExecuteProceduresResponse.getDefaultInstance();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
new file mode 100644
index 0000000..94bcfec
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
@@ -0,0 +1,51 @@
+/**
+ * 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.regionserver.handler;
+
+import org.apache.hadoop.hbase.executor.EventHandler;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A event handler for running procedure.
+ */
+@InterfaceAudience.Private
+public class RSProcedureHandler extends EventHandler {
+
+  private final long procId;
+
+  private final RSProcedureCallable callable;
+
+  public RSProcedureHandler(HRegionServer rs, long procId, RSProcedureCallable callable) {
+    super(rs, callable.getEventType());
+    this.procId = procId;
+    this.callable = callable;
+  }
+
+  @Override
+  public void process() {
+    Exception error = null;
+    try {
+      callable.call();
+    } catch (Exception e) {
+      error = e;
+    }
+    ((HRegionServer) server).reportProcedureDone(procId, error);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
index 3c453bc..3ab915b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -531,26 +531,16 @@ public class TestAssignmentManager {
     @Override
     public ExecuteProceduresResponse sendRequest(ServerName server,
         ExecuteProceduresRequest request) throws IOException {
-      ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
       if (request.getOpenRegionCount() > 0) {
-        for (OpenRegionRequest req: request.getOpenRegionList()) {
-          OpenRegionResponse.Builder resp = OpenRegionResponse.newBuilder();
-          for (RegionOpenInfo openReq: req.getOpenInfoList()) {
-            RegionOpeningState state = execOpenRegion(server, openReq);
-            if (state != null) {
-              resp.addOpeningState(state);
-            }
+        for (OpenRegionRequest req : request.getOpenRegionList()) {
+          for (RegionOpenInfo openReq : req.getOpenInfoList()) {
+            execOpenRegion(server, openReq);
           }
-          builder.addOpenRegion(resp.build());
         }
       }
       if (request.getCloseRegionCount() > 0) {
-        for (CloseRegionRequest req: request.getCloseRegionList()) {
-          CloseRegionResponse resp = execCloseRegion(server,
-              req.getRegion().getValue().toByteArray());
-          if (resp != null) {
-            builder.addCloseRegion(resp);
-          }
+        for (CloseRegionRequest req : request.getCloseRegionList()) {
+          execCloseRegion(server, req.getRegion().getValue().toByteArray());
         }
       }
       return ExecuteProceduresResponse.newBuilder().build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
new file mode 100644
index 0000000..44343d7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
@@ -0,0 +1,41 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+
+public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
+
+  public DummyModifyPeerProcedure() {
+  }
+
+  public DummyModifyPeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ADD;
+  }
+
+  @Override
+  protected boolean updatePeerStorage() throws IOException {
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
new file mode 100644
index 0000000..ec06306
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.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.master.replication;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestDummyModifyPeerProcedure {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID;
+
+  private static Path DIR;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(3);
+    PEER_ID = "testPeer";
+    DIR = new Path("/" + PEER_ID);
+    UTIL.getTestFileSystem().mkdirs(DIR);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    ProcedureExecutor<?> executor =
+        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    long procId = executor.submitProcedure(new DummyModifyPeerProcedure(PEER_ID));
+    UTIL.waitFor(30000, new Waiter.Predicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return executor.isFinished(procId);
+      }
+    });
+    Set<String> serverNames = UTIL.getHBaseCluster().getRegionServerThreads().stream()
+        .map(t -> t.getRegionServer().getServerName().toString())
+        .collect(Collectors.toCollection(HashSet::new));
+    for (FileStatus s : UTIL.getTestFileSystem().listStatus(DIR)) {
+      assertTrue(serverNames.remove(s.getPath().getName()));
+    }
+    assertTrue(serverNames.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/12fd193a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index be1b0e4..99e212d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -30,6 +30,7 @@ import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
+
 import java.io.IOException;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;


[46/48] hbase git commit: HBASE-19544 Add UTs for testing concurrent modifications on replication peer

Posted by zh...@apache.org.
HBASE-19544 Add UTs for testing concurrent modifications on replication peer

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 6b7bed4ea50cd1bef582566b0ebe13ebd8c3cafe
Parents: def5bee
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Jan 2 17:07:41 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../replication/TestReplicationAdmin.java       | 69 ++++++++++++++++++++
 1 file changed, 69 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6b7bed4e/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 f9629bd..9edd4a2 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
@@ -31,6 +31,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -56,6 +57,8 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Unit testing of ReplicationAdmin
@@ -63,6 +66,8 @@ import org.junit.rules.TestName;
 @Category({MediumTests.class, ClientTests.class})
 public class TestReplicationAdmin {
 
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationAdmin.class);
+
   private final static HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
@@ -112,6 +117,70 @@ public class TestReplicationAdmin {
   }
 
   @Test
+  public void testConcurrentPeerOperations() throws Exception {
+    int threadNum = 5;
+    AtomicLong successCount = new AtomicLong(0);
+
+    // Test concurrent add peer operation
+    Thread[] addPeers = new Thread[threadNum];
+    for (int i = 0; i < threadNum; i++) {
+      addPeers[i] = new Thread(() -> {
+        try {
+          hbaseAdmin.addReplicationPeer(ID_ONE,
+            ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build());
+          successCount.incrementAndGet();
+        } catch (Exception e) {
+          LOG.debug("Got exception when add replication peer", e);
+        }
+      });
+      addPeers[i].start();
+    }
+    for (Thread addPeer : addPeers) {
+      addPeer.join();
+    }
+    assertEquals(1, successCount.get());
+
+    // Test concurrent remove peer operation
+    successCount.set(0);
+    Thread[] removePeers = new Thread[threadNum];
+    for (int i = 0; i < threadNum; i++) {
+      removePeers[i] = new Thread(() -> {
+        try {
+          hbaseAdmin.removeReplicationPeer(ID_ONE);
+          successCount.incrementAndGet();
+        } catch (Exception e) {
+          LOG.debug("Got exception when remove replication peer", e);
+        }
+      });
+      removePeers[i].start();
+    }
+    for (Thread removePeer : removePeers) {
+      removePeer.join();
+    }
+    assertEquals(1, successCount.get());
+
+    // Test concurrent add peer operation again
+    successCount.set(0);
+    addPeers = new Thread[threadNum];
+    for (int i = 0; i < threadNum; i++) {
+      addPeers[i] = new Thread(() -> {
+        try {
+          hbaseAdmin.addReplicationPeer(ID_ONE,
+            ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build());
+          successCount.incrementAndGet();
+        } catch (Exception e) {
+          LOG.debug("Got exception when add replication peer", e);
+        }
+      });
+      addPeers[i].start();
+    }
+    for (Thread addPeer : addPeers) {
+      addPeer.join();
+    }
+    assertEquals(1, successCount.get());
+  }
+
+  @Test
   public void testAddInvalidPeer() {
     ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_ONE);


[28/48] hbase git commit: HBASE-19573 Rewrite ReplicationPeer with the new replication storage interface

Posted by zh...@apache.org.
HBASE-19573 Rewrite ReplicationPeer with the new replication storage interface


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 82a427231d7286977734c89910682cf6755dc487
Parents: 80a36ad
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 26 11:39:34 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../replication/VerifyReplication.java          |   5 -
 .../hbase/replication/ReplicationPeer.java      |  42 ++--
 .../hbase/replication/ReplicationPeerImpl.java  | 169 ++++++++++++++
 .../replication/ReplicationPeerZKImpl.java      | 233 -------------------
 .../hbase/replication/ReplicationPeers.java     |   4 +-
 .../replication/ReplicationPeersZKImpl.java     |  23 +-
 .../replication/TestReplicationStateBasic.java  |   7 +-
 .../regionserver/PeerProcedureHandlerImpl.java  |  29 +--
 8 files changed, 216 insertions(+), 296 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/82a42723/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 9065f4e..09d4b4b 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
@@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.mapreduce.TableSplit;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -333,7 +332,6 @@ public class VerifyReplication extends Configured implements Tool {
   private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
       final Configuration conf, String peerId) throws IOException {
     ZKWatcher localZKW = null;
-    ReplicationPeerZKImpl peer = null;
     try {
       localZKW = new ZKWatcher(conf, "VerifyReplication",
           new Abortable() {
@@ -354,9 +352,6 @@ public class VerifyReplication extends Configured implements Tool {
       throw new IOException(
           "An error occurred while trying to connect to the remove peer cluster", e);
     } finally {
-      if (peer != null) {
-        peer.close();
-      }
       if (localZKW != null) {
         localZKW.close();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a42723/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
index b66d76d..4846018 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
-
 /**
  * ReplicationPeer manages enabled / disabled state for the peer.
  */
@@ -49,65 +48,52 @@ public interface ReplicationPeer {
   String getId();
 
   /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
-  public ReplicationPeerConfig getPeerConfig();
-
-  /**
-   * Get the peer config object. if loadFromBackingStore is true, it will load from backing store
-   * directly and update its load peer config. otherwise, just return the local cached peer config.
-   * @return the ReplicationPeerConfig for this peer
-   */
-  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
-      throws ReplicationException;
-
-  /**
    * Returns the state of the peer by reading local cache.
    * @return the enabled state
    */
   PeerState getPeerState();
 
   /**
-   * Returns the state of peer, if loadFromBackingStore is true, it will load from backing store
-   * directly and update its local peer state. otherwise, just return the local cached peer state.
-   * @return the enabled state
+   * Get the peer config object
+   * @return the ReplicationPeerConfig for this peer
    */
-  PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException;
+  ReplicationPeerConfig getPeerConfig();
 
   /**
    * Get the configuration object required to communicate with this peer
    * @return configuration object
    */
-  public Configuration getConfiguration();
+  Configuration getConfiguration();
 
   /**
    * Get replicable (table, cf-list) map of this peer
    * @return the replicable (table, cf-list) map
    */
-  public Map<TableName, List<String>> getTableCFs();
+  Map<TableName, List<String>> getTableCFs();
 
   /**
    * Get replicable namespace set of this peer
    * @return the replicable namespaces set
    */
-  public Set<String> getNamespaces();
+  Set<String> getNamespaces();
 
   /**
    * Get the per node bandwidth upper limit for this peer
    * @return the bandwidth upper limit
    */
-  public long getPeerBandwidth();
+  long getPeerBandwidth();
 
   /**
    * Register a peer config listener to catch the peer config change event.
    * @param listener listener to catch the peer config change event.
    */
-  public void registerPeerConfigListener(ReplicationPeerConfigListener listener);
+  void registerPeerConfigListener(ReplicationPeerConfigListener listener);
 
   /**
-   * Notify all the registered ReplicationPeerConfigListener to update their peer config.
-   * @param newPeerConfig the new peer config.
+   * @deprecated Use {@link #registerPeerConfigListener(ReplicationPeerConfigListener)} instead.
    */
-  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig);
-}
+  @Deprecated
+  default void trackPeerConfigChanges(ReplicationPeerConfigListener listener) {
+    registerPeerConfigListener(listener);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a42723/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
new file mode 100644
index 0000000..2c7ea9b
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
@@ -0,0 +1,169 @@
+/*
+ *
+ * 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.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+@InterfaceAudience.Private
+public class ReplicationPeerImpl implements ReplicationPeer {
+  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerImpl.class);
+
+  private final ReplicationPeerStorage peerStorage;
+
+  private final Configuration conf;
+
+  private final String id;
+
+  private volatile ReplicationPeerConfig peerConfig;
+
+  private volatile PeerState peerState;
+
+  private final List<ReplicationPeerConfigListener> peerConfigListeners;
+
+  /**
+   * Constructor that takes all the objects required to communicate with the specified peer, except
+   * for the region server addresses.
+   * @param conf configuration object to this peer
+   * @param id string representation of this peer's identifier
+   * @param peerConfig configuration for the replication peer
+   */
+  public ReplicationPeerImpl(ZKWatcher zkWatcher, Configuration conf, String id,
+      ReplicationPeerConfig peerConfig) {
+    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkWatcher, conf);
+    this.conf = conf;
+    this.peerConfig = peerConfig;
+    this.id = id;
+    this.peerConfigListeners = new ArrayList<>();
+  }
+
+  public void refreshPeerState() throws ReplicationException {
+    this.peerState = peerStorage.isPeerEnabled(id) ? PeerState.ENABLED : PeerState.DISABLED;
+  }
+
+  public void refreshPeerConfig() throws ReplicationException {
+    this.peerConfig = peerStorage.getPeerConfig(id).orElse(peerConfig);
+    peerConfigListeners.forEach(listener -> listener.peerConfigUpdated(peerConfig));
+  }
+
+  /**
+   * Get the identifier of this peer
+   * @return string representation of the id (short)
+   */
+  @Override
+  public String getId() {
+    return id;
+  }
+
+  @Override
+  public PeerState getPeerState() {
+    return peerState;
+  }
+
+  /**
+   * Get the peer config object
+   * @return the ReplicationPeerConfig for this peer
+   */
+  @Override
+  public ReplicationPeerConfig getPeerConfig() {
+    return peerConfig;
+  }
+
+  /**
+   * Get the configuration object required to communicate with this peer
+   * @return configuration object
+   */
+  @Override
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  /**
+   * Get replicable (table, cf-list) map of this peer
+   * @return the replicable (table, cf-list) map
+   */
+  @Override
+  public Map<TableName, List<String>> getTableCFs() {
+    return this.peerConfig.getTableCFsMap();
+  }
+
+  /**
+   * Get replicable namespace set of this peer
+   * @return the replicable namespaces set
+   */
+  @Override
+  public Set<String> getNamespaces() {
+    return this.peerConfig.getNamespaces();
+  }
+
+  @Override
+  public long getPeerBandwidth() {
+    return this.peerConfig.getBandwidth();
+  }
+
+  @Override
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
+    this.peerConfigListeners.add(listener);
+  }
+
+  /**
+   * Parse the raw data from ZK to get a peer's state
+   * @param bytes raw ZK data
+   * @return True if the passed in <code>bytes</code> are those of a pb serialized ENABLED state.
+   * @throws DeserializationException
+   */
+  public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
+    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
+    return ReplicationProtos.ReplicationState.State.ENABLED == state;
+  }
+
+  /**
+   * @param bytes Content of a state znode.
+   * @return State parsed from the passed bytes.
+   * @throws DeserializationException
+   */
+  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
+      throws DeserializationException {
+    ProtobufUtil.expectPBMagicPrefix(bytes);
+    int pbLen = ProtobufUtil.lengthOfPBMagic();
+    ReplicationProtos.ReplicationState.Builder builder =
+        ReplicationProtos.ReplicationState.newBuilder();
+    ReplicationProtos.ReplicationState state;
+    try {
+      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
+      state = builder.build();
+      return state.getState();
+    } catch (IOException e) {
+      throw new DeserializationException(e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a42723/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
deleted file mode 100644
index 49b9460..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ /dev/null
@@ -1,233 +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.replication;
-
-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.Set;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.log.HBaseMarkers;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-
-@InterfaceAudience.Private
-public class ReplicationPeerZKImpl extends ReplicationStateZKBase
-    implements ReplicationPeer, Abortable, Closeable {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerZKImpl.class);
-
-  private volatile ReplicationPeerConfig peerConfig;
-  private final String id;
-  private volatile PeerState peerState;
-  private volatile Map<TableName, List<String>> tableCFs = new HashMap<>();
-  private final Configuration conf;
-
-  private final List<ReplicationPeerConfigListener> peerConfigListeners;
-
-  /**
-   * Constructor that takes all the objects required to communicate with the specified peer, except
-   * for the region server addresses.
-   * @param conf configuration object to this peer
-   * @param id string representation of this peer's identifier
-   * @param peerConfig configuration for the replication peer
-   */
-  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf, String id,
-      ReplicationPeerConfig peerConfig, Abortable abortable) throws ReplicationException {
-    super(zkWatcher, conf, abortable);
-    this.conf = conf;
-    this.peerConfig = peerConfig;
-    this.id = id;
-    this.peerConfigListeners = new ArrayList<>();
-  }
-
-  private PeerState readPeerState() throws ReplicationException {
-    try {
-      byte[] data = ZKUtil.getData(zookeeper, this.getPeerStateNode(id));
-      this.peerState = isStateEnabled(data) ? PeerState.ENABLED : PeerState.DISABLED;
-    } catch (DeserializationException | KeeperException | InterruptedException e) {
-      throw new ReplicationException("Get and deserialize peer state data from zookeeper failed: ",
-          e);
-    }
-    return this.peerState;
-  }
-
-  private ReplicationPeerConfig readPeerConfig() throws ReplicationException {
-    try {
-      byte[] data = ZKUtil.getData(zookeeper, this.getPeerNode(id));
-      if (data != null) {
-        this.peerConfig = ReplicationPeerConfigUtil.parsePeerFrom(data);
-      }
-    } catch (DeserializationException | KeeperException | InterruptedException e) {
-      throw new ReplicationException("Get and deserialize peer config date from zookeeper failed: ",
-          e);
-    }
-    return this.peerConfig;
-  }
-
-  @Override
-  public PeerState getPeerState() {
-    return peerState;
-  }
-
-  @Override
-  public PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException {
-    if (loadFromBackingStore) {
-      return readPeerState();
-    } else {
-      return peerState;
-    }
-  }
-
-  /**
-   * Get the identifier of this peer
-   * @return string representation of the id (short)
-   */
-  @Override
-  public String getId() {
-    return id;
-  }
-
-  /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
-  @Override
-  public ReplicationPeerConfig getPeerConfig() {
-    return peerConfig;
-  }
-
-  @Override
-  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
-      throws ReplicationException {
-    if (loadFromBackingStore) {
-      return readPeerConfig();
-    } else {
-      return peerConfig;
-    }
-  }
-
-  /**
-   * Get the configuration object required to communicate with this peer
-   * @return configuration object
-   */
-  @Override
-  public Configuration getConfiguration() {
-    return conf;
-  }
-
-  /**
-   * Get replicable (table, cf-list) map of this peer
-   * @return the replicable (table, cf-list) map
-   */
-  @Override
-  public Map<TableName, List<String>> getTableCFs() {
-    this.tableCFs = peerConfig.getTableCFsMap();
-    return this.tableCFs;
-  }
-
-  /**
-   * Get replicable namespace set of this peer
-   * @return the replicable namespaces set
-   */
-  @Override
-  public Set<String> getNamespaces() {
-    return this.peerConfig.getNamespaces();
-  }
-
-  @Override
-  public long getPeerBandwidth() {
-    return this.peerConfig.getBandwidth();
-  }
-
-  @Override
-  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
-    this.peerConfigListeners.add(listener);
-  }
-
-  @Override
-  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig) {
-    for (ReplicationPeerConfigListener listener : this.peerConfigListeners) {
-      listener.peerConfigUpdated(newPeerConfig);
-    }
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    LOG.error(HBaseMarkers.FATAL, "The ReplicationPeer corresponding to peer " +
-        peerConfig + " was aborted for the following reason(s):" + why, e);
-  }
-
-  @Override
-  public boolean isAborted() {
-    // Currently the replication peer is never "Aborted", we just log when the
-    // abort method is called.
-    return false;
-  }
-
-  @Override
-  public void close() throws IOException {
-    // TODO: stop zkw?
-  }
-
-  /**
-   * Parse the raw data from ZK to get a peer's state
-   * @param bytes raw ZK data
-   * @return True if the passed in <code>bytes</code> are those of a pb serialized ENABLED state.
-   * @throws DeserializationException
-   */
-  public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
-    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
-    return ReplicationProtos.ReplicationState.State.ENABLED == state;
-  }
-
-  /**
-   * @param bytes Content of a state znode.
-   * @return State parsed from the passed bytes.
-   * @throws DeserializationException
-   */
-  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
-      throws DeserializationException {
-    ProtobufUtil.expectPBMagicPrefix(bytes);
-    int pbLen = ProtobufUtil.lengthOfPBMagic();
-    ReplicationProtos.ReplicationState.Builder builder =
-        ReplicationProtos.ReplicationState.newBuilder();
-    ReplicationProtos.ReplicationState state;
-    try {
-      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
-      state = builder.build();
-      return state.getState();
-    } catch (IOException e) {
-      throw new DeserializationException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a42723/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index 10936bf..afc19bd 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -116,13 +116,13 @@ public interface ReplicationPeers {
       throws ReplicationException;
 
   /**
-   * Returns the ReplicationPeer for the specified connected peer. This ReplicationPeer will
+   * Returns the ReplicationPeerImpl for the specified connected peer. This ReplicationPeer will
    * continue to track changes to the Peer's state and config. This method returns null if no
    * peer has been connected with the given peerId.
    * @param peerId id for the peer
    * @return ReplicationPeer object
    */
-  ReplicationPeer getConnectedPeer(String peerId);
+  ReplicationPeerImpl getConnectedPeer(String peerId);
 
   /**
    * Returns the set of peerIds of the clusters that have been connected and have an underlying

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a42723/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 7de4619..7f6498d 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
@@ -80,7 +80,7 @@ import org.slf4j.LoggerFactory;
 public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements ReplicationPeers {
 
   // Map of peer clusters keyed by their id
-  private Map<String, ReplicationPeerZKImpl> peerClusters;
+  private ConcurrentMap<String, ReplicationPeerImpl> peerClusters;
   private final ReplicationQueueStorage queueStorage;
   private Abortable abortable;
 
@@ -232,7 +232,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       }
       String peerStateZNode = getPeerStateNode(id);
       try {
-        return ReplicationPeerZKImpl.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode));
+        return ReplicationPeerImpl.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode));
       } catch (KeeperException e) {
         throw new ReplicationException(e);
       } catch (DeserializationException e) {
@@ -270,7 +270,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   @Override
-  public ReplicationPeer getConnectedPeer(String peerId) {
+  public ReplicationPeerImpl getConnectedPeer(String peerId) {
     return peerClusters.get(peerId);
   }
 
@@ -423,7 +423,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   public void peerDisconnected(String peerId) {
     ReplicationPeer rp = this.peerClusters.get(peerId);
     if (rp != null) {
-      ((ConcurrentMap<String, ReplicationPeerZKImpl>) peerClusters).remove(peerId, rp);
+      peerClusters.remove(peerId, rp);
     }
   }
 
@@ -440,7 +440,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       return false;
     }
 
-    ReplicationPeerZKImpl peer = null;
+    ReplicationPeerImpl peer = null;
     try {
       peer = createPeer(peerId);
     } catch (Exception e) {
@@ -449,8 +449,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     if (peer == null) {
       return false;
     }
-    ReplicationPeerZKImpl previous =
-      ((ConcurrentMap<String, ReplicationPeerZKImpl>) peerClusters).putIfAbsent(peerId, peer);
+    ReplicationPeerImpl previous = peerClusters.putIfAbsent(peerId, peer);
     if (previous == null) {
       LOG.info("Added new peer cluster=" + peer.getPeerConfig().getClusterKey());
     } else {
@@ -493,19 +492,19 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
    * @return object representing the peer
    * @throws ReplicationException
    */
-  private ReplicationPeerZKImpl createPeer(String peerId) throws ReplicationException {
+  private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
     Pair<ReplicationPeerConfig, Configuration> pair = getPeerConf(peerId);
     if (pair == null) {
       return null;
     }
     Configuration peerConf = pair.getSecond();
 
-    ReplicationPeerZKImpl peer =
-        new ReplicationPeerZKImpl(zookeeper, peerConf, peerId, pair.getFirst(), abortable);
+    ReplicationPeerImpl peer =
+        new ReplicationPeerImpl(zookeeper, peerConf, peerId, pair.getFirst());
 
     // Load peer state and peer config by reading zookeeper directly.
-    peer.getPeerState(true);
-    peer.getPeerConfig(true);
+    peer.refreshPeerState();
+    peer.refreshPeerConfig();
 
     return peer;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a42723/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 6fe869c..8905d43 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -312,12 +312,15 @@ public abstract class TestReplicationStateBasic {
     rp.disablePeer(ID_ONE);
     // now we do not rely on zk watcher to trigger the state change so we need to trigger it
     // manually...
-    assertEquals(PeerState.DISABLED, rp.getConnectedPeer(ID_ONE).getPeerState(true));
+    ReplicationPeerImpl peer = rp.getConnectedPeer(ID_ONE);
+    peer.refreshPeerState();
+    assertEquals(PeerState.DISABLED, peer.getPeerState());
     assertConnectedPeerStatus(false, ID_ONE);
     rp.enablePeer(ID_ONE);
     // now we do not rely on zk watcher to trigger the state change so we need to trigger it
     // manually...
-    assertEquals(PeerState.ENABLED, rp.getConnectedPeer(ID_ONE).getPeerState(true));
+    peer.refreshPeerState();
+    assertEquals(PeerState.ENABLED, peer.getPeerState());
     assertConnectedPeerStatus(true, ID_ONE);
 
     // Disconnect peer

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a42723/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
index 9b493d9..598357c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -21,15 +21,14 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationPeer;
-import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.log4j.Logger;
+import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
-  private static final Logger LOG = Logger.getLogger(PeerProcedureHandlerImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(PeerProcedureHandlerImpl.class);
 
   private ReplicationSourceManager replicationSourceManager;
 
@@ -49,10 +48,11 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
   @Override
   public void disablePeer(String peerId) throws ReplicationException, IOException {
-    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    ReplicationPeerImpl peer =
+        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
     if (peer != null) {
-      PeerState peerState = peer.getPeerState(true);
-      LOG.info("disablePeer state, peer id: " + peerId + ", state: " + peerState);
+      peer.refreshPeerState();
+      LOG.info("disable replication peer, id: " + peerId + ", new state: " + peer.getPeerState());
     } else {
       throw new ReplicationException("No connected peer found, peerId=" + peerId);
     }
@@ -60,10 +60,11 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
   @Override
   public void enablePeer(String peerId) throws ReplicationException, IOException {
-    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    ReplicationPeerImpl peer =
+        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
     if (peer != null) {
-      PeerState peerState = peer.getPeerState(true);
-      LOG.info("enablePeer state, peer id: " + peerId + ", state: " + peerState);
+      peer.refreshPeerState();
+      LOG.info("enable replication peer, id: " + peerId + ", new state: " + peer.getPeerState());
     } else {
       throw new ReplicationException("No connected peer found, peerId=" + peerId);
     }
@@ -71,11 +72,11 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
   @Override
   public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
-    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    ReplicationPeerImpl peer =
+        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
     if (peer == null) {
       throw new ReplicationException("No connected peer found, peerId=" + peerId);
     }
-    ReplicationPeerConfig rpc = peer.getPeerConfig(true);
-    peer.triggerPeerConfigChange(rpc);
+    peer.refreshPeerConfig();
   }
 }


[18/48] hbase git commit: HBASE-19564 Procedure id is missing in the response of peer related operations

Posted by zh...@apache.org.
HBASE-19564 Procedure id is missing in the response of peer related operations


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: d9551d80ece9ec9aca844e66b9ee576b4a37f554
Parents: eb9f8d5
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 20 20:57:37 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/MasterRpcServices.java  | 24 ++++++++++----------
 .../master/replication/ModifyPeerProcedure.java |  4 +---
 2 files changed, 13 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d9551d80/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 8025a51..72bf2d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -1886,10 +1886,10 @@ public class MasterRpcServices extends RSRpcServices
   public AddReplicationPeerResponse addReplicationPeer(RpcController controller,
       AddReplicationPeerRequest request) throws ServiceException {
     try {
-      master.addReplicationPeer(request.getPeerId(),
-        ReplicationPeerConfigUtil.convert(request.getPeerConfig()), request.getPeerState()
-            .getState().equals(ReplicationState.State.ENABLED));
-      return AddReplicationPeerResponse.newBuilder().build();
+      long procId = master.addReplicationPeer(request.getPeerId(),
+        ReplicationPeerConfigUtil.convert(request.getPeerConfig()),
+        request.getPeerState().getState().equals(ReplicationState.State.ENABLED));
+      return AddReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1899,8 +1899,8 @@ public class MasterRpcServices extends RSRpcServices
   public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller,
       RemoveReplicationPeerRequest request) throws ServiceException {
     try {
-      master.removeReplicationPeer(request.getPeerId());
-      return RemoveReplicationPeerResponse.newBuilder().build();
+      long procId = master.removeReplicationPeer(request.getPeerId());
+      return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1910,8 +1910,8 @@ public class MasterRpcServices extends RSRpcServices
   public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller,
       EnableReplicationPeerRequest request) throws ServiceException {
     try {
-      master.enableReplicationPeer(request.getPeerId());
-      return EnableReplicationPeerResponse.newBuilder().build();
+      long procId = master.enableReplicationPeer(request.getPeerId());
+      return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1921,8 +1921,8 @@ public class MasterRpcServices extends RSRpcServices
   public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller,
       DisableReplicationPeerRequest request) throws ServiceException {
     try {
-      master.disableReplicationPeer(request.getPeerId());
-      return DisableReplicationPeerResponse.newBuilder().build();
+      long procId = master.disableReplicationPeer(request.getPeerId());
+      return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1948,9 +1948,9 @@ public class MasterRpcServices extends RSRpcServices
   public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller,
       UpdateReplicationPeerConfigRequest request) throws ServiceException {
     try {
-      master.updateReplicationPeerConfig(request.getPeerId(),
+      long procId = master.updateReplicationPeerConfig(request.getPeerId(),
         ReplicationPeerConfigUtil.convert(request.getPeerConfig()));
-      return UpdateReplicationPeerConfigResponse.newBuilder().build();
+      return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9551d80/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 7076bab..23f6f87 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -54,9 +54,7 @@ public abstract class ModifyPeerProcedure
 
   protected ModifyPeerProcedure(String peerId) {
     this.peerId = peerId;
-    // TODO: temporarily set a 4.0 here to always wait for the procedure exection completed. Change
-    // to 3.0 or 2.0 after the client modification is done.
-    this.latch = ProcedurePrepareLatch.createLatch(4, 0);
+    this.latch = ProcedurePrepareLatch.createLatch(2, 0);
   }
 
   public ProcedurePrepareLatch getLatch() {


[34/48] hbase git commit: HBASE-19634 Add permission check for executeProcedures in AccessController

Posted by zh...@apache.org.
HBASE-19634 Add permission check for executeProcedures in AccessController


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: ec84d0ef118113a0e2f2805a885efd39798c4356
Parents: 3432e91
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 4 16:18:21 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../hbase/coprocessor/RegionServerObserver.java | 14 +++++
 .../hbase/regionserver/RSRpcServices.java       | 54 +++++++++++---------
 .../RegionServerCoprocessorHost.java            | 18 +++++++
 .../hbase/security/access/AccessController.java | 30 ++++++-----
 .../hadoop/hbase/TestJMXConnectorServer.java    |  7 +++
 .../security/access/TestAccessController.java   | 18 +++++--
 6 files changed, 101 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ec84d0ef/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
index c1af3fb..5b751df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
@@ -126,4 +126,18 @@ public interface RegionServerObserver {
   default void postClearCompactionQueues(
       final ObserverContext<RegionServerCoprocessorEnvironment> ctx)
       throws IOException {}
+
+  /**
+   * This will be called before executing procedures
+   * @param ctx the environment to interact with the framework and region server.
+   */
+  default void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
+      throws IOException {}
+
+  /**
+   * This will be called after executing procedures
+   * @param ctx the environment to interact with the framework and region server.
+   */
+  default void postExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
+      throws IOException {}
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec84d0ef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index e88f70e..695b859 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -41,7 +41,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
-
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -142,6 +141,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -3454,36 +3454,40 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   }
 
   @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
   public ExecuteProceduresResponse executeProcedures(RpcController controller,
       ExecuteProceduresRequest request) throws ServiceException {
-    if (request.getOpenRegionCount() > 0) {
-      for (OpenRegionRequest req : request.getOpenRegionList()) {
-        openRegion(controller, req);
+    try {
+      checkOpen();
+      regionServer.getRegionServerCoprocessorHost().preExecuteProcedures();
+      if (request.getOpenRegionCount() > 0) {
+        for (OpenRegionRequest req : request.getOpenRegionList()) {
+          openRegion(controller, req);
+        }
       }
-    }
-    if (request.getCloseRegionCount() > 0) {
-      for (CloseRegionRequest req : request.getCloseRegionList()) {
-        closeRegion(controller, req);
+      if (request.getCloseRegionCount() > 0) {
+        for (CloseRegionRequest req : request.getCloseRegionList()) {
+          closeRegion(controller, req);
+        }
       }
-    }
-    if (request.getProcCount() > 0) {
-      for (RemoteProcedureRequest req : request.getProcList()) {
-        RSProcedureCallable callable;
-        try {
-          callable =
-            Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance();
-        } catch (Exception e) {
-          // here we just ignore the error as this should not happen and we do not provide a general
-          // way to report errors for all types of remote procedure. The procedure will hang at
-          // master side but after you solve the problem and restart master it will be executed
-          // again and pass.
-          LOG.warn("create procedure of type " + req.getProcClass() + " failed, give up", e);
-          continue;
+      if (request.getProcCount() > 0) {
+        for (RemoteProcedureRequest req : request.getProcList()) {
+          RSProcedureCallable callable;
+          try {
+            callable =
+              Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance();
+          } catch (Exception e) {
+            regionServer.remoteProcedureComplete(req.getProcId(), e);
+            continue;
+          }
+          callable.init(req.getProcData().toByteArray(), regionServer);
+          regionServer.executeProcedure(req.getProcId(), callable);
         }
-        callable.init(req.getProcData().toByteArray(), regionServer);
-        regionServer.executeProcedure(req.getProcId(), callable);
       }
+      regionServer.getRegionServerCoprocessorHost().postExecuteProcedures();
+      return ExecuteProceduresResponse.getDefaultInstance();
+    } catch (IOException e) {
+      throw new ServiceException(e);
     }
-    return ExecuteProceduresResponse.getDefaultInstance();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec84d0ef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
index dc1708c..09617c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
@@ -205,6 +205,24 @@ public class RegionServerCoprocessorHost extends
     });
   }
 
+  public void preExecuteProcedures() throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new RegionServerObserverOperation() {
+      @Override
+      public void call(RegionServerObserver observer) throws IOException {
+        observer.preExecuteProcedures(this);
+      }
+    });
+  }
+
+  public void postExecuteProcedures() throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new RegionServerObserverOperation() {
+      @Override
+      public void call(RegionServerObserver observer) throws IOException {
+        observer.postExecuteProcedures(this);
+      }
+    });
+  }
+
   /**
    * Coprocessor environment extension providing access to region server
    * related services.

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec84d0ef/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 5a3c883..60842be 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
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -117,13 +117,6 @@ 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.security.access.Permission.Action;
-import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
-import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker;
-import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -136,6 +129,14 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
+import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker;
+import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+
 /**
  * Provides basic authorization checks for data access and administrative
  * operations.
@@ -2455,7 +2456,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       throws IOException {
     requirePermission(ctx, "replicateLogEntries", Action.WRITE);
   }
-
+  
   @Override
   public void  preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
           throws IOException {
@@ -2507,8 +2508,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
 
   @Override
   public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
-      TableName tableName, RegionInfo[] regionInfos, String description)
-  throws IOException {
+      TableName tableName, RegionInfo[] regionInfos, String description) throws IOException {
     // There are operations in the CREATE and ADMIN domain which may require lock, READ
     // or WRITE. So for any lock request, we check for these two perms irrespective of lock type.
     String reason = String.format("Description=%s", description);
@@ -2521,12 +2521,18 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
     checkLockPermissions(ctx, null, tableName, null, description);
   }
 
+  @Override
+  public void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
+      throws IOException {
+    checkSystemOrSuperUser(getActiveUser(ctx));
+  }
+
   /**
    * Returns the active user to which authorization checks should be applied.
    * If we are in the context of an RPC call, the remote user is used,
    * otherwise the currently logged in user is used.
    */
-  public User getActiveUser(ObserverContext<?> ctx) throws IOException {
+  private User getActiveUser(ObserverContext<?> ctx) throws IOException {
     // for non-rpc handling, fallback to system user
     Optional<User> optionalUser = ctx.getCaller();
     if (optionalUser.isPresent()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec84d0ef/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
index d09b0a4..6b743b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java
@@ -210,5 +210,12 @@ public class TestJMXConnectorServer {
         throw new AccessDeniedException("Insufficient permissions to shut down cluster.");
       }
     }
+
+    @Override
+    public void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
+        throws IOException {
+      // FIXME: ignore the procedure permission check since in our UT framework master is neither
+      // the systemuser nor the superuser so we can not call executeProcedures...
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec84d0ef/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 99e212d..fe2abe7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -30,7 +30,6 @@ import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
@@ -38,7 +37,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -3064,4 +3062,18 @@ public class TestAccessController extends SecureTestUtil {
     verifyAllowed(
         action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
   }
+
+  @Test
+  public void testExecuteProcedures() throws Exception {
+    AccessTestAction action = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preExecuteProcedures(ObserverContextImpl.createAndPrepare(RSCP_ENV));
+        return null;
+      }
+    };
+
+    verifyAllowed(action, SUPERUSER);
+    verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER, USER_ADMIN);
+  }
 }


[22/48] hbase git commit: HBASE-19580 Use slf4j instead of commons-logging in new, just-added Peer Procedure classes

Posted by zh...@apache.org.
HBASE-19580 Use slf4j instead of commons-logging in new, just-added Peer Procedure classes


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: fa49508a04c026b94f0f10288432441ffaf8baae
Parents: 05468d7
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 21 21:59:46 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/replication/AddPeerProcedure.java      | 6 +++---
 .../hadoop/hbase/master/replication/DisablePeerProcedure.java  | 6 +++---
 .../hadoop/hbase/master/replication/EnablePeerProcedure.java   | 6 +++---
 .../hadoop/hbase/master/replication/ModifyPeerProcedure.java   | 6 +++---
 .../hadoop/hbase/master/replication/RefreshPeerProcedure.java  | 6 +++---
 .../hadoop/hbase/master/replication/RemovePeerProcedure.java   | 6 +++---
 .../hbase/master/replication/UpdatePeerConfigProcedure.java    | 6 +++---
 7 files changed, 21 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fa49508a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index c3862d8..066c3e7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -28,6 +26,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData;
 
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.A
 @InterfaceAudience.Private
 public class AddPeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(AddPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(AddPeerProcedure.class);
 
   private ReplicationPeerConfig peerConfig;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa49508a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
index 0b32db9..9a28de6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for disabling a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class DisablePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(DisablePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(DisablePeerProcedure.class);
 
   public DisablePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa49508a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
index 92ba000..4855901 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for enabling a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class EnablePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(EnablePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(EnablePeerProcedure.class);
 
   public EnablePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa49508a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 23f6f87..c4552ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
@@ -30,6 +28,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyPeerStateData;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
@@ -42,7 +42,7 @@ public abstract class ModifyPeerProcedure
     extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
     implements PeerProcedureInterface {
 
-  private static final Log LOG = LogFactory.getLog(ModifyPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ModifyPeerProcedure.class);
 
   protected String peerId;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa49508a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
index ddc2401..ba4285f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
@@ -35,6 +33,8 @@ import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProced
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.replication.regionserver.RefreshPeerCallable;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.R
 public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
     implements PeerProcedureInterface, RemoteProcedure<MasterProcedureEnv, ServerName> {
 
-  private static final Log LOG = LogFactory.getLog(RefreshPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(RefreshPeerProcedure.class);
 
   private String peerId;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa49508a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index 3daad6d..d40df02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for removing a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class RemovePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(RemovePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(RemovePeerProcedure.class);
 
   public RemovePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa49508a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index 435eefc..d8154dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -19,14 +19,14 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UpdatePeerConfigStateData;
 
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.U
 @InterfaceAudience.Private
 public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(UpdatePeerConfigProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(UpdatePeerConfigProcedure.class);
 
   private ReplicationPeerConfig peerConfig;
 


[40/48] hbase git commit: HBASE-19783 Change replication peer cluster key/endpoint from a not-null value to null is not allowed

Posted by zh...@apache.org.
HBASE-19783 Change replication peer cluster key/endpoint from a not-null value to null is not allowed

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 650a01da72ff2d2c31a8961797e8c2910f3715d4
Parents: 5689da1
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 12 22:04:38 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerManager.java     | 28 +++++++++++++-------
 1 file changed, 19 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/650a01da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index 696b2d7..19fc7f4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -132,20 +132,19 @@ public class ReplicationPeerManager {
     checkPeerConfig(peerConfig);
     ReplicationPeerDescription desc = checkPeerExists(peerId);
     ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
-    if (!StringUtils.isBlank(peerConfig.getClusterKey()) &&
-      !peerConfig.getClusterKey().equals(oldPeerConfig.getClusterKey())) {
+    if (!isStringEquals(peerConfig.getClusterKey(), oldPeerConfig.getClusterKey())) {
       throw new DoNotRetryIOException(
           "Changing the cluster key on an existing peer is not allowed. Existing key '" +
-            oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
-            peerConfig.getClusterKey() + "'");
+              oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
+              peerConfig.getClusterKey() + "'");
     }
 
-    if (!StringUtils.isBlank(peerConfig.getReplicationEndpointImpl()) &&
-      !peerConfig.getReplicationEndpointImpl().equals(oldPeerConfig.getReplicationEndpointImpl())) {
+    if (!isStringEquals(peerConfig.getReplicationEndpointImpl(),
+      oldPeerConfig.getReplicationEndpointImpl())) {
       throw new DoNotRetryIOException("Changing the replication endpoint implementation class " +
-        "on an existing peer is not allowed. Existing class '" +
-        oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
-        " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
+          "on an existing peer is not allowed. Existing class '" +
+          oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
+          " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
     }
   }
 
@@ -341,4 +340,15 @@ public class ReplicationPeerManager {
     return new ReplicationPeerManager(peerStorage,
         ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
   }
+
+  /**
+   * For replication peer cluster key or endpoint class, null and empty string is same. So here
+   * don't use {@link StringUtils#equals(CharSequence, CharSequence)} directly.
+   */
+  private boolean isStringEquals(String s1, String s2) {
+    if (StringUtils.isBlank(s1)) {
+      return StringUtils.isBlank(s2);
+    }
+    return s1.equals(s2);
+  }
 }


[45/48] hbase git commit: HBASE-19711 TestReplicationAdmin.testConcurrentPeerOperations hangs

Posted by zh...@apache.org.
HBASE-19711 TestReplicationAdmin.testConcurrentPeerOperations hangs

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 5c837e042becc624377e42de63b33da07e8a577b
Parents: 01035d1
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 5 15:39:06 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../procedure/MasterProcedureScheduler.java     | 23 ++++++++++++++++----
 1 file changed, 19 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5c837e04/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 4ecb3b1..0400de4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -402,7 +402,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   @Override
   public void completionCleanup(final Procedure proc) {
     if (proc instanceof TableProcedureInterface) {
-      TableProcedureInterface iProcTable = (TableProcedureInterface)proc;
+      TableProcedureInterface iProcTable = (TableProcedureInterface) proc;
       boolean tableDeleted;
       if (proc.hasException()) {
         Exception procEx = proc.getException().unwrapRemoteException();
@@ -423,9 +423,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       }
     } else if (proc instanceof PeerProcedureInterface) {
       PeerProcedureInterface iProcPeer = (PeerProcedureInterface) proc;
-      if (iProcPeer.getPeerOperationType() == PeerOperationType.REMOVE) {
-        removePeerQueue(iProcPeer.getPeerId());
-      }
+      tryCleanupPeerQueue(iProcPeer.getPeerId(), proc);
     } else {
       // No cleanup for ServerProcedureInterface types, yet.
       return;
@@ -514,6 +512,23 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     locking.removePeerLock(peerId);
   }
 
+  private void tryCleanupPeerQueue(String peerId, Procedure procedure) {
+    schedLock();
+    try {
+      PeerQueue queue = AvlTree.get(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR);
+      if (queue == null) {
+        return;
+      }
+
+      final LockAndQueue lock = locking.getPeerLock(peerId);
+      if (queue.isEmpty() && lock.tryExclusiveLock(procedure)) {
+        removeFromRunQueue(peerRunQueue, queue);
+        removePeerQueue(peerId);
+      }
+    } finally {
+      schedUnlock();
+    }
+  }
 
   private static boolean isPeerProcedure(Procedure<?> proc) {
     return proc instanceof PeerProcedureInterface;


[43/48] hbase git commit: HBASE-19748 TestRegionReplicaFailover and TestRegionReplicaReplicationEndpoint UT hangs

Posted by zh...@apache.org.
HBASE-19748 TestRegionReplicaFailover and TestRegionReplicaReplicationEndpoint UT hangs


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 5689da15558a65c7b83bd2563795f7d7f004ea80
Parents: 8887a12
Author: huzheng <op...@gmail.com>
Authored: Wed Jan 10 15:00:30 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/master/HMaster.java   | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5689da15/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 58a0f4b..48c023d 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
@@ -38,7 +38,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
-import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -70,6 +69,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
@@ -3362,13 +3362,12 @@ public class HMaster extends HRegionServer implements MasterServices {
       cpHost.preGetReplicationPeerConfig(peerId);
     }
     LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId);
-    Optional<ReplicationPeerConfig> peerConfig =
-      this.replicationPeerManager.getPeerConfig(peerId);
-
+    ReplicationPeerConfig peerConfig = this.replicationPeerManager.getPeerConfig(peerId)
+        .orElseThrow(() -> new ReplicationPeerNotFoundException(peerId));
     if (cpHost != null) {
       cpHost.postGetReplicationPeerConfig(peerId);
     }
-    return peerConfig.orElse(null);
+    return peerConfig;
   }
 
   @Override


[02/48] hbase git commit: HBASE-19775 hbase shell doesn't handle the exceptions that are wrapped in java.io.UncheckedIOException

Posted by zh...@apache.org.
HBASE-19775 hbase shell doesn't handle the exceptions that are wrapped in java.io.UncheckedIOException

Signed-off-by: Josh Elser <el...@apache.org>
Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: b979487cace818e59945d65bb37db399e777ab72
Parents: f7fbc16
Author: Sergey Soldatov <ss...@apache.org>
Authored: Thu Jan 11 16:02:40 2018 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Fri Jan 12 12:38:13 2018 -0500

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/shell/commands.rb | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b979487c/hbase-shell/src/main/ruby/shell/commands.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands.rb b/hbase-shell/src/main/ruby/shell/commands.rb
index 1b8de9e..990e978 100644
--- a/hbase-shell/src/main/ruby/shell/commands.rb
+++ b/hbase-shell/src/main/ruby/shell/commands.rb
@@ -108,6 +108,7 @@ module Shell
         yield
       rescue => cause
         # let individual command handle exceptions first
+        cause = cause.getCause if cause.is_a? java.io.UncheckedIOException
         handle_exceptions(cause, *args) if respond_to?(:handle_exceptions)
         # Global HBase exception handling below if not handled by respective command above
         if cause.is_a?(org.apache.hadoop.hbase.TableNotFoundException)


[23/48] hbase git commit: HBASE-19520 Add UTs for the new lock type PEER

Posted by zh...@apache.org.
HBASE-19520 Add UTs for the new lock type PEER

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 05468d7a0a4ff24ec439b3939cee97545282e3fe
Parents: d9551d8
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Dec 20 16:43:38 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../procedure/MasterProcedureScheduler.java     |   9 +-
 .../procedure/TestMasterProcedureScheduler.java |  65 ++++++++-
 ...TestMasterProcedureSchedulerConcurrency.java | 135 +++++++++++++++++++
 3 files changed, 201 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/05468d7a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 8ff2d12..a25217c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -389,6 +389,13 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     while (tableIter.hasNext()) {
       count += tableIter.next().size();
     }
+
+    // Peer queues
+    final AvlTreeIterator<PeerQueue> peerIter = new AvlTreeIterator<>(peerMap);
+    while (peerIter.hasNext()) {
+      count += peerIter.next().size();
+    }
+
     return count;
   }
 
@@ -1041,7 +1048,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @see #wakePeerExclusiveLock(Procedure, String)
    * @param procedure the procedure trying to acquire the lock
    * @param peerId peer to lock
-   * @return true if the procedure has to wait for the per to be available
+   * @return true if the procedure has to wait for the peer to be available
    */
   public boolean waitPeerExclusiveLock(Procedure<?> procedure, String peerId) {
     schedLock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/05468d7a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
index 0291165..fd77e1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
@@ -905,6 +905,27 @@ public class TestMasterProcedureScheduler {
     }
   }
 
+  public static class TestPeerProcedure extends TestProcedure implements PeerProcedureInterface {
+    private final String peerId;
+    private final PeerOperationType opType;
+
+    public TestPeerProcedure(long procId, String peerId, PeerOperationType opType) {
+      super(procId);
+      this.peerId = peerId;
+      this.opType = opType;
+    }
+
+    @Override
+    public String getPeerId() {
+      return peerId;
+    }
+
+    @Override
+    public PeerOperationType getPeerOperationType() {
+      return opType;
+    }
+  }
+
   private static LockProcedure createLockProcedure(LockType lockType, long procId) throws Exception {
     LockProcedure procedure = new LockProcedure();
 
@@ -927,22 +948,19 @@ public class TestMasterProcedureScheduler {
     return createLockProcedure(LockType.SHARED, procId);
   }
 
-  private static void assertLockResource(LockedResource resource,
-      LockedResourceType resourceType, String resourceName)
-  {
+  private static void assertLockResource(LockedResource resource, LockedResourceType resourceType,
+      String resourceName) {
     assertEquals(resourceType, resource.getResourceType());
     assertEquals(resourceName, resource.getResourceName());
   }
 
-  private static void assertExclusiveLock(LockedResource resource, Procedure<?> procedure)
-  {
+  private static void assertExclusiveLock(LockedResource resource, Procedure<?> procedure) {
     assertEquals(LockType.EXCLUSIVE, resource.getLockType());
     assertEquals(procedure, resource.getExclusiveLockOwnerProcedure());
     assertEquals(0, resource.getSharedLockCount());
   }
 
-  private static void assertSharedLock(LockedResource resource, int lockCount)
-  {
+  private static void assertSharedLock(LockedResource resource, int lockCount) {
     assertEquals(LockType.SHARED, resource.getLockType());
     assertEquals(lockCount, resource.getSharedLockCount());
   }
@@ -1027,6 +1045,39 @@ public class TestMasterProcedureScheduler {
   }
 
   @Test
+  public void testListLocksPeer() throws Exception {
+    String peerId = "1";
+    LockProcedure procedure = createExclusiveLockProcedure(4);
+    queue.waitPeerExclusiveLock(procedure, peerId);
+
+    List<LockedResource> locks = queue.getLocks();
+    assertEquals(1, locks.size());
+
+    LockedResource resource = locks.get(0);
+    assertLockResource(resource, LockedResourceType.PEER, peerId);
+    assertExclusiveLock(resource, procedure);
+    assertTrue(resource.getWaitingProcedures().isEmpty());
+
+    // Try to acquire the exclusive lock again with same procedure
+    assertFalse(queue.waitPeerExclusiveLock(procedure, peerId));
+
+    // Try to acquire the exclusive lock again with new procedure
+    LockProcedure procedure2 = createExclusiveLockProcedure(5);
+    assertTrue(queue.waitPeerExclusiveLock(procedure2, peerId));
+
+    // Same peerId, still only has 1 LockedResource
+    locks = queue.getLocks();
+    assertEquals(1, locks.size());
+
+    resource = locks.get(0);
+    assertLockResource(resource, LockedResourceType.PEER, peerId);
+    // LockedResource owner still is the origin procedure
+    assertExclusiveLock(resource, procedure);
+    // The new procedure should in the waiting list
+    assertEquals(1, resource.getWaitingProcedures().size());
+  }
+
+  @Test
   public void testListLocksWaiting() throws Exception {
     LockProcedure procedure1 = createExclusiveLockProcedure(1);
     queue.waitTableExclusiveLock(procedure1, TableName.valueOf("ns4", "table4"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/05468d7a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
index 2e8e52a..4e67a63 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
@@ -26,6 +26,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface.PeerOperationType;
+import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestPeerProcedure;
 import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestTableProcedure;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -63,6 +65,85 @@ public class TestMasterProcedureSchedulerConcurrency {
     queue.clear();
   }
 
+  @Test
+  public void testConcurrentPeerOperations() throws Exception {
+    TestPeerProcedureSet procSet = new TestPeerProcedureSet(queue);
+
+    int NUM_ITEMS = 10;
+    int NUM_PEERS = 5;
+    AtomicInteger opsCount = new AtomicInteger(0);
+    for (int i = 0; i < NUM_PEERS; ++i) {
+      String peerId = String.format("test-peer-%04d", i);
+      for (int j = 1; j < NUM_ITEMS; ++j) {
+        procSet.addBack(new TestPeerProcedure(i * 100 + j, peerId, PeerOperationType.ADD));
+        opsCount.incrementAndGet();
+      }
+    }
+    assertEquals(opsCount.get(), queue.size());
+
+    Thread[] threads = new Thread[NUM_PEERS * 2];
+    HashSet<String> concurrentPeers = new HashSet<>();
+    ArrayList<String> failures = new ArrayList<>();
+    AtomicInteger concurrentCount = new AtomicInteger(0);
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          while (opsCount.get() > 0) {
+            try {
+              TestPeerProcedure proc = procSet.acquire();
+              if (proc == null) {
+                queue.signalAll();
+                if (opsCount.get() > 0) {
+                  continue;
+                }
+                break;
+              }
+
+              String peerId = proc.getPeerId();
+              synchronized (concurrentPeers) {
+                assertTrue("unexpected concurrency on " + peerId, concurrentPeers.add(peerId));
+              }
+              assertTrue(opsCount.decrementAndGet() >= 0);
+
+              try {
+                long procId = proc.getProcId();
+                int concurrent = concurrentCount.incrementAndGet();
+                assertTrue("inc-concurrent="+ concurrent +" 1 <= concurrent <= "+ NUM_PEERS,
+                  concurrent >= 1 && concurrent <= NUM_PEERS);
+                LOG.debug("[S] peerId="+ peerId +" procId="+ procId +" concurrent="+ concurrent);
+                Thread.sleep(2000);
+                concurrent = concurrentCount.decrementAndGet();
+                LOG.debug("[E] peerId="+ peerId +" procId="+ procId +" concurrent="+ concurrent);
+                assertTrue("dec-concurrent=" + concurrent, concurrent < NUM_PEERS);
+              } finally {
+                synchronized (concurrentPeers) {
+                  assertTrue(concurrentPeers.remove(peerId));
+                }
+                procSet.release(proc);
+              }
+            } catch (Throwable e) {
+              LOG.error("Failed " + e.getMessage(), e);
+              synchronized (failures) {
+                failures.add(e.getMessage());
+              }
+            } finally {
+              queue.signalAll();
+            }
+          }
+        }
+      };
+      threads[i].start();
+    }
+
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i].join();
+    }
+    assertTrue(failures.toString(), failures.isEmpty());
+    assertEquals(0, opsCount.get());
+    assertEquals(0, queue.size());
+  }
+
   /**
    * Verify that "write" operations for a single table are serialized,
    * but different tables can be executed in parallel.
@@ -221,4 +302,58 @@ public class TestMasterProcedureSchedulerConcurrency {
       return ((TableProcedureInterface)proc).getTableOperationType();
     }
   }
+
+  public static class TestPeerProcedureSet {
+    private final MasterProcedureScheduler queue;
+
+    public TestPeerProcedureSet(final MasterProcedureScheduler queue) {
+      this.queue = queue;
+    }
+
+    public void addBack(TestPeerProcedure proc) {
+      queue.addBack(proc);
+    }
+
+    public TestPeerProcedure acquire() {
+      TestPeerProcedure proc = null;
+      boolean waiting = true;
+      while (waiting && queue.size() > 0) {
+        proc = (TestPeerProcedure) queue.poll(100000000L);
+        if (proc == null) {
+          continue;
+        }
+        switch (proc.getPeerOperationType()) {
+          case ADD:
+          case REMOVE:
+          case ENABLE:
+          case DISABLE:
+          case UPDATE_CONFIG:
+            waiting = queue.waitPeerExclusiveLock(proc, proc.getPeerId());
+            break;
+          case REFRESH:
+            waiting = false;
+            break;
+          default:
+            throw new UnsupportedOperationException();
+        }
+      }
+      return proc;
+    }
+
+    public void release(TestPeerProcedure proc) {
+      switch (proc.getPeerOperationType()) {
+        case ADD:
+        case REMOVE:
+        case ENABLE:
+        case DISABLE:
+        case UPDATE_CONFIG:
+          queue.wakePeerExclusiveLock(proc, proc.getPeerId());
+          break;
+        case REFRESH:
+          break;
+        default:
+          throw new UnsupportedOperationException();
+      }
+    }
+  }
 }


[05/48] hbase git commit: HBASE-19694 The initialization order for a fresh cluster is incorrect Addendum. Adds in a missing line that helps debugging.

Posted by zh...@apache.org.
HBASE-19694 The initialization order for a fresh cluster is incorrect Addendum. Adds in a missing line that helps debugging.


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: d8271b03615c4fff23356c96e3c983a0cfffaf55
Parents: 9ed52ee
Author: Michael Stack <st...@apache.org>
Authored: Thu Jan 11 17:47:00 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Jan 12 13:08:27 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/util/JVMClusterUtil.java     | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d8271b03/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
index 00410af..1ac790f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
@@ -187,7 +187,9 @@ public class JVMClusterUtil {
       int startTimeout = configuration != null ? Integer.parseInt(
         configuration.get("hbase.master.start.timeout.localHBaseCluster", "30000")) : 30000;
       if (System.currentTimeMillis() > startTime + startTimeout) {
-        throw new RuntimeException(String.format("Master not active after %s seconds", startTimeout));
+        String msg = "Master not active after " + startTimeout + "ms";
+        Threads.printThreadInfo(System.out, "Thread dump because: " + msg);
+        throw new RuntimeException(msg);
       }
     }
 
@@ -216,8 +218,7 @@ public class JVMClusterUtil {
       }
       if (System.currentTimeMillis() > startTime + maxwait) {
         String msg = "Master not initialized after " + maxwait + "ms seconds";
-        Threads.printThreadInfo(System.out,
-          "Thread dump because: " + msg);
+        Threads.printThreadInfo(System.out, "Thread dump because: " + msg);
         throw new RuntimeException(msg);
       }
       try {


[03/48] hbase git commit: HBASE-19769 Remove ZK metrics because of classloader issues

Posted by zh...@apache.org.
HBASE-19769 Remove ZK metrics because of classloader issues

When we run MapReduce jobs via `yarn jar`, the special classloader
which is set up by YARN creates a situation where our invocation of
package-private Hadoop classes throws an IllegalAccessError. It's
easiest to just remove these and rethink how to avoid further
Hadoop metrics2 issues.

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: b662080734abfa3a11486147d365eb682e0b4ee7
Parents: b979487
Author: Josh Elser <el...@apache.org>
Authored: Thu Jan 11 18:14:32 2018 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Jan 12 12:40:47 2018 -0500

----------------------------------------------------------------------
 .../hbase/zookeeper/RecoverableZooKeeper.java   |  78 --------------
 .../hadoop/hbase/zookeeper/ZKMetrics.java       | 108 -------------------
 .../hbase/zookeeper/ZKMetricsListener.java      |  91 ----------------
 .../hadoop/hbase/zookeeper/TestZKMetrics.java   |  80 --------------
 4 files changed, 357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b6620807/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
index 4c76a5c..5df31b9 100644
--- 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
@@ -82,7 +82,6 @@ public class RecoverableZooKeeper {
   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)
@@ -112,7 +111,6 @@ public class RecoverableZooKeeper {
     this.watcher = watcher;
     this.sessionTimeout = sessionTimeout;
     this.quorumServers = quorumServers;
-    this.metrics = new ZKMetrics();
 
     try {
       checkZk();
@@ -166,11 +164,8 @@ public class RecoverableZooKeeper {
         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) {
@@ -182,11 +177,9 @@ public class RecoverableZooKeeper {
               throw e;
 
             case CONNECTIONLOSS:
-              this.metrics.registerConnectionLossException();
               retryOrThrow(retryCounter, e, "delete");
               break;
             case OPERATIONTIMEOUT:
-              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "delete");
               break;
 
@@ -211,18 +204,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -246,18 +234,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -293,18 +276,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -329,18 +307,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -365,18 +338,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -401,18 +369,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -442,18 +405,13 @@ public class RecoverableZooKeeper {
         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:
@@ -463,14 +421,11 @@ public class RecoverableZooKeeper {
                   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;
                 }
@@ -498,18 +453,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -534,18 +484,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -603,11 +548,8 @@ public class RecoverableZooKeeper {
       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) {
@@ -616,8 +558,6 @@ public class RecoverableZooKeeper {
               // 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
@@ -632,11 +572,9 @@ public class RecoverableZooKeeper {
             throw e;
 
           case CONNECTIONLOSS:
-            this.metrics.registerConnectionLossException();
             retryOrThrow(retryCounter, e, "create");
             break;
           case OPERATIONTIMEOUT:
-            this.metrics.registerOperationTimeoutException();
             retryOrThrow(retryCounter, e, "create");
             break;
 
@@ -667,18 +605,13 @@ public class RecoverableZooKeeper {
         first = false;
         long startTime = EnvironmentEdgeManager.currentTime();
         String nodePath = checkZk().create(newPath, data, acl, createMode);
-        this.metrics.registerWriteOperationLatency(
-                Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
         return nodePath;
       } 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;
 
@@ -730,18 +663,13 @@ public class RecoverableZooKeeper {
         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;
 
@@ -762,15 +690,11 @@ public class RecoverableZooKeeper {
     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;
       }
@@ -803,8 +727,6 @@ public class RecoverableZooKeeper {
   public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException {
     long startTime = EnvironmentEdgeManager.currentTime();
     checkZk().sync(path, cb, null);
-    this.metrics.registerSyncOperationLatency(
-            Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/b6620807/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
deleted file mode 100644
index c660417..0000000
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetrics.java
+++ /dev/null
@@ -1,108 +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.hbase.thirdparty.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/b6620807/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
deleted file mode 100644
index f17925e..0000000
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMetricsListener.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 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/b6620807/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
deleted file mode 100644
index e43a5c8..0000000
--- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMetrics.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 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.apache.hadoop.hbase.testclassification.ZKTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ ZKTests.class, 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);
-  }
-}


[16/48] hbase git commit: HBASE-19543 Abstract a replication storage interface to extract the zk specific code

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/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
deleted file mode 100644
index b6f8784..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
+++ /dev/null
@@ -1,199 +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.master.replication;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
-import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-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.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Manages and performs all replication admin operations.
- * <p>
- * Used to add/remove a replication peer.
- */
-@InterfaceAudience.Private
-public class ReplicationManager {
-  private final ReplicationQueuesClient replicationQueuesClient;
-  private final ReplicationPeers replicationPeers;
-
-  public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable)
-      throws IOException {
-    try {
-      this.replicationQueuesClient = ReplicationFactory
-          .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw));
-      this.replicationQueuesClient.init();
-      this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf,
-        this.replicationQueuesClient, abortable);
-      this.replicationPeers.init();
-    } catch (Exception e) {
-      throw new IOException("Failed to construct ReplicationManager", e);
-    }
-  }
-
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException {
-    checkPeerConfig(peerConfig);
-    replicationPeers.registerPeer(peerId, peerConfig, enabled);
-    replicationPeers.peerConnected(peerId);
-  }
-
-  public void removeReplicationPeer(String peerId) throws ReplicationException {
-    replicationPeers.peerDisconnected(peerId);
-    replicationPeers.unregisterPeer(peerId);
-  }
-
-  public void enableReplicationPeer(String peerId) throws ReplicationException {
-    this.replicationPeers.enablePeer(peerId);
-  }
-
-  public void disableReplicationPeer(String peerId) throws ReplicationException {
-    this.replicationPeers.disablePeer(peerId);
-  }
-
-  public ReplicationPeerConfig getPeerConfig(String peerId)
-      throws ReplicationException, ReplicationPeerNotFoundException {
-    ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(peerId);
-    if (peerConfig == null) {
-      throw new ReplicationPeerNotFoundException(peerId);
-    }
-    return peerConfig;
-  }
-
-  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
-      throws ReplicationException, IOException {
-    checkPeerConfig(peerConfig);
-    this.replicationPeers.updatePeerConfig(peerId, peerConfig);
-  }
-
-  public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern)
-      throws ReplicationException {
-    List<ReplicationPeerDescription> peers = new ArrayList<>();
-    List<String> peerIds = replicationPeers.getAllPeerIds();
-    for (String peerId : peerIds) {
-      if (pattern == null || (pattern != null && pattern.matcher(peerId).matches())) {
-        peers.add(new ReplicationPeerDescription(peerId,
-            replicationPeers.getStatusOfPeerFromBackingStore(peerId),
-            replicationPeers.getReplicationPeerConfig(peerId)));
-      }
-    }
-    return peers;
-  }
-
-  /**
-   * If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
-   * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to
-   * peer cluster.
-   *
-   * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
-   * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
-   */
-  private void checkPeerConfig(ReplicationPeerConfig peerConfig) {
-    if (peerConfig.replicateAllUserTables()) {
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
-        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new IllegalArgumentException("Need clean namespaces or table-cfs config firstly " +
-          "when you want replicate all cluster");
-      }
-      checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
-        peerConfig.getExcludeTableCFsMap());
-    } else {
-      if ((peerConfig.getExcludeNamespaces() != null
-          && !peerConfig.getExcludeNamespaces().isEmpty())
-          || (peerConfig.getExcludeTableCFsMap() != null
-              && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
-        throw new IllegalArgumentException(
-            "Need clean exclude-namespaces or exclude-table-cfs config firstly"
-                + " when replicate_all flag is false");
-      }
-      checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
-        peerConfig.getTableCFsMap());
-    }
-    checkConfiguredWALEntryFilters(peerConfig);
-  }
-
-  /**
-   * Set a namespace in the peer config means that all tables in this namespace will be replicated
-   * to the peer cluster.
-   * <ol>
-   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
-   * the peer config.</li>
-   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
-   * config.</li>
-   * </ol>
-   * <p>
-   * Set a exclude namespace in the peer config means that all tables in this namespace can't be
-   * replicated to the peer cluster.
-   * <ol>
-   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
-   * this namespace to the peer config.</li>
-   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
-   * exclude namespace.</li>
-   * </ol>
-   */
-  private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
-      Map<TableName, ? extends Collection<String>> tableCfs) {
-    if (namespaces == null || namespaces.isEmpty()) {
-      return;
-    }
-    if (tableCfs == null || tableCfs.isEmpty()) {
-      return;
-    }
-    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
-      TableName table = entry.getKey();
-      if (namespaces.contains(table.getNamespaceAsString())) {
-        throw new IllegalArgumentException("Table-cfs " + table + " is conflict with namespaces "
-            + table.getNamespaceAsString() + " in peer config");
-      }
-    }
-  }
-
-  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) {
-    String filterCSV = peerConfig.getConfiguration()
-        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
-    if (filterCSV != null && !filterCSV.isEmpty()) {
-      String[] filters = filterCSV.split(",");
-      for (String filter : filters) {
-        try {
-          Class.forName(filter).newInstance();
-        } catch (Exception e) {
-          throw new IllegalArgumentException("Configured WALEntryFilter " + filter +
-              " could not be created. Failing add/update " + "peer operation.", e);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
new file mode 100644
index 0000000..5abd874
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -0,0 +1,331 @@
+/**
+ * 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.replication;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
+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.replication.ReplicationPeerStorage;
+import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Manages and performs all replication admin operations.
+ * <p>
+ * Used to add/remove a replication peer.
+ */
+@InterfaceAudience.Private
+public final class ReplicationPeerManager {
+
+  private final ReplicationPeerStorage peerStorage;
+
+  private final ReplicationQueueStorage queueStorage;
+
+  private final ConcurrentMap<String, ReplicationPeerDescription> peers;
+
+  private ReplicationPeerManager(ReplicationPeerStorage peerStorage,
+      ReplicationQueueStorage queueStorage,
+      ConcurrentMap<String, ReplicationPeerDescription> peers) {
+    this.peerStorage = peerStorage;
+    this.queueStorage = queueStorage;
+    this.peers = peers;
+  }
+
+  private void checkQueuesDeleted(String peerId)
+      throws ReplicationException, DoNotRetryIOException {
+    for (ServerName replicator : queueStorage.getListOfReplicators()) {
+      List<String> queueIds = queueStorage.getAllQueues(replicator);
+      for (String queueId : queueIds) {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        if (queueInfo.getPeerId().equals(peerId)) {
+          throw new DoNotRetryIOException("undeleted queue for peerId: " + peerId +
+            ", replicator: " + replicator + ", queueId: " + queueId);
+        }
+      }
+    }
+    if (queueStorage.getAllPeersFromHFileRefsQueue().contains(peerId)) {
+      throw new DoNotRetryIOException("Undeleted queue for peer " + peerId + " in hfile-refs");
+    }
+  }
+
+  public void preAddPeer(String peerId, ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException, ReplicationException {
+    if (peerId.contains("-")) {
+      throw new DoNotRetryIOException("Found invalid peer name: " + peerId);
+    }
+    checkPeerConfig(peerConfig);
+    if (peers.containsKey(peerId)) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " already exists");
+    }
+    // make sure that there is no queues with the same peer id. This may happen when we create a
+    // peer with the same id with a old deleted peer. If the replication queues for the old peer
+    // have not been cleaned up yet then we should not create the new peer, otherwise the old wal
+    // file may also be replicated.
+    checkQueuesDeleted(peerId);
+  }
+
+  private ReplicationPeerDescription checkPeerExists(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    if (desc == null) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " does not exist");
+    }
+    return desc;
+  }
+
+  public void preRemovePeer(String peerId) throws DoNotRetryIOException {
+    checkPeerExists(peerId);
+  }
+
+  public void preEnablePeer(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    if (desc.isEnabled()) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " has already been enabled");
+    }
+  }
+
+  public void preDisablePeer(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    if (!desc.isEnabled()) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " has already been disabled");
+    }
+  }
+
+  public void preUpdatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    checkPeerConfig(peerConfig);
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
+    if (!StringUtils.isBlank(peerConfig.getClusterKey()) &&
+      !peerConfig.getClusterKey().equals(oldPeerConfig.getClusterKey())) {
+      throw new DoNotRetryIOException(
+          "Changing the cluster key on an existing peer is not allowed. Existing key '" +
+            oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
+            peerConfig.getClusterKey() + "'");
+    }
+
+    if (!StringUtils.isBlank(peerConfig.getReplicationEndpointImpl()) &&
+      !peerConfig.getReplicationEndpointImpl().equals(oldPeerConfig.getReplicationEndpointImpl())) {
+      throw new DoNotRetryIOException("Changing the replication endpoint implementation class " +
+        "on an existing peer is not allowed. Existing class '" +
+        oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
+        " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
+    }
+  }
+
+  private ReplicationPeerConfig copy(ReplicationPeerConfig peerConfig) {
+    ReplicationPeerConfig copiedPeerConfig = new ReplicationPeerConfig();
+    copiedPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
+    copiedPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
+    copiedPeerConfig.setTableCFsMap(peerConfig.getTableCFsMap());
+    copiedPeerConfig.setNamespaces(peerConfig.getNamespaces());
+    copiedPeerConfig.setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap());
+    copiedPeerConfig.setExcludeNamespaces(peerConfig.getExcludeNamespaces());
+    copiedPeerConfig.setBandwidth(peerConfig.getBandwidth());
+    copiedPeerConfig.setReplicateAllUserTables(peerConfig.replicateAllUserTables());
+    copiedPeerConfig.setClusterKey(peerConfig.getClusterKey());
+    copiedPeerConfig.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
+    return copiedPeerConfig;
+  }
+
+  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException {
+    if (peers.containsKey(peerId)) {
+      // this should be a retry, just return
+      return;
+    }
+    ReplicationPeerConfig copiedPeerConfig = copy(peerConfig);
+    peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
+  }
+
+  public void removePeer(String peerId) throws ReplicationException {
+    if (!peers.containsKey(peerId)) {
+      // this should be a retry, just return
+      return;
+    }
+    peerStorage.removePeer(peerId);
+    peers.remove(peerId);
+  }
+
+  private void setPeerState(String peerId, boolean enabled) throws ReplicationException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    if (desc.isEnabled() == enabled) {
+      // this should be a retry, just return
+      return;
+    }
+    peerStorage.setPeerState(peerId, enabled);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig()));
+  }
+
+  public void enablePeer(String peerId) throws ReplicationException {
+    setPeerState(peerId, true);
+  }
+
+  public void disablePeer(String peerId) throws ReplicationException {
+    setPeerState(peerId, false);
+  }
+
+  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException {
+    // the checking rules are too complicated here so we give up checking whether this is a retry.
+    ReplicationPeerDescription desc = peers.get(peerId);
+    ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
+    ReplicationPeerConfig newPeerConfig = copy(peerConfig);
+    // we need to use the new conf to overwrite the old one.
+    newPeerConfig.getConfiguration().putAll(oldPeerConfig.getConfiguration());
+    newPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
+    newPeerConfig.getPeerData().putAll(oldPeerConfig.getPeerData());
+    newPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
+
+    peerStorage.updatePeerConfig(peerId, newPeerConfig);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
+  }
+
+  public List<ReplicationPeerDescription> listPeers(Pattern pattern) {
+    if (pattern == null) {
+      return new ArrayList<>(peers.values());
+    }
+    return peers.values().stream().filter(r -> pattern.matcher(r.getPeerId()).matches())
+        .collect(Collectors.toList());
+  }
+
+  public Optional<ReplicationPeerConfig> getPeerConfig(String peerId) {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
+  }
+
+  /**
+   * If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
+   * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer
+   * cluster.
+   * <p>
+   * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
+   * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
+   */
+  private static void checkPeerConfig(ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    if (peerConfig.replicateAllUserTables()) {
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
+        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " +
+          "when you want replicate all cluster");
+      }
+      checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
+        peerConfig.getExcludeTableCFsMap());
+    } else {
+      if ((peerConfig.getExcludeNamespaces() != null &&
+        !peerConfig.getExcludeNamespaces().isEmpty()) ||
+        (peerConfig.getExcludeTableCFsMap() != null &&
+          !peerConfig.getExcludeTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException(
+            "Need clean exclude-namespaces or exclude-table-cfs config firstly" +
+              " when replicate_all flag is false");
+      }
+      checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
+        peerConfig.getTableCFsMap());
+    }
+    checkConfiguredWALEntryFilters(peerConfig);
+  }
+
+  /**
+   * Set a namespace in the peer config means that all tables in this namespace will be replicated
+   * to the peer cluster.
+   * <ol>
+   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
+   * the peer config.</li>
+   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
+   * config.</li>
+   * </ol>
+   * <p>
+   * Set a exclude namespace in the peer config means that all tables in this namespace can't be
+   * replicated to the peer cluster.
+   * <ol>
+   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
+   * this namespace to the peer config.</li>
+   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
+   * exclude namespace.</li>
+   * </ol>
+   */
+  private static void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
+      Map<TableName, ? extends Collection<String>> tableCfs) throws DoNotRetryIOException {
+    if (namespaces == null || namespaces.isEmpty()) {
+      return;
+    }
+    if (tableCfs == null || tableCfs.isEmpty()) {
+      return;
+    }
+    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
+      TableName table = entry.getKey();
+      if (namespaces.contains(table.getNamespaceAsString())) {
+        throw new DoNotRetryIOException("Table-cfs " + table + " is conflict with namespaces " +
+          table.getNamespaceAsString() + " in peer config");
+      }
+    }
+  }
+
+  private static void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    String filterCSV = peerConfig.getConfiguration()
+        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
+    if (filterCSV != null && !filterCSV.isEmpty()) {
+      String[] filters = filterCSV.split(",");
+      for (String filter : filters) {
+        try {
+          Class.forName(filter).newInstance();
+        } catch (Exception e) {
+          throw new DoNotRetryIOException("Configured WALEntryFilter " + filter +
+            " could not be created. Failing add/update " + "peer operation.", e);
+        }
+      }
+    }
+  }
+
+  public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
+      throws ReplicationException {
+    ReplicationPeerStorage peerStorage =
+      ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
+    ConcurrentMap<String, ReplicationPeerDescription> peers = new ConcurrentHashMap<>();
+    for (String peerId : peerStorage.listPeerIds()) {
+      Optional<ReplicationPeerConfig> peerConfig = peerStorage.getPeerConfig(peerId);
+      boolean enabled = peerStorage.isPeerEnabled(peerId);
+      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig.get()));
+    }
+    return new ReplicationPeerManager(peerStorage,
+        ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index d8154dc..a43532d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -59,12 +60,12 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
     }
+    env.getReplicationPeerManager().preUpdatePeerConfig(peerId, peerConfig);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception {
-    env.getReplicationManager().updatePeerConfig(peerId, peerConfig);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().updatePeerConfig(peerId, peerConfig);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/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 aac57dc..d612eab 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
@@ -17,6 +17,12 @@
  */
 package org.apache.hadoop.hbase.client.replication;
 
+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 static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -26,25 +32,23 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 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.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterReplicationEndpointForTest;
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -52,15 +56,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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 static org.junit.Assert.fail;
 
 /**
  * Unit testing of ReplicationAdmin
@@ -68,8 +63,6 @@ import static org.junit.Assert.fail;
 @Category({MediumTests.class, ClientTests.class})
 public class TestReplicationAdmin {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestReplicationAdmin.class);
   private final static HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
@@ -104,16 +97,17 @@ public class TestReplicationAdmin {
   }
 
   @After
-  public void cleanupPeer() {
-    try {
-      hbaseAdmin.removeReplicationPeer(ID_ONE);
-    } catch (Exception e) {
-      LOG.debug("Replication peer " + ID_ONE + " may already be removed");
+  public void tearDown() throws Exception {
+    for (ReplicationPeerDescription desc : hbaseAdmin.listReplicationPeers()) {
+      hbaseAdmin.removeReplicationPeer(desc.getPeerId());
     }
-    try {
-      hbaseAdmin.removeReplicationPeer(ID_SECOND);
-    } catch (Exception e) {
-      LOG.debug("Replication peer " + ID_SECOND + " may already be removed");
+    ReplicationQueueStorage queueStorage = ReplicationStorageFactory
+        .getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), TEST_UTIL.getConfiguration());
+    for (ServerName serverName : queueStorage.getListOfReplicators()) {
+      for (String queue : queueStorage.getAllQueues(serverName)) {
+        queueStorage.removeQueue(serverName, queue);
+      }
+      queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
     }
   }
 
@@ -203,32 +197,29 @@ public class TestReplicationAdmin {
     ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
     rpc2.setClusterKey(KEY_SECOND);
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZKWatcher zkw = new ZKWatcher(conf, "Test HBaseAdmin", null);
-    ReplicationQueues repQueues =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null, zkw));
-    repQueues.init("server1");
+    ReplicationQueueStorage queueStorage =
+      ReplicationStorageFactory.getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), conf);
 
+    ServerName serverName = ServerName.valueOf("server1", 8000, 1234);
     // add queue for ID_ONE
-    repQueues.addLog(ID_ONE, "file1");
+    queueStorage.addWAL(serverName, ID_ONE, "file1");
     try {
       admin.addPeer(ID_ONE, rpc1, null);
       fail();
     } catch (Exception e) {
       // OK!
     }
-    repQueues.removeQueue(ID_ONE);
-    assertEquals(0, repQueues.getAllQueues().size());
+    queueStorage.removeQueue(serverName, ID_ONE);
+    assertEquals(0, queueStorage.getAllQueues(serverName).size());
 
     // add recovered queue for ID_ONE
-    repQueues.addLog(ID_ONE + "-server2", "file1");
+    queueStorage.addWAL(serverName, ID_ONE + "-server2", "file1");
     try {
       admin.addPeer(ID_ONE, rpc2, null);
       fail();
     } catch (Exception e) {
       // OK!
     }
-    repQueues.removeAllQueues();
-    zkw.close();
   }
 
   /**
@@ -424,7 +415,7 @@ public class TestReplicationAdmin {
       tableCFs.clear();
       tableCFs.put(tableName2, null);
       admin.removePeerTableCFs(ID_ONE, tableCFs);
-      assertTrue(false);
+      fail();
     } catch (ReplicationException e) {
     }
     tableCFs.clear();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/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 8442530..7196b7c 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
@@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.master;
 
 import static org.mockito.Mockito.mock;
 
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.List;
 
@@ -42,7 +44,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -56,8 +58,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
-import com.google.protobuf.Service;
-
 public class MockNoopMasterServices implements MasterServices, Server {
   private final Configuration conf;
   private final MetricsMaster metricsMaster;
@@ -462,7 +462,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return null;
   }
 
@@ -477,7 +477,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public ReplicationManager getReplicationManager() {
+  public ReplicationPeerManager getReplicationPeerManager() {
     return null;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/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 fd44c89..f5d36bf 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
@@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -270,7 +272,7 @@ public class TestMasterNoCluster {
 
       @Override
       void initializeZKBasedSystemTrackers() throws IOException, InterruptedException,
-          KeeperException, CoordinatedStateException {
+          KeeperException, CoordinatedStateException, ReplicationException {
         super.initializeZKBasedSystemTrackers();
         // Record a newer server in server manager at first
         getServerManager().recordNewServerWithLock(newServer,

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef8c27c3/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
index 1675496..24bb4d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -47,9 +46,6 @@ public class TestReplicationDisableInactivePeer extends TestReplicationBase {
    */
   @Test(timeout = 600000)
   public void testDisableInactivePeer() throws Exception {
-
-    // enabling and shutdown the peer
-    admin.enablePeer("2");
     utility2.shutdownMiniHBaseCluster();
 
     byte[] rowkey = Bytes.toBytes("disable inactive peer");


[39/48] hbase git commit: HBASE-19636 All rs should already start work with the new peer change when replication peer procedure is finished

Posted by zh...@apache.org.
HBASE-19636 All rs should already start work with the new peer change when replication peer procedure is finished

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 1f0bfe01a3c71182bfa907ee9b8c6809fe417020
Parents: ec84d0e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Jan 4 16:58:01 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfig.java      |   1 -
 .../hbase/replication/ReplicationPeerImpl.java  |   4 +-
 .../hbase/replication/ReplicationQueueInfo.java |  23 +-
 .../hbase/replication/ReplicationUtils.java     |  56 ++
 .../replication/TestReplicationStateZKImpl.java |  22 -
 .../regionserver/ReplicationSourceService.java  |   3 +-
 .../regionserver/PeerProcedureHandler.java      |   3 +
 .../regionserver/PeerProcedureHandlerImpl.java  |  50 +-
 .../RecoveredReplicationSource.java             |   6 +-
 .../RecoveredReplicationSourceShipper.java      |   8 +-
 .../replication/regionserver/Replication.java   |  15 +-
 .../regionserver/ReplicationSource.java         |  34 +-
 .../regionserver/ReplicationSourceFactory.java  |   4 +-
 .../ReplicationSourceInterface.java             |   8 +-
 .../regionserver/ReplicationSourceManager.java  | 895 ++++++++++---------
 .../regionserver/ReplicationSourceShipper.java  |   6 +-
 .../ReplicationSourceWALReader.java             |   2 +-
 .../replication/ReplicationSourceDummy.java     |   2 +-
 .../replication/TestNamespaceReplication.java   |  57 +-
 .../TestReplicationSourceManager.java           |  11 +-
 .../TestReplicationSourceManagerZkImpl.java     |   1 -
 21 files changed, 659 insertions(+), 552 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index fdae288..bf8d030 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
-
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
index 3e17025..604e0bb 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -28,6 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 public class ReplicationPeerImpl implements ReplicationPeer {
+
   private final Configuration conf;
 
   private final String id;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java
index ecd888f..cd65f9b 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java
@@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.ServerName;
 
 /**
- * This class is responsible for the parsing logic for a znode representing a queue.
+ * This class is responsible for the parsing logic for a queue id representing a queue.
  * It will extract the peerId if it's recovered as well as the dead region servers
  * that were part of the queue's history.
  */
@@ -38,21 +38,20 @@ public class ReplicationQueueInfo {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueueInfo.class);
 
   private final String peerId;
-  private final String peerClusterZnode;
+  private final String queueId;
   private boolean queueRecovered;
   // List of all the dead region servers that had this queue (if recovered)
   private List<ServerName> deadRegionServers = new ArrayList<>();
 
   /**
-   * The passed znode will be either the id of the peer cluster or
-   * the handling story of that queue in the form of id-servername-*
+   * The passed queueId will be either the id of the peer or the handling story of that queue
+   * in the form of id-servername-*
    */
-  public ReplicationQueueInfo(String znode) {
-    this.peerClusterZnode = znode;
-    String[] parts = znode.split("-", 2);
+  public ReplicationQueueInfo(String queueId) {
+    this.queueId = queueId;
+    String[] parts = queueId.split("-", 2);
     this.queueRecovered = parts.length != 1;
-    this.peerId = this.queueRecovered ?
-        parts[0] : peerClusterZnode;
+    this.peerId = this.queueRecovered ? parts[0] : queueId;
     if (parts.length >= 2) {
       // extract dead servers
       extractDeadServersFromZNodeString(parts[1], this.deadRegionServers);
@@ -60,7 +59,7 @@ public class ReplicationQueueInfo {
   }
 
   /**
-   * Parse dead server names from znode string servername can contain "-" such as
+   * Parse dead server names from queue id. servername can contain "-" such as
    * "ip-10-46-221-101.ec2.internal", so we need skip some "-" during parsing for the following
    * cases: 2-ip-10-46-221-101.ec2.internal,52170,1364333181125-&lt;server name>-...
    */
@@ -119,8 +118,8 @@ public class ReplicationQueueInfo {
     return this.peerId;
   }
 
-  public String getPeerClusterZnode() {
-    return this.peerClusterZnode;
+  public String getQueueId() {
+    return this.queueId;
   }
 
   public boolean isQueueRecovered() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
index 7b676ca..ebe68a7 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
@@ -18,11 +18,15 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -66,4 +70,56 @@ public final class ReplicationUtils {
       queueStorage.removeReplicatorIfQueueIsEmpty(replicator);
     }
   }
+
+  private static boolean isCollectionEqual(Collection<String> c1, Collection<String> c2) {
+    if (c1 == null) {
+      return c2 == null;
+    }
+    if (c2 == null) {
+      return false;
+    }
+    return c1.size() == c2.size() && c1.containsAll(c2);
+  }
+
+  private static boolean isNamespacesEqual(Set<String> ns1, Set<String> ns2) {
+    return isCollectionEqual(ns1, ns2);
+  }
+
+  private static boolean isTableCFsEqual(Map<TableName, List<String>> tableCFs1,
+      Map<TableName, List<String>> tableCFs2) {
+    if (tableCFs1 == null) {
+      return tableCFs2 == null;
+    }
+    if (tableCFs2 == null) {
+      return false;
+    }
+    if (tableCFs1.size() != tableCFs2.size()) {
+      return false;
+    }
+    for (Map.Entry<TableName, List<String>> entry1 : tableCFs1.entrySet()) {
+      TableName table = entry1.getKey();
+      if (!tableCFs2.containsKey(table)) {
+        return false;
+      }
+      List<String> cfs1 = entry1.getValue();
+      List<String> cfs2 = tableCFs2.get(table);
+      if (!isCollectionEqual(cfs1, cfs2)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public static boolean isKeyConfigEqual(ReplicationPeerConfig rpc1, ReplicationPeerConfig rpc2) {
+    if (rpc1.replicateAllUserTables() != rpc2.replicateAllUserTables()) {
+      return false;
+    }
+    if (rpc1.replicateAllUserTables()) {
+      return isNamespacesEqual(rpc1.getExcludeNamespaces(), rpc2.getExcludeNamespaces()) &&
+        isTableCFsEqual(rpc1.getExcludeTableCFsMap(), rpc2.getExcludeTableCFsMap());
+    } else {
+      return isNamespacesEqual(rpc1.getNamespaces(), rpc2.getNamespaces()) &&
+        isTableCFsEqual(rpc1.getTableCFsMap(), rpc2.getTableCFsMap());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index 6825c36..2790bd0 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -18,9 +18,7 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -37,14 +35,10 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @Category({ ReplicationTests.class, MediumTests.class })
 public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateZKImpl.class);
-
   private static Configuration conf;
   private static HBaseZKTestingUtility utility;
   private static ZKWatcher zkw;
@@ -92,20 +86,4 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
   public static void tearDownAfterClass() throws Exception {
     utility.shutdownMiniZKCluster();
   }
-
-  private static class WarnOnlyAbortable implements Abortable {
-
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.warn("TestReplicationStateZKImpl received abort, ignoring.  Reason: " + why);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(e.toString(), e);
-      }
-    }
-
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
index a82fa3d..2aef0a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
index b392985..65da9af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
@@ -23,6 +23,9 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 
+/**
+ * A handler for modifying replication peer in peer procedures.
+ */
 @InterfaceAudience.Private
 public interface PeerProcedureHandler {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
index c09c6a0..ce8fdae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -15,21 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
 import java.util.concurrent.locks.Lock;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
-  private static final Logger LOG = LoggerFactory.getLogger(PeerProcedureHandlerImpl.class);
 
   private final ReplicationSourceManager replicationSourceManager;
   private final KeyLocker<String> peersLock = new KeyLocker<>();
@@ -39,7 +38,7 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
   }
 
   @Override
-  public void addPeer(String peerId) throws ReplicationException, IOException {
+  public void addPeer(String peerId) throws IOException {
     Lock peerLock = peersLock.acquireLock(peerId);
     try {
       replicationSourceManager.addPeer(peerId);
@@ -49,7 +48,7 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
   }
 
   @Override
-  public void removePeer(String peerId) throws ReplicationException, IOException {
+  public void removePeer(String peerId) throws IOException {
     Lock peerLock = peersLock.acquireLock(peerId);
     try {
       if (replicationSourceManager.getReplicationPeers().getPeer(peerId) != null) {
@@ -60,35 +59,50 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
     }
   }
 
-  @Override
-  public void disablePeer(String peerId) throws ReplicationException, IOException {
+  private void refreshPeerState(String peerId) throws ReplicationException, IOException {
     PeerState newState;
     Lock peerLock = peersLock.acquireLock(peerId);
     try {
+      ReplicationPeerImpl peer = replicationSourceManager.getReplicationPeers().getPeer(peerId);
+      if (peer == null) {
+        throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
+      }
+      PeerState oldState = peer.getPeerState();
       newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+      // RS need to start work with the new replication state change
+      if (oldState.equals(PeerState.ENABLED) && newState.equals(PeerState.DISABLED)) {
+        replicationSourceManager.refreshSources(peerId);
+      }
     } finally {
       peerLock.unlock();
     }
-    LOG.info("disable replication peer, id: {}, new state: {}", peerId, newState);
   }
 
   @Override
   public void enablePeer(String peerId) throws ReplicationException, IOException {
-    PeerState newState;
-    Lock peerLock = peersLock.acquireLock(peerId);
-    try {
-      newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
-    } finally {
-      peerLock.unlock();
-    }
-    LOG.info("enable replication peer, id: {}, new state: {}", peerId, newState);
+    refreshPeerState(peerId);
+  }
+
+  @Override
+  public void disablePeer(String peerId) throws ReplicationException, IOException {
+    refreshPeerState(peerId);
   }
 
   @Override
   public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
     Lock peerLock = peersLock.acquireLock(peerId);
     try {
-      replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+      ReplicationPeerImpl peer = replicationSourceManager.getReplicationPeers().getPeer(peerId);
+      if (peer == null) {
+        throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
+      }
+      ReplicationPeerConfig oldConfig = peer.getPeerConfig();
+      ReplicationPeerConfig newConfig =
+          replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+      // RS need to start work with the new replication config change
+      if (!ReplicationUtils.isKeyConfigEqual(oldConfig, newConfig)) {
+        replicationSourceManager.refreshSources(peerId);
+      }
     } finally {
       peerLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index 7bceb78..1be9a88 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -81,7 +81,7 @@ public class RecoveredReplicationSource extends ReplicationSource {
     ReplicationSourceWALReader walReader = new RecoveredReplicationSourceWALReader(fs,
         conf, queue, startPosition, walEntryFilter, this);
     Threads.setDaemonThreadRunning(walReader, threadName
-        + ".replicationSource.replicationWALReaderThread." + walGroupId + "," + peerClusterZnode,
+        + ".replicationSource.replicationWALReaderThread." + walGroupId + "," + queueId,
       getUncaughtExceptionHandler());
     return walReader;
   }
@@ -178,8 +178,8 @@ public class RecoveredReplicationSource extends ReplicationSource {
         }
       }
       if (allTasksDone) {
-        manager.closeRecoveredQueue(this);
-        LOG.info("Finished recovering queue " + peerClusterZnode + " with the following stats: "
+        manager.removeRecoveredSource(this);
+        LOG.info("Finished recovering queue " + queueId + " with the following stats: "
             + getStats());
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
index fb365bc..1e45496 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
@@ -77,7 +77,7 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
         if (entryBatch.getWalEntries().isEmpty()
             && entryBatch.getLastSeqIds().isEmpty()) {
           LOG.debug("Finished recovering queue for group " + walGroupId + " of peer "
-              + source.getPeerClusterZnode());
+              + source.getQueueId());
           source.getSourceMetrics().incrCompletedRecoveryQueue();
           setWorkerState(WorkerState.FINISHED);
           continue;
@@ -114,7 +114,7 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
   // normally has a position (unless the RS failed between 2 logs)
   private long getRecoveredQueueStartPos() {
     long startPosition = 0;
-    String peerClusterZnode = source.getPeerClusterZnode();
+    String peerClusterZnode = source.getQueueId();
     try {
       startPosition = this.replicationQueues.getWALPosition(source.getServerWALsBelongTo(),
         peerClusterZnode, this.queue.peek().getName());
@@ -130,8 +130,8 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
 
   @Override
   protected void updateLogPosition(long lastReadPosition) {
-    source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getPeerClusterZnode(),
-      lastReadPosition, true, false);
+    source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getQueueId(),
+      lastReadPosition, true);
     lastLoggedPosition = lastReadPosition;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index dca2439..d1a3266 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@@ -59,10 +58,10 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 /**
  * Gateway to Replication.  Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  */
@@ -218,11 +217,7 @@ public class Replication implements
    * @throws IOException
    */
   public void startReplicationService() throws IOException {
-    try {
-      this.replicationManager.init();
-    } catch (ReplicationException e) {
-      throw new IOException(e);
-    }
+    this.replicationManager.init();
     this.replicationSink = new ReplicationSink(this.conf, this.server);
     this.scheduleThreadPool.scheduleAtFixedRate(
       new ReplicationStatisticsThread(this.replicationSink, this.replicationManager),
@@ -280,9 +275,9 @@ public class Replication implements
       throws IOException {
     try {
       this.replicationManager.addHFileRefs(tableName, family, pairs);
-    } catch (ReplicationException e) {
+    } catch (IOException e) {
       LOG.error("Failed to add hfile references in the replication queue.", e);
-      throw new IOException(e);
+      throw e;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 0b44ba4..6b622ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -106,7 +106,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   // total number of edits we replicated
   private AtomicLong totalReplicatedEdits = new AtomicLong(0);
   // The znode we currently play with
-  protected String peerClusterZnode;
+  protected String queueId;
   // Maximum number of retries before taking bold actions
   private int maxRetriesMultiplier;
   // Indicates if this particular source is running
@@ -142,14 +142,14 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
    * @param fs file system to use
    * @param manager replication manager to ping to
    * @param server the server for this region server
-   * @param peerClusterZnode the name of our znode
+   * @param queueId the id of our replication queue
    * @param clusterId unique UUID for the cluster
    * @param metrics metrics for replication source
    */
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
       ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
-      String peerClusterZnode, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
+      String queueId, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
       MetricsSource metrics) throws IOException {
     this.server = server;
     this.conf = HBaseConfiguration.create(conf);
@@ -168,8 +168,8 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     this.metrics = metrics;
     this.clusterId = clusterId;
 
-    this.peerClusterZnode = peerClusterZnode;
-    this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode);
+    this.queueId = queueId;
+    this.replicationQueueInfo = new ReplicationQueueInfo(queueId);
     // ReplicationQueueInfo parses the peerId out of the znode for us
     this.peerId = this.replicationQueueInfo.getPeerId();
     this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
@@ -179,7 +179,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0);
     this.totalBufferUsed = manager.getTotalBufferUsed();
     this.walFileLengthProvider = walFileLengthProvider;
-    LOG.info("peerClusterZnode=" + peerClusterZnode + ", ReplicationSource : " + peerId
+    LOG.info("queueId=" + queueId + ", ReplicationSource : " + peerId
         + ", currentBandwidth=" + this.currentBandwidth);
   }
 
@@ -217,12 +217,6 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   @Override
   public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
       throws ReplicationException {
-    String peerId = peerClusterZnode;
-    if (peerId.contains("-")) {
-      // peerClusterZnode will be in the form peerId + "-" + rsZNode.
-      // A peerId will not have "-" in its name, see HBASE-11394
-      peerId = peerClusterZnode.split("-")[0];
-    }
     Map<TableName, List<String>> tableCFMap = replicationPeer.getTableCFs();
     if (tableCFMap != null) {
       List<String> tableCfs = tableCFMap.get(tableName);
@@ -311,7 +305,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
           + peerClusterId + " which is not allowed by ReplicationEndpoint:"
           + replicationEndpoint.getClass().getName(), null, false);
-      this.manager.closeQueue(this);
+      this.manager.removeSource(this);
       return;
     }
     LOG.info("Replicating " + clusterId + " -> " + peerClusterId);
@@ -356,7 +350,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     ReplicationSourceWALReader walReader =
         new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this);
     return (ReplicationSourceWALReader) Threads.setDaemonThreadRunning(walReader,
-      threadName + ".replicationSource.wal-reader." + walGroupId + "," + peerClusterZnode,
+      threadName + ".replicationSource.wal-reader." + walGroupId + "," + queueId,
       getUncaughtExceptionHandler());
   }
 
@@ -450,7 +444,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
         LOG.error("Unexpected exception in ReplicationSource", e);
       }
     };
-    Threads.setDaemonThreadRunning(this, n + ".replicationSource," + this.peerClusterZnode,
+    Threads.setDaemonThreadRunning(this, n + ".replicationSource," + this.queueId,
       handler);
   }
 
@@ -466,9 +460,9 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
 
   public void terminate(String reason, Exception cause, boolean join) {
     if (cause == null) {
-      LOG.info("Closing source " + this.peerClusterZnode + " because: " + reason);
+      LOG.info("Closing source " + this.queueId + " because: " + reason);
     } else {
-      LOG.error("Closing source " + this.peerClusterZnode + " because an error occurred: " + reason,
+      LOG.error("Closing source " + this.queueId + " because an error occurred: " + reason,
         cause);
     }
     this.sourceRunning = false;
@@ -491,7 +485,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
           this.replicationEndpoint.awaitTerminated(sleepForRetries * maxRetriesMultiplier, TimeUnit.MILLISECONDS);
         } catch (TimeoutException te) {
           LOG.warn("Got exception while waiting for endpoint to shutdown for replication source :"
-              + this.peerClusterZnode,
+              + this.queueId,
             te);
         }
       }
@@ -499,8 +493,8 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   }
 
   @Override
-  public String getPeerClusterZnode() {
-    return this.peerClusterZnode;
+  public String getQueueId() {
+    return this.queueId;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java
index 865a202..93e8331 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java
@@ -32,8 +32,8 @@ public class ReplicationSourceFactory {
 
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationSourceFactory.class);
 
-  static ReplicationSourceInterface create(Configuration conf, String peerId) {
-    ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
+  static ReplicationSourceInterface create(Configuration conf, String queueId) {
+    ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queueId);
     boolean isQueueRecovered = replicationQueueInfo.isQueueRecovered();
     ReplicationSourceInterface src;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index 4f10c73..d7cf9a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -51,7 +51,7 @@ public interface ReplicationSourceInterface {
    */
   void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
       ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
-      String peerClusterZnode, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
+      String queueId, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
       MetricsSource metrics) throws IOException;
 
   /**
@@ -96,11 +96,11 @@ public interface ReplicationSourceInterface {
   Path getCurrentPath();
 
   /**
-   * Get the id that the source is replicating to
+   * Get the queue id that the source is replicating to
    *
-   * @return peer cluster id
+   * @return queue id
    */
-  String getPeerClusterZnode();
+  String getQueueId();
 
   /**
    * Get the id that the source is replicating to.


[19/48] hbase git commit: HBASE-19524 Master side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19524 Master side changes for moving peer modification from zk watcher to procedure


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 1966abfbb9f57ce32ebe031a8b36b40e66c909fb
Parents: 12fd193
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 18 15:22:36 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../procedure2/RemoteProcedureDispatcher.java   |  3 +-
 .../src/main/protobuf/MasterProcedure.proto     | 21 ++++-
 .../src/main/protobuf/RegionServerStatus.proto  |  3 +-
 .../src/main/protobuf/Replication.proto         |  5 +
 .../replication/ReplicationPeersZKImpl.java     |  4 +-
 .../org/apache/hadoop/hbase/master/HMaster.java | 98 ++++++++------------
 .../hadoop/hbase/master/MasterRpcServices.java  |  4 +-
 .../hadoop/hbase/master/MasterServices.java     | 26 +++---
 .../assignment/RegionTransitionProcedure.java   | 13 +--
 .../master/procedure/MasterProcedureEnv.java    |  5 +
 .../master/procedure/ProcedurePrepareLatch.java |  2 +-
 .../master/replication/AddPeerProcedure.java    | 97 +++++++++++++++++++
 .../replication/DisablePeerProcedure.java       | 70 ++++++++++++++
 .../master/replication/EnablePeerProcedure.java | 69 ++++++++++++++
 .../master/replication/ModifyPeerProcedure.java | 97 ++++++++++++++++---
 .../master/replication/RefreshPeerCallable.java | 67 -------------
 .../replication/RefreshPeerProcedure.java       | 28 ++++--
 .../master/replication/RemovePeerProcedure.java | 69 ++++++++++++++
 .../master/replication/ReplicationManager.java  | 76 +++++++--------
 .../replication/UpdatePeerConfigProcedure.java  | 92 ++++++++++++++++++
 .../hbase/regionserver/HRegionServer.java       |  5 +-
 .../regionserver/RefreshPeerCallable.java       | 70 ++++++++++++++
 .../hbase/master/MockNoopMasterServices.java    | 23 +++--
 .../replication/DummyModifyPeerProcedure.java   | 13 ++-
 24 files changed, 735 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
index 78c49fb..3b925a6 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -247,9 +247,8 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
     /**
      * Called when RS tells the remote procedure is failed through the {@code reportProcedureDone}
      * method.
-     * @param error the error message
      */
-    void remoteOperationFailed(TEnv env, String error);
+    void remoteOperationFailed(TEnv env, RemoteProcedureException error);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 0e2bdba..ae676ea 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -27,6 +27,7 @@ option optimize_for = SPEED;
 import "HBase.proto";
 import "RPC.proto";
 import "Snapshot.proto";
+import "Replication.proto";
 
 // ============================================================================
 //  WARNING - Compatibility rules
@@ -367,9 +368,10 @@ message GCMergedRegionsStateData {
 }
 
 enum PeerModificationState {
-  UPDATE_PEER_STORAGE = 1;
-  REFRESH_PEER_ON_RS = 2;
-  POST_PEER_MODIFICATION = 3;
+  PRE_PEER_MODIFICATION = 1;
+  UPDATE_PEER_STORAGE = 2;
+  REFRESH_PEER_ON_RS = 3;
+  POST_PEER_MODIFICATION = 4;
 }
 
 message PeerModificationStateData {
@@ -394,4 +396,17 @@ message RefreshPeerParameter {
   required string peer_id = 1;
   required PeerModificationType type = 2;
   required ServerName target_server = 3;
+}
+
+message ModifyPeerStateData {
+  required string peer_id = 1;
+}
+
+message AddPeerStateData {
+  required ReplicationPeer peer_config = 1;
+  required bool enabled = 2;
+}
+
+message UpdatePeerConfigStateData {
+  required ReplicationPeer peer_config = 1;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index eb396ac..4f75941 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -28,6 +28,7 @@ option optimize_for = SPEED;
 
 import "HBase.proto";
 import "ClusterStatus.proto";
+import "ErrorHandling.proto";
 
 message RegionServerStartupRequest {
   /** Port number this regionserver is up on */
@@ -152,7 +153,7 @@ message ReportProcedureDoneRequest {
     ERROR = 2;
   }
   required Status status = 2;
-  optional string error = 3;
+  optional ForeignExceptionMessage error = 3;
 }
 
 message ReportProcedureDoneResponse {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-protocol-shaded/src/main/protobuf/Replication.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Replication.proto b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
index 8657c25..9f7b4c2 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -84,6 +84,7 @@ message AddReplicationPeerRequest {
 }
 
 message AddReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message RemoveReplicationPeerRequest {
@@ -91,6 +92,7 @@ message RemoveReplicationPeerRequest {
 }
 
 message RemoveReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message EnableReplicationPeerRequest {
@@ -98,6 +100,7 @@ message EnableReplicationPeerRequest {
 }
 
 message EnableReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message DisableReplicationPeerRequest {
@@ -105,6 +108,7 @@ message DisableReplicationPeerRequest {
 }
 
 message DisableReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message GetReplicationPeerConfigRequest {
@@ -122,6 +126,7 @@ message UpdateReplicationPeerConfigRequest {
 }
 
 message UpdateReplicationPeerConfigResponse {
+  optional uint64 proc_id = 1;
 }
 
 message ListReplicationPeersRequest {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/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 4e469ed..e4b1875 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
@@ -530,7 +530,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           if (queueInfo.getPeerId().equals(peerId)) {
-            throw new ReplicationException("undeleted queue for peerId: " + peerId
+            throw new IllegalArgumentException("undeleted queue for peerId: " + peerId
                 + ", replicator: " + replicator + ", queueId: " + queueId);
           }
         }
@@ -538,7 +538,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       // Check for hfile-refs queue
       if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode)
           && queuesClient.getAllPeersFromHFileRefsQueue().contains(peerId)) {
-        throw new ReplicationException("Undeleted queue for peerId: " + peerId
+        throw new IllegalArgumentException("Undeleted queue for peerId: " + peerId
             + ", found in hfile-refs node path " + hfileRefsZNode);
       }
     } catch (KeeperException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/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 9487946..ad81cc6 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
@@ -128,7 +128,13 @@ import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.RecoverMetaProcedure;
 import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
+import org.apache.hadoop.hbase.master.replication.AddPeerProcedure;
+import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
+import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
+import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
+import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
@@ -141,6 +147,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
@@ -169,7 +176,6 @@ import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EncryptionTest;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.IdLock;
@@ -198,6 +204,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
@@ -330,15 +337,15 @@ public class HMaster extends HRegionServer implements MasterServices {
   private volatile boolean activeMaster = false;
 
   // flag set after we complete initialization once active
-  private final ProcedureEvent initialized = new ProcedureEvent("master initialized");
+  private final ProcedureEvent<?> initialized = new ProcedureEvent<>("master initialized");
 
   // flag set after master services are started,
   // initialization may have not completed yet.
   volatile boolean serviceStarted = false;
 
   // flag set after we complete assignMeta.
-  private final ProcedureEvent serverCrashProcessingEnabled =
-    new ProcedureEvent("server crash processing");
+  private final ProcedureEvent<?> serverCrashProcessingEnabled =
+    new ProcedureEvent<>("server crash processing");
 
   // Maximum time we should run balancer for
   private final int maxBlancingTime;
@@ -1219,7 +1226,6 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   private void startProcedureExecutor() throws IOException {
     final MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
-    final Path rootDir = FSUtils.getRootDir(conf);
 
     procedureStore = new WALProcedureStore(conf,
         new MasterProcedureEnv.WALStoreLeaseRecovery(this));
@@ -2325,11 +2331,8 @@ public class HMaster extends HRegionServer implements MasterServices {
             return true;
           }
           Pair<RegionInfo, ServerName> pair =
-              new Pair(MetaTableAccessor.getRegionInfo(data),
+              new Pair<>(MetaTableAccessor.getRegionInfo(data),
                   MetaTableAccessor.getServerName(data,0));
-          if (pair == null) {
-            return false;
-          }
           if (!pair.getFirst().getTable().equals(tableName)) {
             return false;
           }
@@ -2763,7 +2766,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return initialized;
   }
 
@@ -2782,7 +2785,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureExecutor.getEnvironment().setEventReady(serverCrashProcessingEnabled, b);
   }
 
-  public ProcedureEvent getServerCrashProcessingEnabledEvent() {
+  public ProcedureEvent<?> getServerCrashProcessingEnabledEvent() {
     return serverCrashProcessingEnabled;
   }
 
@@ -3333,54 +3336,36 @@ public class HMaster extends HRegionServer implements MasterServices {
     return favoredNodesManager;
   }
 
+  private long executePeerProcedure(ModifyPeerProcedure procedure) throws IOException {
+    long procId = procedureExecutor.submitProcedure(procedure);
+    procedure.getLatch().await();
+    return procId;
+  }
+
   @Override
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preAddReplicationPeer(peerId, peerConfig);
-    }
-    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config="
-        + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
-    this.replicationManager.addReplicationPeer(peerId, peerConfig, enabled);
-    if (cpHost != null) {
-      cpHost.postAddReplicationPeer(peerId, peerConfig);
-    }
+    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config=" +
+      peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
+    return executePeerProcedure(new AddPeerProcedure(peerId, peerConfig, enabled));
   }
 
   @Override
-  public void removeReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preRemoveReplicationPeer(peerId);
-    }
+  public long removeReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId);
-    this.replicationManager.removeReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postRemoveReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new RemovePeerProcedure(peerId));
   }
 
   @Override
-  public void enableReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preEnableReplicationPeer(peerId);
-    }
+  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId);
-    this.replicationManager.enableReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postEnableReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new EnablePeerProcedure(peerId));
   }
 
   @Override
-  public void disableReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preDisableReplicationPeer(peerId);
-    }
+  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId);
-    this.replicationManager.disableReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postDisableReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new DisablePeerProcedure(peerId));
   }
 
   @Override
@@ -3399,17 +3384,11 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
-    }
-    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId
-        + ", config=" + peerConfig);
-    this.replicationManager.updatePeerConfig(peerId, peerConfig);
-    if (cpHost != null) {
-      cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig);
-    }
+    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId +
+      ", config=" + peerConfig);
+    return executePeerProcedure(new UpdatePeerConfigProcedure(peerId, peerConfig));
   }
 
   @Override
@@ -3562,10 +3541,15 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  public void remoteProcedureFailed(long procId, String error) {
+  public void remoteProcedureFailed(long procId, RemoteProcedureException error) {
     RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
     if (procedure != null) {
       procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
     }
   }
-}
\ No newline at end of file
+
+  @Override
+  public ReplicationManager getReplicationManager() {
+    return replicationManager;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index f875e20..8025a51 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
@@ -2256,7 +2257,8 @@ public class MasterRpcServices extends RSRpcServices
     if (request.getStatus() == ReportProcedureDoneRequest.Status.SUCCESS) {
       master.remoteProcedureCompleted(request.getProcId());
     } else {
-      master.remoteProcedureFailed(request.getProcId(), request.getError());
+      master.remoteProcedureFailed(request.getProcId(),
+        RemoteProcedureException.fromProto(request.getError()));
     }
     return ReportProcedureDoneResponse.getDefaultInstance();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 9786fde..e798455 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -18,10 +17,11 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -52,8 +53,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
-import com.google.protobuf.Service;
-
 /**
  * A curated subset of services provided by {@link HMaster}.
  * For use internally only. Passed to Managers, Services and Chores so can pass less-than-a
@@ -136,7 +135,7 @@ public interface MasterServices extends Server {
    * @return Tripped when Master has finished initialization.
    */
   @VisibleForTesting
-  public ProcedureEvent getInitializedEvent();
+  public ProcedureEvent<?> getInitializedEvent();
 
   /**
    * @return Master's instance of {@link MetricsMaster}
@@ -430,26 +429,26 @@ public interface MasterServices extends Server {
    * @param peerConfig configuration for the replication slave cluster
    * @param enabled peer state, true if ENABLED and false if DISABLED
    */
-  void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException, IOException;
 
   /**
    * Removes a peer and stops the replication
    * @param peerId a short name that identifies the peer
    */
-  void removeReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long removeReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Restart the replication stream to the specified peer
    * @param peerId a short name that identifies the peer
    */
-  void enableReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long enableReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Stop the replication stream to the specified peer
    * @param peerId a short name that identifies the peer
    */
-  void disableReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long disableReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Returns the configured ReplicationPeerConfig for the specified peer
@@ -460,11 +459,16 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
+   * Returns the {@link ReplicationManager}.
+   */
+  ReplicationManager getReplicationManager();
+
+  /**
    * Update the peerConfig for the specified peer
    * @param peerId a short name that identifies the peer
    * @param peerConfig new config for the peer
    */
-  void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 04dccc4..1724a38 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -16,12 +16,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -33,13 +31,16 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 /**
  * Base class for the Assign and Unassign Procedure.
  *
@@ -415,7 +416,7 @@ public abstract class RegionTransitionProcedure
   }
 
   @Override
-  public void remoteOperationFailed(MasterProcedureEnv env, String error) {
+  public void remoteOperationFailed(MasterProcedureEnv env, RemoteProcedureException error) {
     // should not be called for region operation until we modified the open/close region procedure
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index 0a4c97d..fa4d371 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
@@ -137,6 +138,10 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return remoteDispatcher;
   }
 
+  public ReplicationManager getReplicationManager() {
+    return master.getReplicationManager();
+  }
+
   public boolean isRunning() {
     if (this.master == null || this.master.getMasterProcedureExecutor() == null) return false;
     return master.getMasterProcedureExecutor().isRunning();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
index 09d05e6..dbea6fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
@@ -64,7 +64,7 @@ public abstract class ProcedurePrepareLatch {
   protected abstract void countDown(final Procedure proc);
   public abstract void await() throws IOException;
 
-  protected static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
+  public static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
     if (latch != null) {
       latch.countDown(proc);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
new file mode 100644
index 0000000..c3862d8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -0,0 +1,97 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData;
+
+/**
+ * The procedure for adding a new replication peer.
+ */
+@InterfaceAudience.Private
+public class AddPeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(AddPeerProcedure.class);
+
+  private ReplicationPeerConfig peerConfig;
+
+  private boolean enabled;
+
+  public AddPeerProcedure() {
+  }
+
+  public AddPeerProcedure(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) {
+    super(peerId);
+    this.peerConfig = peerConfig;
+    this.enabled = enabled;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ADD;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preAddReplicationPeer(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationManager().addReplicationPeer(peerId, peerConfig, enabled);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully added " + (enabled ? "ENABLED" : "DISABLED") + " peer " + peerId +
+      ", config " + peerConfig);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      env.getMasterCoprocessorHost().postAddReplicationPeer(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(AddPeerStateData.newBuilder()
+        .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).setEnabled(enabled).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    AddPeerStateData data = serializer.deserialize(AddPeerStateData.class);
+    peerConfig = ReplicationPeerConfigUtil.convert(data.getPeerConfig());
+    enabled = data.getEnabled();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
new file mode 100644
index 0000000..0b32db9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -0,0 +1,70 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for disabling a replication peer.
+ */
+@InterfaceAudience.Private
+public class DisablePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(DisablePeerProcedure.class);
+
+  public DisablePeerProcedure() {
+  }
+
+  public DisablePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.DISABLE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preDisableReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception {
+    env.getReplicationManager().disableReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully disabled peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postDisableReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
new file mode 100644
index 0000000..92ba000
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -0,0 +1,69 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for enabling a replication peer.
+ */
+@InterfaceAudience.Private
+public class EnablePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(EnablePeerProcedure.class);
+
+  public EnablePeerProcedure() {
+  }
+
+  public EnablePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ENABLE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preEnableReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
+    env.getReplicationManager().enableReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully enabled peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postEnableReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index fca05a7..7076bab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -21,15 +21,22 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyPeerStateData;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
 
+/**
+ * The base class for all replication peer related procedure.
+ */
 @InterfaceAudience.Private
 public abstract class ModifyPeerProcedure
     extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
@@ -39,11 +46,21 @@ public abstract class ModifyPeerProcedure
 
   protected String peerId;
 
+  // used to keep compatible with old client where we can only returns after updateStorage.
+  protected ProcedurePrepareLatch latch;
+
   protected ModifyPeerProcedure() {
   }
 
   protected ModifyPeerProcedure(String peerId) {
     this.peerId = peerId;
+    // TODO: temporarily set a 4.0 here to always wait for the procedure exection completed. Change
+    // to 3.0 or 2.0 after the client modification is done.
+    this.latch = ProcedurePrepareLatch.createLatch(4, 0);
+  }
+
+  public ProcedurePrepareLatch getLatch() {
+    return latch;
   }
 
   @Override
@@ -52,28 +69,58 @@ public abstract class ModifyPeerProcedure
   }
 
   /**
-   * Return {@code false} means that the operation is invalid and we should give up, otherwise
-   * {@code true}.
+   * Called before we start the actual processing. If an exception is thrown then we will give up
+   * and mark the procedure as failed directly.
+   */
+  protected abstract void prePeerModification(MasterProcedureEnv env) throws IOException;
+
+  /**
+   * We will give up and mark the procedure as failure if {@link IllegalArgumentException} is
+   * thrown, for other type of Exception we will retry.
+   */
+  protected abstract void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception;
+
+  /**
+   * Called before we finish the procedure. The implementation can do some logging work, and also
+   * call the coprocessor hook if any.
    * <p>
-   * You need to call {@link #setFailure(String, Throwable)} to give the detail failure information.
+   * Notice that, since we have already done the actual work, throwing exception here will not fail
+   * this procedure, we will just ignore it and finish the procedure as suceeded.
    */
-  protected abstract boolean updatePeerStorage() throws IOException;
+  protected abstract void postPeerModification(MasterProcedureEnv env) throws IOException;
 
-  protected void postPeerModification() {
+  private void releaseLatch() {
+    ProcedurePrepareLatch.releaseLatch(latch, this);
   }
 
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     switch (state) {
-      case UPDATE_PEER_STORAGE:
+      case PRE_PEER_MODIFICATION:
         try {
-          if (!updatePeerStorage()) {
-            assert isFailed() : "setFailure is not called";
-            return Flow.NO_MORE_STATE;
-          }
+          prePeerModification(env);
         } catch (IOException e) {
-          LOG.warn("update peer storage failed, retry", e);
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", mark the procedure as failure and give up", e);
+          setFailure("prePeerModification", e);
+          releaseLatch();
+          return Flow.NO_MORE_STATE;
+        }
+        setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
+        return Flow.HAS_MORE_STATE;
+      case UPDATE_PEER_STORAGE:
+        try {
+          updatePeerStorage(env);
+        } catch (IllegalArgumentException e) {
+          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer",
+            new DoNotRetryIOException(e));
+          releaseLatch();
+          return Flow.NO_MORE_STATE;
+        } catch (Exception e) {
+          LOG.warn(
+            getClass().getName() + " update peer storage for peer " + peerId + " failed, retry", e);
           throw new ProcedureYieldException();
         }
         setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
@@ -85,7 +132,13 @@ public abstract class ModifyPeerProcedure
         setNextState(PeerModificationState.POST_PEER_MODIFICATION);
         return Flow.HAS_MORE_STATE;
       case POST_PEER_MODIFICATION:
-        postPeerModification();
+        try {
+          postPeerModification(env);
+        } catch (IOException e) {
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", ignore since the procedure has already done", e);
+        }
+        releaseLatch();
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
@@ -107,6 +160,12 @@ public abstract class ModifyPeerProcedure
   @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
       throws IOException, InterruptedException {
+    if (state == PeerModificationState.PRE_PEER_MODIFICATION ||
+      state == PeerModificationState.UPDATE_PEER_STORAGE) {
+      // actually the peer related operations has no rollback, but if we haven't done any
+      // modifications on the peer storage, we can just return.
+      return;
+    }
     throw new UnsupportedOperationException();
   }
 
@@ -122,6 +181,18 @@ public abstract class ModifyPeerProcedure
 
   @Override
   protected PeerModificationState getInitialState() {
-    return PeerModificationState.UPDATE_PEER_STORAGE;
+    return PeerModificationState.PRE_PEER_MODIFICATION;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(ModifyPeerStateData.newBuilder().setPeerId(peerId).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    peerId = serializer.deserialize(ModifyPeerStateData.class).getPeerId();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
deleted file mode 100644
index 4e09107..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
+++ /dev/null
@@ -1,67 +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.master.replication;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
-
-/**
- * The callable executed at RS side to refresh the peer config/state.
- * <p>
- * TODO: only a dummy implementation for verifying the framework, will add implementation later.
- */
-@InterfaceAudience.Private
-public class RefreshPeerCallable implements RSProcedureCallable {
-
-  private HRegionServer rs;
-
-  private String peerId;
-
-  private Exception initError;
-
-  @Override
-  public Void call() throws Exception {
-    if (initError != null) {
-      throw initError;
-    }
-    rs.getFileSystem().create(new Path("/" + peerId + "/" + rs.getServerName().toString())).close();
-    return null;
-  }
-
-  @Override
-  public void init(byte[] parameter, HRegionServer rs) {
-    this.rs = rs;
-    try {
-      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
-    } catch (InvalidProtocolBufferException e) {
-      initError = e;
-      return;
-    }
-  }
-
-  @Override
-  public EventType getEventType() {
-    return EventType.RS_REFRESH_PEER;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
index 18da487..ddc2401 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
+import org.apache.hadoop.hbase.replication.regionserver.RefreshPeerCallable;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -118,15 +120,22 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
             .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray());
   }
 
-  private void complete(MasterProcedureEnv env, boolean succ) {
+  private void complete(MasterProcedureEnv env, Throwable error) {
     if (event == null) {
       LOG.warn("procedure event for " + getProcId() +
-          " is null, maybe the procedure is created when recovery", new Exception());
+          " is null, maybe the procedure is created when recovery",
+        new Exception());
       return;
     }
-    LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer +
-        (succ ? " suceeded" : " failed"));
-    this.succ = succ;
+    if (error != null) {
+      LOG.warn("Refresh peer " + peerId + " for " + type + " on " + targetServer + " failed",
+        error);
+      this.succ = false;
+    } else {
+      LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer + " suceeded");
+      this.succ = true;
+    }
+
     event.wake(env.getProcedureScheduler());
     event = null;
   }
@@ -134,17 +143,18 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
   @Override
   public synchronized void remoteCallFailed(MasterProcedureEnv env, ServerName remote,
       IOException exception) {
-    complete(env, false);
+    complete(env, exception);
   }
 
   @Override
   public synchronized void remoteOperationCompleted(MasterProcedureEnv env) {
-    complete(env, true);
+    complete(env, null);
   }
 
   @Override
-  public synchronized void remoteOperationFailed(MasterProcedureEnv env, String error) {
-    complete(env, false);
+  public synchronized void remoteOperationFailed(MasterProcedureEnv env,
+      RemoteProcedureException error) {
+    complete(env, error);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
new file mode 100644
index 0000000..3daad6d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -0,0 +1,69 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for removing a replication peer.
+ */
+@InterfaceAudience.Private
+public class RemovePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(RemovePeerProcedure.class);
+
+  public RemovePeerProcedure() {
+  }
+
+  public RemovePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.REMOVE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preRemoveReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
+    env.getReplicationManager().removeReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully removed peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postRemoveReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/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 f36b2e2..b6f8784 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
@@ -27,10 +27,8 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
 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.hadoop.hbase.replication.BaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
@@ -39,24 +37,21 @@ 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.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Manages and performs all replication admin operations.
+ * <p>
  * Used to add/remove a replication peer.
  */
 @InterfaceAudience.Private
 public class ReplicationManager {
-
-  private final Configuration conf;
-  private final ZKWatcher zkw;
   private final ReplicationQueuesClient replicationQueuesClient;
   private final ReplicationPeers replicationPeers;
 
   public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable)
       throws IOException {
-    this.conf = conf;
-    this.zkw = zkw;
     try {
       this.replicationQueuesClient = ReplicationFactory
           .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw));
@@ -70,7 +65,7 @@ public class ReplicationManager {
   }
 
   public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException, IOException {
+      throws ReplicationException {
     checkPeerConfig(peerConfig);
     replicationPeers.registerPeer(peerId, peerConfig, enabled);
     replicationPeers.peerConnected(peerId);
@@ -89,8 +84,8 @@ public class ReplicationManager {
     this.replicationPeers.disablePeer(peerId);
   }
 
-  public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException,
-      ReplicationPeerNotFoundException {
+  public ReplicationPeerConfig getPeerConfig(String peerId)
+      throws ReplicationException, ReplicationPeerNotFoundException {
     ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(peerId);
     if (peerConfig == null) {
       throw new ReplicationPeerNotFoundException(peerId);
@@ -110,9 +105,9 @@ public class ReplicationManager {
     List<String> peerIds = replicationPeers.getAllPeerIds();
     for (String peerId : peerIds) {
       if (pattern == null || (pattern != null && pattern.matcher(peerId).matches())) {
-        peers.add(new ReplicationPeerDescription(peerId, replicationPeers
-            .getStatusOfPeerFromBackingStore(peerId), replicationPeers
-            .getReplicationPeerConfig(peerId)));
+        peers.add(new ReplicationPeerDescription(peerId,
+            replicationPeers.getStatusOfPeerFromBackingStore(peerId),
+            replicationPeers.getReplicationPeerConfig(peerId)));
       }
     }
     return peers;
@@ -126,13 +121,12 @@ public class ReplicationManager {
    * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
    * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
    */
-  private void checkPeerConfig(ReplicationPeerConfig peerConfig)
-      throws ReplicationException, IOException {
+  private void checkPeerConfig(ReplicationPeerConfig peerConfig) {
     if (peerConfig.replicateAllUserTables()) {
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty())
-          || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new ReplicationException("Need clean namespaces or table-cfs config firstly"
-            + " when replicate_all flag is true");
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
+        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new IllegalArgumentException("Need clean namespaces or table-cfs config firstly " +
+          "when you want replicate all cluster");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
         peerConfig.getExcludeTableCFsMap());
@@ -141,7 +135,7 @@ public class ReplicationManager {
           && !peerConfig.getExcludeNamespaces().isEmpty())
           || (peerConfig.getExcludeTableCFsMap() != null
               && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
-        throw new ReplicationException(
+        throw new IllegalArgumentException(
             "Need clean exclude-namespaces or exclude-table-cfs config firstly"
                 + " when replicate_all flag is false");
       }
@@ -154,20 +148,24 @@ public class ReplicationManager {
   /**
    * Set a namespace in the peer config means that all tables in this namespace will be replicated
    * to the peer cluster.
-   * 1. If peer config already has a namespace, then not allow set any table of this namespace
-   *    to the peer config.
-   * 2. If peer config already has a table, then not allow set this table's namespace to the peer
-   *    config.
-   *
+   * <ol>
+   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
+   * the peer config.</li>
+   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
+   * config.</li>
+   * </ol>
+   * <p>
    * Set a exclude namespace in the peer config means that all tables in this namespace can't be
    * replicated to the peer cluster.
-   * 1. If peer config already has a exclude namespace, then not allow set any exclude table of
-   *    this namespace to the peer config.
-   * 2. If peer config already has a exclude table, then not allow set this table's namespace
-   *    as a exclude namespace.
+   * <ol>
+   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
+   * this namespace to the peer config.</li>
+   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
+   * exclude namespace.</li>
+   * </ol>
    */
   private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
-      Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
+      Map<TableName, ? extends Collection<String>> tableCfs) {
     if (namespaces == null || namespaces.isEmpty()) {
       return;
     }
@@ -177,24 +175,22 @@ public class ReplicationManager {
     for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
       TableName table = entry.getKey();
       if (namespaces.contains(table.getNamespaceAsString())) {
-        throw new ReplicationException("Table-cfs " + table + " is conflict with namespaces "
+        throw new IllegalArgumentException("Table-cfs " + table + " is conflict with namespaces "
             + table.getNamespaceAsString() + " in peer config");
       }
     }
   }
 
-  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
-      throws IOException {
-    String filterCSV = peerConfig.getConfiguration().
-        get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
-    if (filterCSV != null && !filterCSV.isEmpty()){
-      String [] filters = filterCSV.split(",");
+  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) {
+    String filterCSV = peerConfig.getConfiguration()
+        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
+    if (filterCSV != null && !filterCSV.isEmpty()) {
+      String[] filters = filterCSV.split(",");
       for (String filter : filters) {
         try {
-          Class clazz = Class.forName(filter);
-          Object o = clazz.newInstance();
+          Class.forName(filter).newInstance();
         } catch (Exception e) {
-          throw new DoNotRetryIOException("Configured WALEntryFilter " + filter +
+          throw new IllegalArgumentException("Configured WALEntryFilter " + filter +
               " could not be created. Failing add/update " + "peer operation.", e);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
new file mode 100644
index 0000000..435eefc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -0,0 +1,92 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UpdatePeerConfigStateData;
+
+/**
+ * The procedure for updating the config for a replication peer.
+ */
+@InterfaceAudience.Private
+public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(UpdatePeerConfigProcedure.class);
+
+  private ReplicationPeerConfig peerConfig;
+
+  public UpdatePeerConfigProcedure() {
+  }
+
+  public UpdatePeerConfigProcedure(String peerId, ReplicationPeerConfig peerConfig) {
+    super(peerId);
+    this.peerConfig = peerConfig;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.UPDATE_CONFIG;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception {
+    env.getReplicationManager().updatePeerConfig(peerId, peerConfig);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully updated peer config of " + peerId + " to " + peerConfig);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(UpdatePeerConfigStateData.newBuilder()
+        .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    peerConfig = ReplicationPeerConfigUtil
+        .convert(serializer.deserialize(UpdatePeerConfigStateData.class).getPeerConfig());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/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 65cfd18..9ff6295 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
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -146,6 +146,7 @@ import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
 import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
@@ -3713,7 +3714,7 @@ public class HRegionServer extends HasThread implements
       ReportProcedureDoneRequest.newBuilder().setProcId(procId);
     if (error != null) {
       builder.setStatus(ReportProcedureDoneRequest.Status.ERROR)
-          .setError(Throwables.getStackTraceAsString(error));
+          .setError(ForeignExceptionUtil.toProtoForeignException(serverName.toString(), error));
     } else {
       builder.setStatus(ReportProcedureDoneRequest.Status.SUCCESS);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
new file mode 100644
index 0000000..a47a483
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
@@ -0,0 +1,70 @@
+/**
+ * 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.replication.regionserver;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+
+/**
+ * The callable executed at RS side to refresh the peer config/state.
+ * <p>
+ * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ */
+@InterfaceAudience.Private
+public class RefreshPeerCallable implements RSProcedureCallable {
+
+  private HRegionServer rs;
+
+  private String peerId;
+
+  private Exception initError;
+
+  @Override
+  public Void call() throws Exception {
+    if (initError != null) {
+      throw initError;
+    }
+    Path dir = new Path("/" + peerId);
+    if (rs.getFileSystem().exists(dir)) {
+      rs.getFileSystem().create(new Path(dir, rs.getServerName().toString())).close();
+    }
+    return null;
+  }
+
+  @Override
+  public void init(byte[] parameter, HRegionServer rs) {
+    this.rs = rs;
+    try {
+      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+    } catch (InvalidProtocolBufferException e) {
+      initError = e;
+      return;
+    }
+  }
+
+  @Override
+  public EventType getEventType() {
+    return EventType.RS_REFRESH_PEER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/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 7b6af0e..8442530 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -369,7 +370,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
 
   @Override
   public ClusterConnection getClusterConnection() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -399,20 +399,24 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
+    return 0;
   }
 
   @Override
-  public void removeReplicationPeer(String peerId) throws ReplicationException {
+  public long removeReplicationPeer(String peerId) throws ReplicationException {
+    return 0;
   }
 
   @Override
-  public void enableReplicationPeer(String peerId) throws ReplicationException, IOException {
+  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
-  public void disableReplicationPeer(String peerId) throws ReplicationException, IOException {
+  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
@@ -422,8 +426,9 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
@@ -458,7 +463,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
 
   @Override
   public ProcedureEvent getInitializedEvent() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -471,4 +475,9 @@ public class MockNoopMasterServices implements MasterServices, Server {
   public Connection createConnection(Configuration conf) throws IOException {
     return null;
   }
+
+  @Override
+  public ReplicationManager getReplicationManager() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1966abfb/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
index 44343d7..ed7c6fa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.replication;
 
-import java.io.IOException;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 
 public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
 
@@ -34,8 +34,15 @@ public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
-  protected boolean updatePeerStorage() throws IOException {
-    return true;
+  protected void prePeerModification(MasterProcedureEnv env) {
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) {
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) {
   }
 
 }


[09/48] hbase git commit: HBASE-19768 RegionServer startup failing when DN is dead

Posted by zh...@apache.org.
HBASE-19768 RegionServer startup failing when DN is dead


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: ffa28502c471945384845b0f072d1c468b0c7f31
Parents: 842f794
Author: zhangduo <zh...@apache.org>
Authored: Sun Jan 14 17:30:50 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Jan 14 17:31:23 2018 +0800

----------------------------------------------------------------------
 .../FanOutOneBlockAsyncDFSOutputHelper.java     | 140 ++++++++++++-------
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  65 ++-------
 .../TestFanOutOneBlockAsyncDFSOutput.java       |  78 ++++-------
 .../regionserver/wal/TestAsyncLogRolling.java   |   3 +-
 4 files changed, 126 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ffa28502/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
index 4fa06a4..d7aa897 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
@@ -21,23 +21,23 @@ import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper.createEncryptor;
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper.trySaslNegotiate;
-import static org.apache.hbase.thirdparty.io.netty.channel.ChannelOption.CONNECT_TIMEOUT_MILLIS;
-import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.READER_IDLE;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
 import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage.PIPELINE_SETUP_CREATE;
+import static org.apache.hbase.thirdparty.io.netty.channel.ChannelOption.CONNECT_TIMEOUT_MILLIS;
+import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.READER_IDLE;
 
 import com.google.protobuf.CodedOutputStream;
-
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
-
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.Encryptor;
@@ -85,6 +85,7 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
 import org.apache.hbase.thirdparty.io.netty.bootstrap.Bootstrap;
@@ -121,6 +122,9 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
   private FanOutOneBlockAsyncDFSOutputHelper() {
   }
 
+  public static final String ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES = "hbase.fs.async.create.retries";
+
+  public static final int DEFAULT_ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES = 10;
   // use pooled allocator for performance.
   private static final ByteBufAllocator ALLOC = PooledByteBufAllocator.DEFAULT;
 
@@ -129,8 +133,8 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
 
   // Timeouts for communicating with DataNode for streaming writes/reads
   public static final int READ_TIMEOUT = 60 * 1000;
-  public static final int READ_TIMEOUT_EXTENSION = 5 * 1000;
-  public static final int WRITE_TIMEOUT = 8 * 60 * 1000;
+
+  private static final DatanodeInfo[] EMPTY_DN_ARRAY = new DatanodeInfo[0];
 
   // helper class for getting Status from PipelineAckProto. In hadoop 2.6 or before, there is a
   // getStatus method, and for hadoop 2.7 or after, the status is retrieved from flag. The flag may
@@ -744,58 +748,90 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
     DFSClient client = dfs.getClient();
     String clientName = client.getClientName();
     ClientProtocol namenode = client.getNamenode();
-    HdfsFileStatus stat;
-    try {
-      stat = FILE_CREATOR.create(namenode, src,
-        FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
-        new EnumSetWritable<>(overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
-        createParent, replication, blockSize, CryptoProtocolVersion.supported());
-    } catch (Exception e) {
-      if (e instanceof RemoteException) {
-        throw (RemoteException) e;
-      } else {
-        throw new NameNodeException(e);
-      }
-    }
-    beginFileLease(client, stat.getFileId());
-    boolean succ = false;
-    LocatedBlock locatedBlock = null;
-    List<Future<Channel>> futureList = null;
-    try {
-      DataChecksum summer = createChecksum(client);
-      locatedBlock = BLOCK_ADDER.addBlock(namenode, src, client.getClientName(), null, null,
-        stat.getFileId(), null);
-      List<Channel> datanodeList = new ArrayList<>();
-      futureList = connectToDataNodes(conf, client, clientName, locatedBlock, 0L, 0L,
-        PIPELINE_SETUP_CREATE, summer, eventLoopGroup, channelClass);
-      for (Future<Channel> future : futureList) {
-        // fail the creation if there are connection failures since we are fail-fast. The upper
-        // layer should retry itself if needed.
-        datanodeList.add(future.syncUninterruptibly().getNow());
+    int createMaxRetries = conf.getInt(ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES,
+      DEFAULT_ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES);
+    DatanodeInfo[] excludesNodes = EMPTY_DN_ARRAY;
+    for (int retry = 0;; retry++) {
+      HdfsFileStatus stat;
+      try {
+        stat = FILE_CREATOR.create(namenode, src,
+          FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
+          new EnumSetWritable<>(overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
+          createParent, replication, blockSize, CryptoProtocolVersion.supported());
+      } catch (Exception e) {
+        if (e instanceof RemoteException) {
+          throw (RemoteException) e;
+        } else {
+          throw new NameNodeException(e);
+        }
       }
-      Encryptor encryptor = createEncryptor(conf, stat, client);
-      FanOutOneBlockAsyncDFSOutput output =
+      beginFileLease(client, stat.getFileId());
+      boolean succ = false;
+      LocatedBlock locatedBlock = null;
+      List<Future<Channel>> futureList = null;
+      try {
+        DataChecksum summer = createChecksum(client);
+        locatedBlock = BLOCK_ADDER.addBlock(namenode, src, client.getClientName(), null,
+          excludesNodes, stat.getFileId(), null);
+        List<Channel> datanodeList = new ArrayList<>();
+        futureList = connectToDataNodes(conf, client, clientName, locatedBlock, 0L, 0L,
+          PIPELINE_SETUP_CREATE, summer, eventLoopGroup, channelClass);
+        for (int i = 0, n = futureList.size(); i < n; i++) {
+          try {
+            datanodeList.add(futureList.get(i).syncUninterruptibly().getNow());
+          } catch (Exception e) {
+            // exclude the broken DN next time
+            excludesNodes = ArrayUtils.add(excludesNodes, locatedBlock.getLocations()[i]);
+            throw e;
+          }
+        }
+        Encryptor encryptor = createEncryptor(conf, stat, client);
+        FanOutOneBlockAsyncDFSOutput output =
           new FanOutOneBlockAsyncDFSOutput(conf, fsUtils, dfs, client, namenode, clientName, src,
               stat.getFileId(), locatedBlock, encryptor, datanodeList, summer, ALLOC);
-      succ = true;
-      return output;
-    } finally {
-      if (!succ) {
-        if (futureList != null) {
-          for (Future<Channel> f : futureList) {
-            f.addListener(new FutureListener<Channel>() {
-
-              @Override
-              public void operationComplete(Future<Channel> future) throws Exception {
-                if (future.isSuccess()) {
-                  future.getNow().close();
+        succ = true;
+        return output;
+      } catch (RemoteException e) {
+        LOG.warn("create fan-out dfs output {} failed, retry = {}", src, retry, e);
+        if (shouldRetryCreate(e)) {
+          if (retry >= createMaxRetries) {
+            throw e.unwrapRemoteException();
+          }
+        } else {
+          throw e.unwrapRemoteException();
+        }
+      } catch (NameNodeException e) {
+        throw e;
+      } catch (IOException e) {
+        LOG.warn("create fan-out dfs output {} failed, retry = {}", src, retry, e);
+        if (retry >= createMaxRetries) {
+          throw e;
+        }
+        // overwrite the old broken file.
+        overwrite = true;
+        try {
+          Thread.sleep(ConnectionUtils.getPauseTime(100, retry));
+        } catch (InterruptedException ie) {
+          throw new InterruptedIOException();
+        }
+      } finally {
+        if (!succ) {
+          if (futureList != null) {
+            for (Future<Channel> f : futureList) {
+              f.addListener(new FutureListener<Channel>() {
+
+                @Override
+                public void operationComplete(Future<Channel> future) throws Exception {
+                  if (future.isSuccess()) {
+                    future.getNow().close();
+                  }
                 }
-              }
-            });
+              });
+            }
           }
+          endFileLease(client, stat.getFileId());
+          fsUtils.recoverFileLease(dfs, new Path(src), conf, new CancelOnClose(client));
         }
-        endFileLease(client, stat.getFileId());
-        fsUtils.recoverFileLease(dfs, new Path(src), conf, new CancelOnClose(client));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffa28502/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index be8665b..0ace782 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -17,14 +17,10 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.shouldRetryCreate;
-
 import com.lmax.disruptor.RingBuffer;
 import com.lmax.disruptor.Sequence;
 import com.lmax.disruptor.Sequencer;
-
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.lang.reflect.Field;
 import java.util.ArrayDeque;
 import java.util.Comparator;
@@ -44,26 +40,23 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Supplier;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
-import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.NameNodeException;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.ipc.RemoteException;
 import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
@@ -140,9 +133,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   public static final String WAL_BATCH_SIZE = "hbase.wal.batch.size";
   public static final long DEFAULT_WAL_BATCH_SIZE = 64L * 1024;
 
-  public static final String ASYNC_WAL_CREATE_MAX_RETRIES = "hbase.wal.async.create.retries";
-  public static final int DEFAULT_ASYNC_WAL_CREATE_MAX_RETRIES = 10;
-
   public static final String ASYNC_WAL_USE_SHARED_EVENT_LOOP =
     "hbase.wal.async.use-shared-event-loop";
   public static final boolean DEFAULT_ASYNC_WAL_USE_SHARED_EVENT_LOOP = false;
@@ -189,8 +179,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
 
   private final long batchSize;
 
-  private final int createMaxRetries;
-
   private final ExecutorService closeExecutor = Executors.newCachedThreadPool(
     new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Close-WAL-Writer-%d").build());
 
@@ -257,8 +245,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     waitingConsumePayloadsGatingSequence.set(waitingConsumePayloads.getCursor());
 
     batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE);
-    createMaxRetries =
-      conf.getInt(ASYNC_WAL_CREATE_MAX_RETRIES, DEFAULT_ASYNC_WAL_CREATE_MAX_RETRIES);
     waitOnShutdownInSeconds = conf.getInt(ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS,
       DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS);
     rollWriter();
@@ -622,46 +608,19 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
 
   @Override
   protected AsyncWriter createWriterInstance(Path path) throws IOException {
-    boolean overwrite = false;
-    for (int retry = 0;; retry++) {
-      try {
-        return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, overwrite, eventLoopGroup,
-          channelClass);
-      } catch (RemoteException e) {
-        LOG.warn("create wal log writer " + path + " failed, retry = " + retry, e);
-        if (shouldRetryCreate(e)) {
-          if (retry >= createMaxRetries) {
-            break;
-          }
-        } else {
-          IOException ioe = e.unwrapRemoteException();
-          // this usually means master already think we are dead so let's fail all the pending
-          // syncs. The shutdown process of RS will wait for all regions to be closed before calling
-          // WAL.close so if we do not wake up the thread blocked by sync here it will cause dead
-          // lock.
-          if (e.getMessage().contains("Parent directory doesn't exist:")) {
-            syncFutures.forEach(f -> f.done(f.getTxid(), ioe));
-          }
-          throw ioe;
-        }
-      } catch (NameNodeException e) {
-        throw e;
-      } catch (IOException e) {
-        LOG.warn("create wal log writer " + path + " failed, retry = " + retry, e);
-        if (retry >= createMaxRetries) {
-          break;
-        }
-        // overwrite the old broken file.
-        overwrite = true;
-        try {
-          Thread.sleep(ConnectionUtils.getPauseTime(100, retry));
-        } catch (InterruptedException ie) {
-          throw new InterruptedIOException();
-        }
+    try {
+      return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup,
+        channelClass);
+    } catch (IOException e) {
+      // this usually means master already think we are dead so let's fail all the pending
+      // syncs. The shutdown process of RS will wait for all regions to be closed before calling
+      // WAL.close so if we do not wake up the thread blocked by sync here it will cause dead
+      // lock.
+      if (e.getMessage().contains("Parent directory doesn't exist:")) {
+        syncFutures.forEach(f -> f.done(f.getTxid(), e));
       }
+      throw e;
     }
-    throw new IOException("Failed to create wal log writer " + path + " after retrying " +
-      createMaxRetries + " time(s)");
   }
 
   private void waitForSafePoint() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffa28502/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java
index 3a8da21..42539c2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java
@@ -35,7 +35,6 @@ import java.util.Random;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
-
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,9 +42,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.Daemon;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -54,6 +53,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
@@ -95,23 +95,6 @@ public class TestFanOutOneBlockAsyncDFSOutput {
     TEST_UTIL.shutdownMiniDFSCluster();
   }
 
-  private void ensureAllDatanodeAlive() throws InterruptedException {
-    // FanOutOneBlockAsyncDFSOutputHelper.createOutput is fail-fast, so we need to make sure that we
-    // can create a FanOutOneBlockAsyncDFSOutput after a datanode restarting, otherwise some tests
-    // will fail.
-    for (;;) {
-      try {
-        FanOutOneBlockAsyncDFSOutput out =
-            FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, new Path("/ensureDatanodeAlive"),
-              true, true, (short) 3, FS.getDefaultBlockSize(), EVENT_LOOP_GROUP, CHANNEL_CLASS);
-        out.close();
-        break;
-      } catch (IOException e) {
-        Thread.sleep(100);
-      }
-    }
-  }
-
   static void writeAndVerify(FileSystem fs, Path f, AsyncFSOutput out)
       throws IOException, InterruptedException, ExecutionException {
     List<CompletableFuture<Long>> futures = new ArrayList<>();
@@ -163,25 +146,21 @@ public class TestFanOutOneBlockAsyncDFSOutput {
     out.flush(false).get();
     // restart one datanode which causes one connection broken
     TEST_UTIL.getDFSCluster().restartDataNode(0);
+    out.write(b, 0, b.length);
     try {
-      out.write(b, 0, b.length);
-      try {
-        out.flush(false).get();
-        fail("flush should fail");
-      } catch (ExecutionException e) {
-        // we restarted one datanode so the flush should fail
-        LOG.info("expected exception caught", e);
-      }
-      out.recoverAndClose(null);
-      assertEquals(b.length, FS.getFileStatus(f).getLen());
-      byte[] actual = new byte[b.length];
-      try (FSDataInputStream in = FS.open(f)) {
-        in.readFully(actual);
-      }
-      assertArrayEquals(b, actual);
-    } finally {
-      ensureAllDatanodeAlive();
+      out.flush(false).get();
+      fail("flush should fail");
+    } catch (ExecutionException e) {
+      // we restarted one datanode so the flush should fail
+      LOG.info("expected exception caught", e);
+    }
+    out.recoverAndClose(null);
+    assertEquals(b.length, FS.getFileStatus(f).getLen());
+    byte[] actual = new byte[b.length];
+    try (FSDataInputStream in = FS.open(f)) {
+      in.readFully(actual);
     }
+    assertArrayEquals(b, actual);
   }
 
   @Test
@@ -219,28 +198,19 @@ public class TestFanOutOneBlockAsyncDFSOutput {
     Field xceiverServerDaemonField = DataNode.class.getDeclaredField("dataXceiverServer");
     xceiverServerDaemonField.setAccessible(true);
     Class<?> xceiverServerClass =
-        Class.forName("org.apache.hadoop.hdfs.server.datanode.DataXceiverServer");
+      Class.forName("org.apache.hadoop.hdfs.server.datanode.DataXceiverServer");
     Method numPeersMethod = xceiverServerClass.getDeclaredMethod("getNumPeers");
     numPeersMethod.setAccessible(true);
     // make one datanode broken
-    TEST_UTIL.getDFSCluster().getDataNodes().get(0).shutdownDatanode(true);
-    try {
-      Path f = new Path("/test");
-      EventLoop eventLoop = EVENT_LOOP_GROUP.next();
-      try {
-        FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
-          FS.getDefaultBlockSize(), eventLoop, CHANNEL_CLASS);
-        fail("should fail with connection error");
-      } catch (IOException e) {
-        LOG.info("expected exception caught", e);
-      }
-      for (DataNode dn : TEST_UTIL.getDFSCluster().getDataNodes()) {
-        Daemon daemon = (Daemon) xceiverServerDaemonField.get(dn);
-        assertEquals(0, numPeersMethod.invoke(daemon.getRunnable()));
-      }
+    DataNodeProperties dnProp = TEST_UTIL.getDFSCluster().stopDataNode(0);
+    Path f = new Path("/test");
+    EventLoop eventLoop = EVENT_LOOP_GROUP.next();
+    try (FanOutOneBlockAsyncDFSOutput output = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS,
+      f, true, false, (short) 3, FS.getDefaultBlockSize(), eventLoop, CHANNEL_CLASS)) {
+      // should exclude the dead dn when retry so here we only have 2 DNs in pipeline
+      assertEquals(2, output.getPipeline().length);
     } finally {
-      TEST_UTIL.getDFSCluster().restartDataNode(0);
-      ensureAllDatanodeAlive();
+      TEST_UTIL.getDFSCluster().restartDataNode(dnProp);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffa28502/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
index 325da94..01e44fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
@@ -45,7 +46,7 @@ public class TestAsyncLogRolling extends AbstractTestLogRolling {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     Configuration conf = TestAsyncLogRolling.TEST_UTIL.getConfiguration();
-    conf.setInt(AsyncFSWAL.ASYNC_WAL_CREATE_MAX_RETRIES, 100);
+    conf.setInt(FanOutOneBlockAsyncDFSOutputHelper.ASYNC_DFS_OUTPUT_CREATE_MAX_RETRIES, 100);
     conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
     conf.set(WALFactory.META_WAL_PROVIDER, "asyncfs");
     AbstractTestLogRolling.setUpBeforeClass();


[47/48] hbase git commit: HBASE-19687 Move the logic in ReplicationZKNodeCleaner to ReplicationChecker and remove ReplicationZKNodeCleanerChore

Posted by zh...@apache.org.
HBASE-19687 Move the logic in ReplicationZKNodeCleaner to ReplicationChecker and remove ReplicationZKNodeCleanerChore


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: eb0bb207fd5fe4f7c0b6b0a3012f8482bee4091a
Parents: 6b7bed4
Author: zhangduo <zh...@apache.org>
Authored: Wed Jan 3 09:39:44 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../replication/VerifyReplication.java          |   6 +-
 .../hbase/replication/ReplicationPeers.java     |  26 +--
 .../hbase/replication/ReplicationUtils.java     |  69 +++++++
 .../replication/TestReplicationStateBasic.java  |   2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  13 --
 .../cleaner/ReplicationZKNodeCleaner.java       | 192 -------------------
 .../cleaner/ReplicationZKNodeCleanerChore.java  |  54 ------
 .../replication/ReplicationPeerManager.java     |  18 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  13 +-
 .../hbase/util/hbck/ReplicationChecker.java     | 109 +++++++----
 .../cleaner/TestReplicationZKNodeCleaner.java   | 109 -----------
 .../hbase/util/TestHBaseFsckReplication.java    | 101 ++++++++++
 .../hadoop/hbase/util/hbck/HbckTestingUtil.java |   6 +-
 13 files changed, 259 insertions(+), 459 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/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 fe45762..fac4875 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
@@ -50,8 +50,8 @@ import org.apache.hadoop.hbase.mapreduce.TableSplit;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -345,10 +345,10 @@ public class VerifyReplication extends Configured implements Tool {
         }
       });
       ReplicationPeerStorage storage =
-          ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf);
+        ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf);
       ReplicationPeerConfig peerConfig = storage.getPeerConfig(peerId);
       return Pair.newPair(peerConfig,
-        ReplicationPeers.getPeerClusterConfiguration(peerConfig, conf));
+        ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf));
     } catch (ReplicationException e) {
       throw new IOException("An error occurred while trying to connect to the remove peer cluster",
           e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index 45940a5..fcbc350 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -17,14 +17,11 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CompoundConfiguration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -106,25 +103,6 @@ public class ReplicationPeers {
     return Collections.unmodifiableSet(peerCache.keySet());
   }
 
-  public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,
-      Configuration baseConf) throws ReplicationException {
-    Configuration otherConf;
-    try {
-      otherConf = HBaseConfiguration.createClusterConf(baseConf, peerConfig.getClusterKey());
-    } catch (IOException e) {
-      throw new ReplicationException("Can't get peer configuration for peer " + peerConfig, e);
-    }
-
-    if (!peerConfig.getConfiguration().isEmpty()) {
-      CompoundConfiguration compound = new CompoundConfiguration();
-      compound.add(otherConf);
-      compound.addStringMap(peerConfig.getConfiguration());
-      return compound;
-    }
-
-    return otherConf;
-  }
-
   public PeerState refreshPeerState(String peerId) throws ReplicationException {
     ReplicationPeerImpl peer = peerCache.get(peerId);
     if (peer == null) {
@@ -158,7 +136,7 @@ public class ReplicationPeers {
   private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
     ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
     boolean enabled = peerStorage.isPeerEnabled(peerId);
-    return new ReplicationPeerImpl(getPeerClusterConfiguration(peerConfig, conf), peerId, enabled,
-        peerConfig);
+    return new ReplicationPeerImpl(ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf),
+        peerId, enabled, peerConfig);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
new file mode 100644
index 0000000..7b676ca
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
@@ -0,0 +1,69 @@
+/**
+ * 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.replication;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompoundConfiguration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Helper class for replication.
+ */
+@InterfaceAudience.Private
+public final class ReplicationUtils {
+
+  private ReplicationUtils() {
+  }
+
+  public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,
+      Configuration baseConf) throws ReplicationException {
+    Configuration otherConf;
+    try {
+      otherConf = HBaseConfiguration.createClusterConf(baseConf, peerConfig.getClusterKey());
+    } catch (IOException e) {
+      throw new ReplicationException("Can't get peer configuration for peer " + peerConfig, e);
+    }
+
+    if (!peerConfig.getConfiguration().isEmpty()) {
+      CompoundConfiguration compound = new CompoundConfiguration();
+      compound.add(otherConf);
+      compound.addStringMap(peerConfig.getConfiguration());
+      return compound;
+    }
+
+    return otherConf;
+  }
+
+  public static void removeAllQueues(ReplicationQueueStorage queueStorage, String peerId)
+      throws ReplicationException {
+    for (ServerName replicator : queueStorage.getListOfReplicators()) {
+      List<String> queueIds = queueStorage.getAllQueues(replicator);
+      for (String queueId : queueIds) {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        if (queueInfo.getPeerId().equals(peerId)) {
+          queueStorage.removeQueue(replicator, queueId);
+        }
+      }
+      queueStorage.removeReplicatorIfQueueIsEmpty(replicator);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index f3eeccc..fccffb5 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -240,7 +240,7 @@ public abstract class TestReplicationStateBasic {
     rp.getPeerStorage().addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), true);
     assertNumberOfPeers(2);
 
-    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationPeers
+    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationUtils
         .getPeerClusterConfiguration(rp.getPeerStorage().getPeerConfig(ID_ONE), rp.getConf())));
     rp.getPeerStorage().removePeer(ID_ONE);
     rp.removePeer(ID_ONE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/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 0e8ef3b..df5e466 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
@@ -109,8 +109,6 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.cleaner.ReplicationMetaCleaner;
-import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
-import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleanerChore;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
@@ -364,7 +362,6 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   CatalogJanitor catalogJanitorChore;
   private ReplicationMetaCleaner replicationMetaCleaner;
-  private ReplicationZKNodeCleanerChore replicationZKNodeCleanerChore;
   private LogCleaner logCleaner;
   private HFileCleaner hfileCleaner;
   private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
@@ -1176,15 +1173,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Started service threads");
     }
-
-    // Start replication zk node cleaner
-    try {
-      replicationZKNodeCleanerChore = new ReplicationZKNodeCleanerChore(this, cleanerInterval,
-          new ReplicationZKNodeCleaner(this.conf, this.getZooKeeper(), this));
-      getChoreService().scheduleChore(replicationZKNodeCleanerChore);
-    } catch (Exception e) {
-      LOG.error("start replicationZKNodeCleanerChore failed", e);
-    }
     replicationMetaCleaner = new ReplicationMetaCleaner(this, this, cleanerInterval);
     getChoreService().scheduleChore(replicationMetaCleaner);
   }
@@ -1209,7 +1197,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     // Clean up and close up shop
     if (this.logCleaner != null) this.logCleaner.cancel(true);
     if (this.hfileCleaner != null) this.hfileCleaner.cancel(true);
-    if (this.replicationZKNodeCleanerChore != null) this.replicationZKNodeCleanerChore.cancel(true);
     if (this.replicationMetaCleaner != null) this.replicationMetaCleaner.cancel(true);
     if (this.quotaManager != null) this.quotaManager.stop();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/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
deleted file mode 100644
index f2c3ec9..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
+++ /dev/null
@@ -1,192 +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.master.cleaner;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
-import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
-import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
-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.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Used to clean the replication queues belonging to the peer which does not exist.
- */
-@InterfaceAudience.Private
-public class ReplicationZKNodeCleaner {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleaner.class);
-  private final ReplicationQueueStorage queueStorage;
-  private final ReplicationPeerStorage peerStorage;
-  private final ReplicationQueueDeletor queueDeletor;
-
-  public ReplicationZKNodeCleaner(Configuration conf, ZKWatcher zkw, Abortable abortable)
-      throws IOException {
-    this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
-    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf);
-    this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, abortable);
-  }
-
-  /**
-   * @return undeletedQueues replicator with its queueIds for removed peers
-   * @throws IOException
-   */
-  public Map<ServerName, List<String>> getUnDeletedQueues() throws IOException {
-    Map<ServerName, List<String>> undeletedQueues = new HashMap<>();
-    try {
-      Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds());
-      List<ServerName> replicators = this.queueStorage.getListOfReplicators();
-      if (replicators == null || replicators.isEmpty()) {
-        return undeletedQueues;
-      }
-      for (ServerName replicator : replicators) {
-        List<String> queueIds = this.queueStorage.getAllQueues(replicator);
-        for (String queueId : queueIds) {
-          ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-          if (!peerIds.contains(queueInfo.getPeerId())) {
-            undeletedQueues.computeIfAbsent(replicator, (key) -> new ArrayList<>()).add(queueId);
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Undeleted replication queue for removed peer found: "
-                  + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]",
-                    queueInfo.getPeerId(), replicator, queueId));
-            }
-          }
-        }
-      }
-    } catch (ReplicationException ke) {
-      throw new IOException("Failed to get the replication queues of all replicators", ke);
-    }
-    return undeletedQueues;
-  }
-
-  /**
-   * @return undeletedHFileRefsQueue replicator with its undeleted queueIds for removed peers in
-   *         hfile-refs queue
-   */
-  public Set<String> getUnDeletedHFileRefsQueues() throws IOException {
-    Set<String> undeletedHFileRefsQueue = new HashSet<>();
-    String hfileRefsZNode = queueDeletor.getHfileRefsZNode();
-    try {
-      Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds());
-      List<String> listOfPeers = this.queueStorage.getAllPeersFromHFileRefsQueue();
-      Set<String> peers = new HashSet<>(listOfPeers);
-      peers.removeAll(peerIds);
-      if (!peers.isEmpty()) {
-        undeletedHFileRefsQueue.addAll(peers);
-      }
-    } catch (ReplicationException e) {
-      throw new IOException("Failed to get list of all peers from hfile-refs znode "
-          + hfileRefsZNode, e);
-    }
-    return undeletedHFileRefsQueue;
-  }
-
-  private class ReplicationQueueDeletor extends ReplicationStateZKBase {
-
-    ReplicationQueueDeletor(ZKWatcher zk, Configuration conf, Abortable abortable) {
-      super(zk, conf, abortable);
-    }
-
-    /**
-     * @param replicator The regionserver which has undeleted queue
-     * @param queueId The undeleted queue id
-     */
-    void removeQueue(final ServerName replicator, final String queueId) throws IOException {
-      String queueZnodePath =
-          ZNodePaths.joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator.getServerName()),
-            queueId);
-      try {
-        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-        if (!peerStorage.listPeerIds().contains(queueInfo.getPeerId())) {
-          ZKUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath);
-          LOG.info("Successfully removed replication queue, replicator: " + replicator
-              + ", queueId: " + queueId);
-        }
-      } catch (ReplicationException | KeeperException e) {
-        throw new IOException("Failed to delete queue, replicator: " + replicator + ", queueId: "
-            + queueId);
-      }
-    }
-
-    /**
-     * @param hfileRefsQueueId The undeleted hfile-refs queue id
-     * @throws IOException
-     */
-    void removeHFileRefsQueue(final String hfileRefsQueueId) throws IOException {
-      String node = ZNodePaths.joinZNode(this.hfileRefsZNode, hfileRefsQueueId);
-      try {
-        if (!peerStorage.listPeerIds().contains(hfileRefsQueueId)) {
-          ZKUtil.deleteNodeRecursively(this.zookeeper, node);
-          LOG.info("Successfully removed hfile-refs queue " + hfileRefsQueueId + " from path "
-              + hfileRefsZNode);
-        }
-      } catch (ReplicationException | KeeperException e) {
-        throw new IOException("Failed to delete hfile-refs queue " + hfileRefsQueueId
-            + " from path " + hfileRefsZNode, e);
-      }
-    }
-
-    String getHfileRefsZNode() {
-      return this.hfileRefsZNode;
-    }
-  }
-
-  /**
-   * Remove the undeleted replication queue's zk node for removed peers.
-   * @param undeletedQueues replicator with its queueIds for removed peers
-   * @throws IOException
-   */
-  public void removeQueues(final Map<ServerName, List<String>> undeletedQueues) throws IOException {
-    for (Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueues.entrySet()) {
-      ServerName replicator = replicatorAndQueueIds.getKey();
-      for (String queueId : replicatorAndQueueIds.getValue()) {
-        queueDeletor.removeQueue(replicator, queueId);
-      }
-    }
-  }
-
-  /**
-   * Remove the undeleted hfile-refs queue's zk node for removed peers.
-   * @param undeletedHFileRefsQueues replicator with its undeleted queueIds for removed peers in
-   *          hfile-refs queue
-   * @throws IOException
-   */
-  public void removeHFileRefsQueues(final Set<String> undeletedHFileRefsQueues) throws IOException {
-    for (String hfileRefsQueueId : undeletedHFileRefsQueues) {
-      queueDeletor.removeHFileRefsQueue(hfileRefsQueueId);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
deleted file mode 100644
index 19ca804..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
+++ /dev/null
@@ -1,54 +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.master.cleaner;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Chore that will clean the replication queues belonging to the peer which does not exist.
- */
-@InterfaceAudience.Private
-public class ReplicationZKNodeCleanerChore extends ScheduledChore {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleanerChore.class);
-  private final ReplicationZKNodeCleaner cleaner;
-
-  public ReplicationZKNodeCleanerChore(Stoppable stopper, int period,
-      ReplicationZKNodeCleaner cleaner) {
-    super("ReplicationZKNodeCleanerChore", stopper, period);
-    this.cleaner = cleaner;
-  }
-
-  @Override
-  protected void chore() {
-    try {
-      Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
-      cleaner.removeQueues(undeletedQueues);
-    } catch (IOException e) {
-      LOG.warn("Failed to clean replication zk node", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index 1414d22..696b2d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -216,19 +217,6 @@ public class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
-  private void removeAllQueues0(String peerId) throws ReplicationException {
-    for (ServerName replicator : queueStorage.getListOfReplicators()) {
-      List<String> queueIds = queueStorage.getAllQueues(replicator);
-      for (String queueId : queueIds) {
-        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-        if (queueInfo.getPeerId().equals(peerId)) {
-          queueStorage.removeQueue(replicator, queueId);
-        }
-      }
-      queueStorage.removeReplicatorIfQueueIsEmpty(replicator);
-    }
-  }
-
   public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
     // Here we need two passes to address the problem of claimQueue. Maybe a claimQueue is still
     // on-going when the refresh peer config procedure is done, if a RS which has already been
@@ -241,8 +229,8 @@ public class ReplicationPeerManager {
     // claimed once after the refresh peer procedure done(as the next claim queue will just delete
     // it), so we can make sure that a two pass scan will finally find the queue and remove it,
     // unless it has already been removed by others.
-    removeAllQueues0(peerId);
-    removeAllQueues0(peerId);
+    ReplicationUtils.removeAllQueues(queueStorage, peerId);
+    ReplicationUtils.removeAllQueues(queueStorage, peerId);
     queueStorage.removePeerFromHFileRefs(peerId);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/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 d0276c0..8b5a4e6 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
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
@@ -142,6 +143,7 @@ import org.apache.yetus.audience.InterfaceStability;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
@@ -149,6 +151,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Ordering;
 import org.apache.hbase.thirdparty.com.google.common.collect.TreeMultimap;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
 
@@ -752,7 +755,7 @@ public class HBaseFsck extends Configured implements Closeable {
    * @return 0 on success, non-zero on failure
    */
   public int onlineHbck()
-      throws IOException, KeeperException, InterruptedException {
+      throws IOException, KeeperException, InterruptedException, ReplicationException {
     // print hbase server version
     errors.print("Version: " + status.getHBaseVersion());
 
@@ -3572,8 +3575,8 @@ public class HBaseFsck extends Configured implements Closeable {
     return hbi;
   }
 
-  private void checkAndFixReplication() throws IOException {
-    ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, connection, errors);
+  private void checkAndFixReplication() throws ReplicationException {
+    ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, errors);
     checker.checkUnDeletedQueues();
 
     if (checker.hasUnDeletedQueues() && this.fixReplication) {
@@ -4861,8 +4864,8 @@ public class HBaseFsck extends Configured implements Closeable {
   };
 
 
-  public HBaseFsck exec(ExecutorService exec, String[] args) throws KeeperException, IOException,
-      InterruptedException {
+  public HBaseFsck exec(ExecutorService exec, String[] args)
+      throws KeeperException, IOException, InterruptedException, ReplicationException {
     long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN;
 
     boolean checkCorruptHFiles = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/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 85fa729..c08c654 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
@@ -17,84 +17,115 @@
  */
 package org.apache.hadoop.hbase.util.hbck;
 
-import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Check and fix undeleted replication queues for removed peerId.
  */
 @InterfaceAudience.Private
 public class ReplicationChecker {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReplicationChecker.class);
+
   private final ErrorReporter errorReporter;
   // replicator with its queueIds for removed peers
   private Map<ServerName, List<String>> undeletedQueueIds = new HashMap<>();
   // replicator with its undeleted queueIds for removed peers in hfile-refs queue
-  private Set<String> undeletedHFileRefsQueueIds = new HashSet<>();
-  private final ReplicationZKNodeCleaner cleaner;
+  private Set<String> undeletedHFileRefsPeerIds = new HashSet<>();
 
-  public ReplicationChecker(Configuration conf, ZKWatcher zkw, ClusterConnection connection,
-                            ErrorReporter errorReporter) throws IOException {
-    this.cleaner = new ReplicationZKNodeCleaner(conf, zkw, connection);
+  private final ReplicationPeerStorage peerStorage;
+  private final ReplicationQueueStorage queueStorage;
+
+  public ReplicationChecker(Configuration conf, ZKWatcher zkw, ErrorReporter errorReporter) {
+    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf);
+    this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
     this.errorReporter = errorReporter;
   }
 
   public boolean hasUnDeletedQueues() {
-    return errorReporter.getErrorList().contains(
-      HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE);
+    return errorReporter.getErrorList()
+        .contains(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE);
   }
 
-  public void checkUnDeletedQueues() throws IOException {
-    undeletedQueueIds = cleaner.getUnDeletedQueues();
-    for (Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueueIds.entrySet()) {
-      ServerName replicator = replicatorAndQueueIds.getKey();
-      for (String queueId : replicatorAndQueueIds.getValue()) {
+  private Map<ServerName, List<String>> getUnDeletedQueues() throws ReplicationException {
+    Map<ServerName, List<String>> undeletedQueues = new HashMap<>();
+    Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds());
+    for (ServerName replicator : queueStorage.getListOfReplicators()) {
+      for (String queueId : queueStorage.getAllQueues(replicator)) {
         ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-        String msg = "Undeleted replication queue for removed peer found: "
-            + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]", queueInfo.getPeerId(),
-              replicator, queueId);
-        errorReporter.reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE,
-          msg);
+        if (!peerIds.contains(queueInfo.getPeerId())) {
+          undeletedQueues.computeIfAbsent(replicator, key -> new ArrayList<>()).add(queueId);
+          LOG.debug(
+            "Undeleted replication queue for removed peer found: " +
+              "[removedPeerId={}, replicator={}, queueId={}]",
+            queueInfo.getPeerId(), replicator, queueId);
+        }
       }
     }
-
-    checkUnDeletedHFileRefsQueues();
+    return undeletedQueues;
   }
 
-  private void checkUnDeletedHFileRefsQueues() throws IOException {
-    undeletedHFileRefsQueueIds = cleaner.getUnDeletedHFileRefsQueues();
-    if (undeletedHFileRefsQueueIds != null && !undeletedHFileRefsQueueIds.isEmpty()) {
-      String msg = "Undeleted replication hfile-refs queue for removed peer found: "
-          + undeletedHFileRefsQueueIds + " under hfile-refs node";
-      errorReporter
-          .reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, msg);
+  private Set<String> getUndeletedHFileRefsPeers() throws ReplicationException {
+    Set<String> undeletedHFileRefsPeerIds =
+      new HashSet<>(queueStorage.getAllPeersFromHFileRefsQueue());
+    Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds());
+    undeletedHFileRefsPeerIds.removeAll(peerIds);
+    if (LOG.isDebugEnabled()) {
+      for (String peerId : undeletedHFileRefsPeerIds) {
+        LOG.debug("Undeleted replication hfile-refs queue for removed peer {} found", peerId);
+      }
     }
+    return undeletedHFileRefsPeerIds;
   }
 
-  public void fixUnDeletedQueues() throws IOException {
-    if (!undeletedQueueIds.isEmpty()) {
-      cleaner.removeQueues(undeletedQueueIds);
-    }
-    fixUnDeletedHFileRefsQueue();
+  public void checkUnDeletedQueues() throws ReplicationException {
+    undeletedQueueIds = getUnDeletedQueues();
+    undeletedQueueIds.forEach((replicator, queueIds) -> {
+      queueIds.forEach(queueId -> {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        String msg = "Undeleted replication queue for removed peer found: " +
+          String.format("[removedPeerId=%s, replicator=%s, queueId=%s]", queueInfo.getPeerId(),
+            replicator, queueId);
+        errorReporter.reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE,
+          msg);
+      });
+    });
+    undeletedHFileRefsPeerIds = getUndeletedHFileRefsPeers();
+    undeletedHFileRefsPeerIds.stream()
+        .map(
+          peerId -> "Undeleted replication hfile-refs queue for removed peer " + peerId + " found")
+        .forEach(msg -> errorReporter
+            .reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, msg));
   }
 
-  private void fixUnDeletedHFileRefsQueue() throws IOException {
-    if (undeletedHFileRefsQueueIds != null && !undeletedHFileRefsQueueIds.isEmpty()) {
-      cleaner.removeHFileRefsQueues(undeletedHFileRefsQueueIds);
+  public void fixUnDeletedQueues() throws ReplicationException {
+    for (Map.Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueueIds.entrySet()) {
+      ServerName replicator = replicatorAndQueueIds.getKey();
+      for (String queueId : replicatorAndQueueIds.getValue()) {
+        queueStorage.removeQueue(replicator, queueId);
+      }
+      queueStorage.removeReplicatorIfQueueIsEmpty(replicator);
+    }
+    for (String peerId : undeletedHFileRefsPeerIds) {
+      queueStorage.removePeerFromHFileRefs(peerId);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/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
deleted file mode 100644
index 2ad8bd7..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
+++ /dev/null
@@ -1,109 +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.master.cleaner;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ MasterTests.class, MediumTests.class })
-public class TestReplicationZKNodeCleaner {
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  private final String ID_ONE = "1";
-  private final ServerName SERVER_ONE = ServerName.valueOf("server1", 8000, 1234);
-  private final String ID_TWO = "2";
-  private final ServerName SERVER_TWO = ServerName.valueOf("server2", 8000, 1234);
-
-  private final Configuration conf;
-  private final ZKWatcher zkw;
-  private final ReplicationQueueStorage repQueues;
-
-  public TestReplicationZKNodeCleaner() throws Exception {
-    conf = TEST_UTIL.getConfiguration();
-    zkw = new ZKWatcher(conf, "TestReplicationZKNodeCleaner", null);
-    repQueues = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setInt("hbase.master.cleaner.interval", 10000);
-    TEST_UTIL.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void testReplicationZKNodeCleaner() throws Exception {
-    // add queue for ID_ONE which isn't exist
-    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
-
-    ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
-    Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
-    assertEquals(1, undeletedQueues.size());
-    assertTrue(undeletedQueues.containsKey(SERVER_ONE));
-    assertEquals(1, undeletedQueues.get(SERVER_ONE).size());
-    assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE));
-
-    // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
-
-    undeletedQueues = cleaner.getUnDeletedQueues();
-    assertEquals(1, undeletedQueues.size());
-    assertTrue(undeletedQueues.containsKey(SERVER_ONE));
-    assertEquals(2, undeletedQueues.get(SERVER_ONE).size());
-    assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE));
-    assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_TWO + "-" + SERVER_TWO));
-
-    cleaner.removeQueues(undeletedQueues);
-    undeletedQueues = cleaner.getUnDeletedQueues();
-    assertEquals(0, undeletedQueues.size());
-  }
-
-  @Test
-  public void testReplicationZKNodeCleanerChore() throws Exception {
-    // add queue for ID_ONE which isn't exist
-    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
-    // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
-
-    // Wait the cleaner chore to run
-    Thread.sleep(20000);
-
-    ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
-    assertEquals(0, cleaner.getUnDeletedQueues().size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
new file mode 100644
index 0000000..e64255c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
@@ -0,0 +1,101 @@
+/**
+ * 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.util;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+import java.util.stream.Stream;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
+import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestHBaseFsckReplication {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    ReplicationPeerStorage peerStorage = ReplicationStorageFactory
+        .getReplicationPeerStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+    ReplicationQueueStorage queueStorage = ReplicationStorageFactory
+        .getReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+
+    String peerId1 = "1";
+    String peerId2 = "2";
+    peerStorage.addPeer(peerId1, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(),
+      true);
+    peerStorage.addPeer(peerId2, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(),
+      true);
+    for (int i = 0; i < 10; i++) {
+      queueStorage.addWAL(ServerName.valueOf("localhost", 10000 + i, 100000 + i), peerId1,
+        "file-" + i);
+    }
+    queueStorage.addWAL(ServerName.valueOf("localhost", 10000, 100000), peerId2, "file");
+    HBaseFsck fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), true);
+    HbckTestingUtil.assertNoErrors(fsck);
+
+    // should not remove anything since the replication peer is still alive
+    assertEquals(10, queueStorage.getListOfReplicators().size());
+    peerStorage.removePeer(peerId1);
+    // there should be orphan queues
+    assertEquals(10, queueStorage.getListOfReplicators().size());
+    fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), false);
+    HbckTestingUtil.assertErrors(fsck, Stream.generate(() -> {
+      return ERROR_CODE.UNDELETED_REPLICATION_QUEUE;
+    }).limit(10).toArray(ERROR_CODE[]::new));
+
+    // should not delete anything when fix is false
+    assertEquals(10, queueStorage.getListOfReplicators().size());
+
+    fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), true);
+    HbckTestingUtil.assertErrors(fsck, Stream.generate(() -> {
+      return ERROR_CODE.UNDELETED_REPLICATION_QUEUE;
+    }).limit(10).toArray(ERROR_CODE[]::new));
+
+    List<ServerName> replicators = queueStorage.getListOfReplicators();
+    // should not remove the server with queue for peerId2
+    assertEquals(1, replicators.size());
+    assertEquals(ServerName.valueOf("localhost", 10000, 100000), replicators.get(0));
+    for (String queueId : queueStorage.getAllQueues(replicators.get(0))) {
+      assertEquals(peerId2, queueId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb0bb207/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
index 60d7324..99e4f08 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
@@ -46,7 +46,7 @@ public class HbckTestingUtil {
   public static HBaseFsck doFsck(Configuration conf, boolean fixAssignments, boolean fixMeta,
       boolean fixHdfsHoles, boolean fixHdfsOverlaps, boolean fixHdfsOrphans,
       boolean fixTableOrphans, boolean fixVersionFile, boolean fixReferenceFiles, boolean fixHFileLinks,
-      boolean fixEmptyMetaRegionInfo, boolean fixTableLocks, Boolean fixReplication,
+      boolean fixEmptyMetaRegionInfo, boolean fixTableLocks, boolean fixReplication,
       TableName table) throws Exception {
     HBaseFsck fsck = new HBaseFsck(conf, exec);
     try {
@@ -78,10 +78,8 @@ public class HbckTestingUtil {
 
   /**
    * Runs hbck with the -sidelineCorruptHFiles option
-   * @param conf
    * @param table table constraint
-   * @return <returncode, hbckInstance>
-   * @throws Exception
+   * @return hbckInstance
    */
   public static HBaseFsck doHFileQuarantine(Configuration conf, TableName table) throws Exception {
     String[] args = {"-sidelineCorruptHFiles", "-ignorePreCheckPermission", table.getNameAsString()};


[31/48] hbase git commit: HBASE-19635 Introduce a thread at RS side to call reportProcedureDone

Posted by zh...@apache.org.
HBASE-19635 Introduce a thread at RS side to call reportProcedureDone


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 8d0b8b9d1447b723122bad7a801f58071954e44e
Parents: e504f1d
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 27 20:13:42 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/RegionServerStatus.proto  |   5 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |  15 ++-
 .../hbase/regionserver/HRegionServer.java       |  72 ++++--------
 .../RemoteProcedureResultReporter.java          | 111 +++++++++++++++++++
 .../handler/RSProcedureHandler.java             |   2 +-
 5 files changed, 149 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8d0b8b9d/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index 4f75941..3f836cd 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -146,7 +146,7 @@ message RegionSpaceUseReportRequest {
 message RegionSpaceUseReportResponse {
 }
 
-message ReportProcedureDoneRequest {
+message RemoteProcedureResult {
   required uint64 proc_id = 1;
   enum Status {
     SUCCESS = 1;
@@ -155,6 +155,9 @@ message ReportProcedureDoneRequest {
   required Status status = 2;
   optional ForeignExceptionMessage error = 3;
 }
+message ReportProcedureDoneRequest {
+  repeated RemoteProcedureResult result = 1;
+}
 
 message ReportProcedureDoneResponse {
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8d0b8b9d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 72bf2d1..377a9c6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -265,6 +265,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
@@ -2254,12 +2255,14 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public ReportProcedureDoneResponse reportProcedureDone(RpcController controller,
       ReportProcedureDoneRequest request) throws ServiceException {
-    if (request.getStatus() == ReportProcedureDoneRequest.Status.SUCCESS) {
-      master.remoteProcedureCompleted(request.getProcId());
-    } else {
-      master.remoteProcedureFailed(request.getProcId(),
-        RemoteProcedureException.fromProto(request.getError()));
-    }
+    request.getResultList().forEach(result -> {
+      if (result.getStatus() == RemoteProcedureResult.Status.SUCCESS) {
+        master.remoteProcedureCompleted(result.getProcId());
+      } else {
+        master.remoteProcedureFailed(result.getProcId(),
+          RemoteProcedureException.fromProto(result.getError()));
+      }
+    });
     return ReportProcedureDoneResponse.getDefaultInstance();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8d0b8b9d/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 9ff6295..516a77e 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
@@ -146,7 +146,6 @@ import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
 import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
@@ -379,6 +378,9 @@ public class HRegionServer extends HasThread implements
   // eclipse warning when accessed by inner classes
   protected LogRoller walRoller;
 
+  // A thread which calls reportProcedureDone
+  private RemoteProcedureResultReporter procedureResultReporter;
+
   // flag set after we're done setting up server threads
   final AtomicBoolean online = new AtomicBoolean(false);
 
@@ -1884,6 +1886,7 @@ public class HRegionServer extends HasThread implements
 
     this.walRoller = new LogRoller(this, this);
     this.flushThroughputController = FlushThroughputControllerFactory.create(this, conf);
+    this.procedureResultReporter = new RemoteProcedureResultReporter(this);
 
     // Create the CompactedFileDischarger chore executorService. This chore helps to
     // remove the compacted files
@@ -1927,6 +1930,8 @@ public class HRegionServer extends HasThread implements
     Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
     uncaughtExceptionHandler);
     this.cacheFlusher.start(uncaughtExceptionHandler);
+    Threads.setDaemonThreadRunning(this.procedureResultReporter,
+      getName() + ".procedureResultReporter", uncaughtExceptionHandler);
 
     if (this.compactionChecker != null) choreService.scheduleChore(compactionChecker);
     if (this.periodicFlusher != null) choreService.scheduleChore(periodicFlusher);
@@ -3709,55 +3714,26 @@ public class HRegionServer extends HasThread implements
     executorService.submit(new RSProcedureHandler(this, procId, callable));
   }
 
-  public void reportProcedureDone(long procId, Throwable error) {
-    ReportProcedureDoneRequest.Builder builder =
-      ReportProcedureDoneRequest.newBuilder().setProcId(procId);
-    if (error != null) {
-      builder.setStatus(ReportProcedureDoneRequest.Status.ERROR)
-          .setError(ForeignExceptionUtil.toProtoForeignException(serverName.toString(), error));
-    } else {
-      builder.setStatus(ReportProcedureDoneRequest.Status.SUCCESS);
+  public void remoteProcedureComplete(long procId, Throwable error) {
+    procedureResultReporter.complete(procId, error);
+  }
+
+  void reportProcedureDone(ReportProcedureDoneRequest request) throws IOException {
+    RegionServerStatusService.BlockingInterface rss = rssStub;
+    for (;;) {
+      rss = rssStub;
+      if (rss != null) {
+        break;
+      }
+      createRegionServerStatusStub();
     }
-    ReportProcedureDoneRequest request = builder.build();
-    int tries = 0;
-    long pauseTime = INIT_PAUSE_TIME_MS;
-    while (keepLooping()) {
-      RegionServerStatusService.BlockingInterface rss = rssStub;
-      try {
-        if (rss == null) {
-          createRegionServerStatusStub();
-          continue;
-        }
-        rss.reportProcedureDone(null, request);
-        // Log if we had to retry else don't log unless TRACE. We want to
-        // know if were successful after an attempt showed in logs as failed.
-        if (tries > 0 || LOG.isTraceEnabled()) {
-          LOG.info("PROCEDURE REPORTED " + request);
-        }
-        return;
-      } catch (ServiceException se) {
-        IOException ioe = ProtobufUtil.getRemoteException(se);
-        boolean pause =
-          ioe instanceof ServerNotRunningYetException || ioe instanceof PleaseHoldException;
-        if (pause) {
-          // Do backoff else we flood the Master with requests.
-          pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
-        } else {
-          pauseTime = INIT_PAUSE_TIME_MS; // Reset.
-        }
-        LOG.info(
-          "Failed to report transition " + TextFormat.shortDebugString(request) + "; retry (#" +
-            tries + ")" + (pause ? " after " + pauseTime + "ms delay (Master is coming online...)."
-              : " immediately."),
-          ioe);
-        if (pause) {
-          Threads.sleep(pauseTime);
-        }
-        tries++;
-        if (rssStub == rss) {
-          rssStub = null;
-        }
+    try {
+      rss.reportProcedureDone(null, request);
+    } catch (ServiceException se) {
+      if (rssStub == rss) {
+        rssStub = null;
       }
+      throw ProtobufUtil.getRemoteException(se);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8d0b8b9d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
new file mode 100644
index 0000000..e4be422
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
@@ -0,0 +1,111 @@
+/**
+ * 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.regionserver;
+
+import java.io.IOException;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
+
+/**
+ * A thread which calls {@code reportProcedureDone} to tell master the result of a remote procedure.
+ */
+@InterfaceAudience.Private
+class RemoteProcedureResultReporter extends Thread {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RemoteProcedureResultReporter.class);
+
+  // Time to pause if master says 'please hold'. Make configurable if needed.
+  private static final int INIT_PAUSE_TIME_MS = 1000;
+
+  private static final int MAX_BATCH = 100;
+
+  private final HRegionServer server;
+
+  private final LinkedBlockingQueue<RemoteProcedureResult> results = new LinkedBlockingQueue<>();
+
+  public RemoteProcedureResultReporter(HRegionServer server) {
+    this.server = server;
+  }
+
+  public void complete(long procId, Throwable error) {
+    RemoteProcedureResult.Builder builder = RemoteProcedureResult.newBuilder().setProcId(procId);
+    if (error != null) {
+      builder.setStatus(RemoteProcedureResult.Status.ERROR).setError(
+        ForeignExceptionUtil.toProtoForeignException(server.getServerName().toString(), error));
+    } else {
+      builder.setStatus(RemoteProcedureResult.Status.SUCCESS);
+    }
+    results.add(builder.build());
+  }
+
+  @Override
+  public void run() {
+    ReportProcedureDoneRequest.Builder builder = ReportProcedureDoneRequest.newBuilder();
+    int tries = 0;
+    while (!server.isStopped()) {
+      if (builder.getResultCount() == 0) {
+        try {
+          builder.addResult(results.take());
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          continue;
+        }
+      }
+      while (builder.getResultCount() < MAX_BATCH) {
+        RemoteProcedureResult result = results.poll();
+        if (result == null) {
+          break;
+        }
+        builder.addResult(result);
+      }
+      ReportProcedureDoneRequest request = builder.build();
+      try {
+        server.reportProcedureDone(builder.build());
+        builder.clear();
+        tries = 0;
+      } catch (IOException e) {
+        boolean pause =
+          e instanceof ServerNotRunningYetException || e instanceof PleaseHoldException;
+        long pauseTime;
+        if (pause) {
+          // Do backoff else we flood the Master with requests.
+          pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
+        } else {
+          pauseTime = INIT_PAUSE_TIME_MS; // Reset.
+        }
+        LOG.info("Failed report procedure " + TextFormat.shortDebugString(request) + "; retry (#" +
+          tries + ")" + (pause ? " after " + pauseTime + "ms delay (Master is coming online...)."
+            : " immediately."),
+          e);
+        Threads.sleep(pauseTime);
+        tries++;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8d0b8b9d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
index 240b0a7..d2175d0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
@@ -49,6 +49,6 @@ public class RSProcedureHandler extends EventHandler {
       LOG.error("Catch exception when call RSProcedureCallable: ", e);
       error = e;
     }
-    ((HRegionServer) server).reportProcedureDone(procId, error);
+    ((HRegionServer) server).remoteProcedureComplete(procId, error);
   }
 }


[37/48] hbase git commit: HBASE-19686 Use KeyLocker instead of ReentrantLock in PeerProcedureHandlerImpl

Posted by zh...@apache.org.
HBASE-19686 Use KeyLocker instead of ReentrantLock in PeerProcedureHandlerImpl


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: def5bee927c821ff3610cca4f777b9da25899075
Parents: 7fe3a90
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 2 16:13:55 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:28:50 2018 +0800

----------------------------------------------------------------------
 .../regionserver/PeerProcedureHandlerImpl.java  | 41 ++++++++++++++------
 1 file changed, 29 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/def5bee9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
index 1efe180..c09c6a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -19,10 +19,10 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-import java.util.concurrent.locks.ReentrantLock;
-
+import java.util.concurrent.locks.Lock;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -32,7 +32,7 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
   private static final Logger LOG = LoggerFactory.getLogger(PeerProcedureHandlerImpl.class);
 
   private final ReplicationSourceManager replicationSourceManager;
-  private final ReentrantLock peersLock = new ReentrantLock();
+  private final KeyLocker<String> peersLock = new KeyLocker<>();
 
   public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) {
     this.replicationSourceManager = replicationSourceManager;
@@ -40,40 +40,57 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
   @Override
   public void addPeer(String peerId) throws ReplicationException, IOException {
-    peersLock.lock();
+    Lock peerLock = peersLock.acquireLock(peerId);
     try {
       replicationSourceManager.addPeer(peerId);
     } finally {
-      peersLock.unlock();
+      peerLock.unlock();
     }
   }
 
   @Override
   public void removePeer(String peerId) throws ReplicationException, IOException {
-    peersLock.lock();
+    Lock peerLock = peersLock.acquireLock(peerId);
     try {
       if (replicationSourceManager.getReplicationPeers().getPeer(peerId) != null) {
         replicationSourceManager.removePeer(peerId);
       }
     } finally {
-      peersLock.unlock();
+      peerLock.unlock();
     }
   }
 
   @Override
   public void disablePeer(String peerId) throws ReplicationException, IOException {
-    PeerState newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
-    LOG.info("disable replication peer, id: " + peerId + ", new state: " + newState);
+    PeerState newState;
+    Lock peerLock = peersLock.acquireLock(peerId);
+    try {
+      newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+    } finally {
+      peerLock.unlock();
+    }
+    LOG.info("disable replication peer, id: {}, new state: {}", peerId, newState);
   }
 
   @Override
   public void enablePeer(String peerId) throws ReplicationException, IOException {
-    PeerState newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
-    LOG.info("enable replication peer, id: " + peerId + ", new state: " + newState);
+    PeerState newState;
+    Lock peerLock = peersLock.acquireLock(peerId);
+    try {
+      newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+    } finally {
+      peerLock.unlock();
+    }
+    LOG.info("enable replication peer, id: {}, new state: {}", peerId, newState);
   }
 
   @Override
   public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
-    replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+    Lock peerLock = peersLock.acquireLock(peerId);
+    try {
+      replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+    } finally {
+      peerLock.unlock();
+    }
   }
 }


[21/48] hbase git commit: HBASE-19525 RS side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19525 RS side changes for moving peer modification from zk watcher to procedure


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 1b2ce999f8471f19a1d79cfc02ea082bb5bfcb46
Parents: fa49508
Author: huzheng <op...@gmail.com>
Authored: Wed Dec 20 10:47:18 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Jan 15 18:27:56 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  11 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  13 +-
 .../hbase/replication/ReplicationListener.java  |  14 --
 .../hbase/replication/ReplicationPeer.java      |  28 ++-
 .../replication/ReplicationPeerZKImpl.java      | 186 ++++-----------
 .../replication/ReplicationPeersZKImpl.java     |  19 +-
 .../replication/ReplicationTrackerZKImpl.java   |  73 +-----
 .../regionserver/ReplicationSourceService.java  |   9 +-
 .../handler/RSProcedureHandler.java             |   3 +
 .../replication/BaseReplicationEndpoint.java    |   2 +-
 .../regionserver/PeerProcedureHandler.java      |  38 ++++
 .../regionserver/PeerProcedureHandlerImpl.java  |  81 +++++++
 .../regionserver/RefreshPeerCallable.java       |  39 +++-
 .../replication/regionserver/Replication.java   |  10 +
 .../regionserver/ReplicationSource.java         |   9 +-
 .../regionserver/ReplicationSourceManager.java  |  37 ++-
 .../TestReplicationAdminUsingProcedure.java     | 226 +++++++++++++++++++
 .../replication/DummyModifyPeerProcedure.java   |  48 ----
 .../TestDummyModifyPeerProcedure.java           |  80 -------
 .../TestReplicationTrackerZKImpl.java           |  51 -----
 .../TestReplicationSourceManager.java           |  32 ++-
 21 files changed, 535 insertions(+), 474 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 7b63cd6..7d83687 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.protobuf;
 
+import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC;
+
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.InvalidProtocolBufferException;
@@ -200,7 +202,7 @@ public final class ProtobufUtil {
    * byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length.
    */
   public static byte [] prependPBMagic(final byte [] bytes) {
-    return Bytes.add(ProtobufMagic.PB_MAGIC, bytes);
+    return Bytes.add(PB_MAGIC, bytes);
   }
 
   /**
@@ -225,10 +227,11 @@ public final class ProtobufUtil {
    * @param bytes bytes to check
    * @throws DeserializationException if we are missing the pb magic prefix
    */
-  public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
+  public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException {
     if (!isPBMagicPrefix(bytes)) {
-      throw new DeserializationException("Missing pb magic " +
-          Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix");
+      String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length);
+      throw new DeserializationException(
+          "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix, bytes: " + bytesPrefix);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index b26802f..5e6b3db 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.shaded.protobuf;
 
+import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -280,7 +282,7 @@ public final class ProtobufUtil {
    * byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length.
    */
   public static byte [] prependPBMagic(final byte [] bytes) {
-    return Bytes.add(ProtobufMagic.PB_MAGIC, bytes);
+    return Bytes.add(PB_MAGIC, bytes);
   }
 
   /**
@@ -305,10 +307,11 @@ public final class ProtobufUtil {
    * @param bytes bytes to check
    * @throws DeserializationException if we are missing the pb magic prefix
    */
-  public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
+  public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException {
     if (!isPBMagicPrefix(bytes)) {
-      throw new DeserializationException("Missing pb magic " +
-          Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix");
+      String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length);
+      throw new DeserializationException(
+          "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix" + ", bytes: " + bytesPrefix);
     }
   }
 
@@ -1946,7 +1949,7 @@ public final class ProtobufUtil {
   public static byte [] toDelimitedByteArray(final Message m) throws IOException {
     // Allocate arbitrary big size so we avoid resizing.
     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
-    baos.write(ProtobufMagic.PB_MAGIC);
+    baos.write(PB_MAGIC);
     m.writeDelimitedTo(baos);
     return baos.toByteArray();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
index 3edfcf9..f040bf9 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import java.util.List;
-
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -36,16 +34,4 @@ public interface ReplicationListener {
    * @param regionServer the removed region server
    */
   public void regionServerRemoved(String regionServer);
-
-  /**
-   * A peer cluster has been removed (i.e. unregistered) from replication.
-   * @param peerId The peer id of the cluster that has been removed
-   */
-  public void peerRemoved(String peerId);
-
-  /**
-   * The list of registered peer clusters has changed.
-   * @param peerIds A list of all currently registered peer clusters
-   */
-  public void peerListChanged(List<String> peerIds);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
index 97e2ddb..b66d76d 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
@@ -55,12 +55,27 @@ public interface ReplicationPeer {
   public ReplicationPeerConfig getPeerConfig();
 
   /**
-   * Returns the state of the peer
+   * Get the peer config object. if loadFromBackingStore is true, it will load from backing store
+   * directly and update its load peer config. otherwise, just return the local cached peer config.
+   * @return the ReplicationPeerConfig for this peer
+   */
+  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
+      throws ReplicationException;
+
+  /**
+   * Returns the state of the peer by reading local cache.
    * @return the enabled state
    */
   PeerState getPeerState();
 
   /**
+   * Returns the state of peer, if loadFromBackingStore is true, it will load from backing store
+   * directly and update its local peer state. otherwise, just return the local cached peer state.
+   * @return the enabled state
+   */
+  PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException;
+
+  /**
    * Get the configuration object required to communicate with this peer
    * @return configuration object
    */
@@ -84,6 +99,15 @@ public interface ReplicationPeer {
    */
   public long getPeerBandwidth();
 
-  void trackPeerConfigChanges(ReplicationPeerConfigListener listener);
+  /**
+   * Register a peer config listener to catch the peer config change event.
+   * @param listener listener to catch the peer config change event.
+   */
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener);
 
+  /**
+   * Notify all the registered ReplicationPeerConfigListener to update their peer config.
+   * @param newPeerConfig the new peer config.
+   */
+  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/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 454d09c..49b9460 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
@@ -20,41 +20,39 @@ package org.apache.hadoop.hbase.replication;
 
 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.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
-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.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
 @InterfaceAudience.Private
 public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     implements ReplicationPeer, Abortable, Closeable {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerZKImpl.class);
 
-  private ReplicationPeerConfig peerConfig;
+  private volatile ReplicationPeerConfig peerConfig;
   private final String id;
   private volatile PeerState peerState;
   private volatile Map<TableName, List<String>> tableCFs = new HashMap<>();
   private final Configuration conf;
-  private PeerStateTracker peerStateTracker;
-  private PeerConfigTracker peerConfigTracker;
 
+  private final List<ReplicationPeerConfigListener> peerConfigListeners;
 
   /**
    * Constructor that takes all the objects required to communicate with the specified peer, except
@@ -63,62 +61,35 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf,
-                               String id, ReplicationPeerConfig peerConfig,
-                               Abortable abortable)
-      throws ReplicationException {
+  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf, String id,
+      ReplicationPeerConfig peerConfig, Abortable abortable) throws ReplicationException {
     super(zkWatcher, conf, abortable);
     this.conf = conf;
     this.peerConfig = peerConfig;
     this.id = id;
+    this.peerConfigListeners = new ArrayList<>();
   }
 
-  /**
-   * start a state tracker to check whether this peer is enabled or not
-   *
-   * @param peerStateNode path to zk node which stores peer state
-   * @throws KeeperException
-   */
-  public void startStateTracker(String peerStateNode)
-      throws KeeperException {
-    ensurePeerEnabled(peerStateNode);
-    this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this);
-    this.peerStateTracker.start();
+  private PeerState readPeerState() throws ReplicationException {
     try {
-      this.readPeerStateZnode();
-    } catch (DeserializationException e) {
-      throw ZKUtil.convert(e);
+      byte[] data = ZKUtil.getData(zookeeper, this.getPeerStateNode(id));
+      this.peerState = isStateEnabled(data) ? PeerState.ENABLED : PeerState.DISABLED;
+    } catch (DeserializationException | KeeperException | InterruptedException e) {
+      throw new ReplicationException("Get and deserialize peer state data from zookeeper failed: ",
+          e);
     }
+    return this.peerState;
   }
 
-  private void readPeerStateZnode() throws DeserializationException {
-    this.peerState =
-        isStateEnabled(this.peerStateTracker.getData(false))
-          ? PeerState.ENABLED
-          : PeerState.DISABLED;
-  }
-
-  /**
-   * start a table-cfs tracker to listen the (table, cf-list) map change
-   * @param peerConfigNode path to zk node which stores table-cfs
-   * @throws KeeperException
-   */
-  public void startPeerConfigTracker(String peerConfigNode)
-    throws KeeperException {
-    this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper,
-        this);
-    this.peerConfigTracker.start();
-    this.readPeerConfig();
-  }
-
-  private ReplicationPeerConfig readPeerConfig() {
+  private ReplicationPeerConfig readPeerConfig() throws ReplicationException {
     try {
-      byte[] data = peerConfigTracker.getData(false);
+      byte[] data = ZKUtil.getData(zookeeper, this.getPeerNode(id));
       if (data != null) {
         this.peerConfig = ReplicationPeerConfigUtil.parsePeerFrom(data);
       }
-    } catch (DeserializationException e) {
-      LOG.error("", e);
+    } catch (DeserializationException | KeeperException | InterruptedException e) {
+      throw new ReplicationException("Get and deserialize peer config date from zookeeper failed: ",
+          e);
     }
     return this.peerConfig;
   }
@@ -128,6 +99,15 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     return peerState;
   }
 
+  @Override
+  public PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException {
+    if (loadFromBackingStore) {
+      return readPeerState();
+    } else {
+      return peerState;
+    }
+  }
+
   /**
    * Get the identifier of this peer
    * @return string representation of the id (short)
@@ -146,6 +126,16 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     return peerConfig;
   }
 
+  @Override
+  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
+      throws ReplicationException {
+    if (loadFromBackingStore) {
+      return readPeerConfig();
+    } else {
+      return peerConfig;
+    }
+  }
+
   /**
    * Get the configuration object required to communicate with this peer
    * @return configuration object
@@ -180,9 +170,14 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
   }
 
   @Override
-  public void trackPeerConfigChanges(ReplicationPeerConfigListener listener) {
-    if (this.peerConfigTracker != null){
-      this.peerConfigTracker.setListener(listener);
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
+    this.peerConfigListeners.add(listener);
+  }
+
+  @Override
+  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig) {
+    for (ReplicationPeerConfigListener listener : this.peerConfigListeners) {
+      listener.peerConfigUpdated(newPeerConfig);
     }
   }
 
@@ -223,97 +218,16 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
   private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
       throws DeserializationException {
     ProtobufUtil.expectPBMagicPrefix(bytes);
-    int pblen = ProtobufUtil.lengthOfPBMagic();
+    int pbLen = ProtobufUtil.lengthOfPBMagic();
     ReplicationProtos.ReplicationState.Builder builder =
         ReplicationProtos.ReplicationState.newBuilder();
     ReplicationProtos.ReplicationState state;
     try {
-      ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
       state = builder.build();
       return state.getState();
     } catch (IOException e) {
       throw new DeserializationException(e);
     }
   }
-
-  /**
-   * Utility method to ensure an ENABLED znode is in place; if not present, we create it.
-   * @param path Path to znode to check
-   * @return True if we created the znode.
-   * @throws NodeExistsException
-   * @throws KeeperException
-   */
-  private boolean ensurePeerEnabled(final String path)
-      throws NodeExistsException, KeeperException {
-    if (ZKUtil.checkExists(zookeeper, path) == -1) {
-      // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the
-      // peer-state znode. This happens while adding a peer.
-      // The peer state data is set as "ENABLED" by default.
-      ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path,
-        ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Tracker for state of this peer
-   */
-  public class PeerStateTracker extends ZKNodeTracker {
-
-    public PeerStateTracker(String peerStateZNode, ZKWatcher watcher,
-        Abortable abortable) {
-      super(watcher, peerStateZNode, abortable);
-    }
-
-    @Override
-    public synchronized void nodeDataChanged(String path) {
-      if (path.equals(node)) {
-        super.nodeDataChanged(path);
-        try {
-          readPeerStateZnode();
-        } catch (DeserializationException e) {
-          LOG.warn("Failed deserializing the content of " + path, e);
-        }
-      }
-    }
-  }
-
-  /**
-   * Tracker for PeerConfigNode of this peer
-   */
-  public class PeerConfigTracker extends ZKNodeTracker {
-
-    ReplicationPeerConfigListener listener;
-
-    public PeerConfigTracker(String peerConfigNode, ZKWatcher watcher,
-        Abortable abortable) {
-      super(watcher, peerConfigNode, abortable);
-    }
-
-    public synchronized void setListener(ReplicationPeerConfigListener listener){
-      this.listener = listener;
-    }
-
-    @Override
-    public synchronized void nodeCreated(String path) {
-      if (path.equals(node)) {
-        super.nodeCreated(path);
-        ReplicationPeerConfig config = readPeerConfig();
-        if (listener != null){
-          listener.peerConfigUpdated(config);
-        }
-      }
-    }
-
-    @Override
-    public synchronized void nodeDataChanged(String path) {
-      //superclass calls nodeCreated
-      if (path.equals(node)) {
-        super.nodeDataChanged(path);
-      }
-
-    }
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/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 e4b1875..8ed0150 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
@@ -499,21 +499,12 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     }
     Configuration peerConf = pair.getSecond();
 
-    ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(zookeeper,
-        peerConf, peerId, pair.getFirst(), abortable);
-    try {
-      peer.startStateTracker(this.getPeerStateNode(peerId));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error starting the peer state tracker for peerId=" +
-          peerId, e);
-    }
+    ReplicationPeerZKImpl peer =
+        new ReplicationPeerZKImpl(zookeeper, peerConf, peerId, pair.getFirst(), abortable);
 
-    try {
-      peer.startPeerConfigTracker(this.getPeerNode(peerId));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" +
-          peerId, e);
-    }
+    // Load peer state and peer config by reading zookeeper directly.
+    peer.getPeerState(true);
+    peer.getPeerConfig(true);
 
     return peer;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/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 9a1d9aa..2c522f6 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
@@ -48,16 +48,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   private final List<ReplicationListener> listeners = new CopyOnWriteArrayList<>();
   // List of all the other region servers in this cluster
   private final ArrayList<String> otherRegionServers = new ArrayList<>();
-  private final ReplicationPeers replicationPeers;
 
-  public ReplicationTrackerZKImpl(ZKWatcher zookeeper,
-      final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
-      Stoppable stopper) {
+  public ReplicationTrackerZKImpl(ZKWatcher zookeeper, final ReplicationPeers replicationPeers,
+      Configuration conf, Abortable abortable, Stoppable stopper) {
     super(zookeeper, conf, abortable);
-    this.replicationPeers = replicationPeers;
     this.stopper = stopper;
     this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper));
-    this.zookeeper.registerListener(new PeersWatcher(this.zookeeper));
   }
 
   @Override
@@ -146,71 +142,6 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   }
 
   /**
-   * Watcher used to follow the creation and deletion of peer clusters.
-   */
-  public class PeersWatcher extends ZKListener {
-
-    /**
-     * Construct a ZooKeeper event listener.
-     */
-    public PeersWatcher(ZKWatcher watcher) {
-      super(watcher);
-    }
-
-    /**
-     * Called when a node has been deleted
-     * @param path full path of the deleted node
-     */
-    @Override
-    public void nodeDeleted(String path) {
-      List<String> peers = refreshPeersList(path);
-      if (peers == null) {
-        return;
-      }
-      if (isPeerPath(path)) {
-        String id = getZNodeName(path);
-        LOG.info(path + " znode expired, triggering peerRemoved event");
-        for (ReplicationListener rl : listeners) {
-          rl.peerRemoved(id);
-        }
-      }
-    }
-
-    /**
-     * Called when an existing node has a child node added or removed.
-     * @param path full path of the node whose children have changed
-     */
-    @Override
-    public void nodeChildrenChanged(String path) {
-      List<String> peers = refreshPeersList(path);
-      if (peers == null) {
-        return;
-      }
-      LOG.info(path + " znode expired, triggering peerListChanged event");
-      for (ReplicationListener rl : listeners) {
-        rl.peerListChanged(peers);
-      }
-    }
-  }
-
-  /**
-   * Verify if this event is meant for us, and if so then get the latest peers' list from ZK. Also
-   * reset the watches.
-   * @param path path to check against
-   * @return A list of peers' identifiers if the event concerns this watcher, else null.
-   */
-  private List<String> refreshPeersList(String path) {
-    if (!path.startsWith(getPeersZNode())) {
-      return null;
-    }
-    return this.replicationPeers.getAllPeerIds();
-  }
-
-  private String getPeersZNode() {
-    return this.peersZNode;
-  }
-
-  /**
    * Extracts the znode name of a peer cluster from a ZK path
    * @param fullPath Path to extract the id from
    * @return the id or an empty string if path is invalid

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
index 835ffbf..a82fa3d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import org.apache.hadoop.hbase.replication.regionserver.PeerProcedureHandler;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 
@@ -29,8 +30,14 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 @InterfaceAudience.Private
 public interface ReplicationSourceService extends ReplicationService {
   /**
-   * Returns a WALObserver for the service. This is needed to 
+   * Returns a WALObserver for the service. This is needed to
    * observe log rolls and log archival events.
    */
   WALActionsListener getWALActionsListener();
+
+
+  /**
+   * Returns a Handler to handle peer procedures.
+   */
+  PeerProcedureHandler getPeerProcedureHandler();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
index 94bcfec..240b0a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.handler;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.log4j.Logger;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -28,6 +29,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class RSProcedureHandler extends EventHandler {
 
+  private static final Logger LOG = Logger.getLogger(RSProcedureHandler.class);
   private final long procId;
 
   private final RSProcedureCallable callable;
@@ -44,6 +46,7 @@ public class RSProcedureHandler extends EventHandler {
     try {
       callable.call();
     } catch (Exception e) {
+      LOG.error("Catch exception when call RSProcedureCallable: ", e);
       error = e;
     }
     ((HRegionServer) server).reportProcedureDone(procId, error);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
index 2093421..7b91bee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
@@ -49,7 +49,7 @@ public abstract class BaseReplicationEndpoint extends AbstractService
     if (this.ctx != null){
       ReplicationPeer peer = this.ctx.getReplicationPeer();
       if (peer != null){
-        peer.trackPeerConfigChanges(this);
+        peer.registerPeerConfigListener(this);
       } else {
         LOG.warn("Not tracking replication peer config changes for Peer Id " + this.ctx.getPeerId() +
             " because there's no such peer");

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
new file mode 100644
index 0000000..b392985
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface PeerProcedureHandler {
+
+  public void addPeer(String peerId) throws ReplicationException, IOException;
+
+  public void removePeer(String peerId) throws ReplicationException, IOException;
+
+  public void disablePeer(String peerId) throws ReplicationException, IOException;
+
+  public void enablePeer(String peerId) throws ReplicationException, IOException;
+
+  public void updatePeerConfig(String peerId) throws ReplicationException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
new file mode 100644
index 0000000..9b493d9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.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.replication.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeer;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.log4j.Logger;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
+  private static final Logger LOG = Logger.getLogger(PeerProcedureHandlerImpl.class);
+
+  private ReplicationSourceManager replicationSourceManager;
+
+  public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) {
+    this.replicationSourceManager = replicationSourceManager;
+  }
+
+  @Override
+  public void addPeer(String peerId) throws ReplicationException, IOException {
+    replicationSourceManager.addPeer(peerId);
+  }
+
+  @Override
+  public void removePeer(String peerId) throws ReplicationException, IOException {
+    replicationSourceManager.removePeer(peerId);
+  }
+
+  @Override
+  public void disablePeer(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer != null) {
+      PeerState peerState = peer.getPeerState(true);
+      LOG.info("disablePeer state, peer id: " + peerId + ", state: " + peerState);
+    } else {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+  }
+
+  @Override
+  public void enablePeer(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer != null) {
+      PeerState peerState = peer.getPeerState(true);
+      LOG.info("enablePeer state, peer id: " + peerId + ", state: " + peerState);
+    } else {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+  }
+
+  @Override
+  public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer == null) {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+    ReplicationPeerConfig rpc = peer.getPeerConfig(true);
+    peer.triggerPeerConfigChange(rpc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
index a47a483..c3f33aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
@@ -17,27 +17,29 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
+import org.apache.log4j.Logger;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
 
 /**
- * The callable executed at RS side to refresh the peer config/state.
- * <p>
- * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ * The callable executed at RS side to refresh the peer config/state. <br/>
  */
 @InterfaceAudience.Private
 public class RefreshPeerCallable implements RSProcedureCallable {
 
+  private static final Logger LOG = Logger.getLogger(RefreshPeerCallable.class);
   private HRegionServer rs;
 
   private String peerId;
 
+  private PeerModificationType type;
+
   private Exception initError;
 
   @Override
@@ -45,9 +47,27 @@ public class RefreshPeerCallable implements RSProcedureCallable {
     if (initError != null) {
       throw initError;
     }
-    Path dir = new Path("/" + peerId);
-    if (rs.getFileSystem().exists(dir)) {
-      rs.getFileSystem().create(new Path(dir, rs.getServerName().toString())).close();
+
+    LOG.info("Received a peer change event, peerId=" + peerId + ", type=" + type);
+    PeerProcedureHandler handler = rs.getReplicationSourceService().getPeerProcedureHandler();
+    switch (type) {
+      case ADD_PEER:
+        handler.addPeer(this.peerId);
+        break;
+      case REMOVE_PEER:
+        handler.removePeer(this.peerId);
+        break;
+      case ENABLE_PEER:
+        handler.enablePeer(this.peerId);
+        break;
+      case DISABLE_PEER:
+        handler.disablePeer(this.peerId);
+        break;
+      case UPDATE_PEER_CONFIG:
+        handler.updatePeerConfig(this.peerId);
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown peer modification type: " + type);
     }
     return null;
   }
@@ -56,10 +76,11 @@ public class RefreshPeerCallable implements RSProcedureCallable {
   public void init(byte[] parameter, HRegionServer rs) {
     this.rs = rs;
     try {
-      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+      RefreshPeerParameter param = RefreshPeerParameter.parseFrom(parameter);
+      this.peerId = param.getPeerId();
+      this.type = param.getType();
     } catch (InvalidProtocolBufferException e) {
       initError = e;
-      return;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index dcd79a6..ff45f53 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -87,6 +87,8 @@ public class Replication implements
   // ReplicationLoad to access replication metrics
   private ReplicationLoad replicationLoad;
 
+  private PeerProcedureHandler peerProcedureHandler;
+
   /**
    * Instantiate the replication management (if rep is enabled).
    * @param server Hosting server
@@ -151,6 +153,8 @@ public class Replication implements
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
     LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
     this.replicationLoad = new ReplicationLoad();
+
+    this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager);
   }
 
   /**
@@ -168,6 +172,12 @@ public class Replication implements
   public WALActionsListener getWALActionsListener() {
     return this;
   }
+
+  @Override
+  public PeerProcedureHandler getPeerProcedureHandler() {
+    return peerProcedureHandler;
+  }
+
   /**
    * Stops replication service.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 16fb4a7..3744017 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -446,12 +445,10 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
 
   public void terminate(String reason, Exception cause, boolean join) {
     if (cause == null) {
-      LOG.info("Closing source "
-          + this.peerClusterZnode + " because: " + reason);
-
+      LOG.info("Closing source " + this.peerClusterZnode + " because: " + reason);
     } else {
-      LOG.error("Closing source " + this.peerClusterZnode
-          + " because an error occurred: " + reason, cause);
+      LOG.error("Closing source " + this.peerClusterZnode + " because an error occurred: " + reason,
+        cause);
     }
     this.sourceRunning = false;
     Collection<ReplicationSourceShipper> workers = workerThreads.values();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 23ae704..b1d82c8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -564,6 +564,18 @@ public class ReplicationSourceManager implements ReplicationListener {
     this.walsById.remove(src.getPeerClusterZnode());
   }
 
+  public void addPeer(String id) throws ReplicationException, IOException {
+    LOG.info("Trying to add peer, peerId: " + id);
+    boolean added = this.replicationPeers.peerConnected(id);
+    if (added) {
+      LOG.info("Peer " + id + " connected success, trying to start the replication source thread.");
+      addSource(id);
+      if (replicationForBulkLoadDataEnabled) {
+        this.replicationQueues.addPeerToHFileRefs(id);
+      }
+    }
+  }
+
   /**
    * Thie method first deletes all the recovered sources for the specified
    * id, then deletes the normal source (deleting all related data in ZK).
@@ -611,6 +623,8 @@ public class ReplicationSourceManager implements ReplicationListener {
       }
       deleteSource(id, true);
     }
+    // Remove HFile Refs znode from zookeeper
+    this.replicationQueues.removePeerFromHFileRefs(id);
   }
 
   @Override
@@ -618,29 +632,6 @@ public class ReplicationSourceManager implements ReplicationListener {
     transferQueues(regionserver);
   }
 
-  @Override
-  public void peerRemoved(String peerId) {
-    removePeer(peerId);
-    this.replicationQueues.removePeerFromHFileRefs(peerId);
-  }
-
-  @Override
-  public void peerListChanged(List<String> peerIds) {
-    for (String id : peerIds) {
-      try {
-        boolean added = this.replicationPeers.peerConnected(id);
-        if (added) {
-          addSource(id);
-          if (replicationForBulkLoadDataEnabled) {
-            this.replicationQueues.addPeerToHFileRefs(id);
-          }
-        }
-      } catch (Exception e) {
-        LOG.error("Error while adding a new peer", e);
-      }
-    }
-  }
-
   /**
    * Class responsible to setup new ReplicationSources to take care of the
    * queues from dead region servers.

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
new file mode 100644
index 0000000..b09a8a7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
@@ -0,0 +1,226 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client.replication;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.TestReplicationBase;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestReplicationAdminUsingProcedure extends TestReplicationBase {
+
+  private static final String PEER_ID = "2";
+  private static final Logger LOG = Logger.getLogger(TestReplicationAdminUsingProcedure.class);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.setInt("hbase.multihconnection.threads.max", 10);
+
+    // Start the master & slave mini cluster.
+    TestReplicationBase.setUpBeforeClass();
+
+    // Remove the replication peer
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+  }
+
+  private void loadData(int startRowKey, int endRowKey) throws IOException {
+    for (int i = startRowKey; i < endRowKey; i++) {
+      byte[] rowKey = Bytes.add(row, Bytes.toBytes(i));
+      Put put = new Put(rowKey);
+      put.addColumn(famName, null, Bytes.toBytes(i));
+      htable1.put(put);
+    }
+  }
+
+  private void waitForReplication(int expectedRows, int retries)
+      throws IOException, InterruptedException {
+    Scan scan;
+    for (int i = 0; i < retries; i++) {
+      scan = new Scan();
+      if (i == retries - 1) {
+        throw new IOException("Waited too much time for normal batch replication");
+      }
+      try (ResultScanner scanner = htable2.getScanner(scan)) {
+        int count = 0;
+        for (Result res : scanner) {
+          count++;
+        }
+        if (count != expectedRows) {
+          LOG.info("Only got " + count + " rows,  expected rows: " + expectedRows);
+          Thread.sleep(SLEEP_TIME);
+        } else {
+          return;
+        }
+      }
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.addReplicationPeer(PEER_ID, rpc);
+
+    utility1.waitUntilAllRegionsAssigned(tableName);
+    utility2.waitUntilAllRegionsAssigned(tableName);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+    truncateBoth();
+  }
+
+  private void truncateBoth() throws IOException {
+    utility1.deleteTableData(tableName);
+    utility2.deleteTableData(tableName);
+  }
+
+  @Test
+  public void testAddPeer() throws Exception {
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+
+  @Test
+  public void testRemovePeer() throws Exception {
+    // prev-check
+    waitForReplication(0, NB_RETRIES);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+
+    // Remove the peer id
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+
+    // Load data again
+    loadData(NB_ROWS_IN_BATCH, 2 * NB_ROWS_IN_BATCH);
+
+    // Wait the replication again
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH * 2, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Truncate the table in source cluster
+    truncateBoth();
+
+    // Add peer again
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.addReplicationPeer(PEER_ID, rpc);
+
+    // Load data again
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+
+  @Test
+  public void testDisableAndEnablePeer() throws Exception {
+    // disable peer
+    hbaseAdmin.disableReplicationPeer(PEER_ID);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Will failed to wait the replication.
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Enable the peer
+    hbaseAdmin.enableReplicationPeer(PEER_ID);
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+
+    // Load more data
+    loadData(NB_ROWS_IN_BATCH, NB_ROWS_IN_BATCH * 2);
+
+    // Wait replication again.
+    waitForReplication(NB_ROWS_IN_BATCH * 2, NB_RETRIES);
+  }
+
+  @Test
+  public void testUpdatePeerConfig() throws Exception {
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    rpc.setExcludeTableCFsMap(
+      ImmutableMap.of(tableName, ImmutableList.of(Bytes.toString(famName))));
+
+    // Update the peer config to exclude the test table name.
+    hbaseAdmin.updateReplicationPeerConfig(PEER_ID, rpc);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Will failed to wait the replication
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Truncate the table in source cluster
+    truncateBoth();
+
+    // Update the peer config to include the test table name.
+    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+    rpc2.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.updateReplicationPeerConfig(PEER_ID, rpc2);
+
+    // Load data again
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
deleted file mode 100644
index ed7c6fa..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
+++ /dev/null
@@ -1,48 +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.master.replication;
-
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-
-public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
-
-  public DummyModifyPeerProcedure() {
-  }
-
-  public DummyModifyPeerProcedure(String peerId) {
-    super(peerId);
-  }
-
-  @Override
-  public PeerOperationType getPeerOperationType() {
-    return PeerOperationType.ADD;
-  }
-
-  @Override
-  protected void prePeerModification(MasterProcedureEnv env) {
-  }
-
-  @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) {
-  }
-
-  @Override
-  protected void postPeerModification(MasterProcedureEnv env) {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
deleted file mode 100644
index ec06306..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.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.master.replication;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ MasterTests.class, LargeTests.class })
-public class TestDummyModifyPeerProcedure {
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static String PEER_ID;
-
-  private static Path DIR;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    UTIL.startMiniCluster(3);
-    PEER_ID = "testPeer";
-    DIR = new Path("/" + PEER_ID);
-    UTIL.getTestFileSystem().mkdirs(DIR);
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void test() throws Exception {
-    ProcedureExecutor<?> executor =
-        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
-    long procId = executor.submitProcedure(new DummyModifyPeerProcedure(PEER_ID));
-    UTIL.waitFor(30000, new Waiter.Predicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return executor.isFinished(procId);
-      }
-    });
-    Set<String> serverNames = UTIL.getHBaseCluster().getRegionServerThreads().stream()
-        .map(t -> t.getRegionServer().getServerName().toString())
-        .collect(Collectors.toCollection(HashSet::new));
-    for (FileStatus s : UTIL.getTestFileSystem().listStatus(DIR)) {
-      assertTrue(serverNames.remove(s.getPath().getName()));
-    }
-    assertTrue(serverNames.isEmpty());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/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 a04d524..f118ca3 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
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -156,41 +155,6 @@ public class TestReplicationTrackerZKImpl {
   }
 
   @Test(timeout = 30000)
-  public void testPeerRemovedEvent() throws Exception {
-    rp.registerPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    rt.registerListener(new DummyReplicationListener());
-    rp.unregisterPeer("5");
-    // wait for event
-    while (peerRemovedCount.get() < 1) {
-      Thread.sleep(5);
-    }
-    assertEquals("5", peerRemovedData);
-  }
-
-  @Test(timeout = 30000)
-  public void testPeerListChangedEvent() throws Exception {
-    // add a peer
-    rp.registerPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    zkw.getRecoverableZooKeeper().getZooKeeper().getChildren("/hbase/replication/peers/5", true);
-    rt.registerListener(new DummyReplicationListener());
-    rp.disablePeer("5");
-    int tmp = plChangedCount.get();
-    LOG.info("Peer count=" + tmp);
-    ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5/peer-state");
-    // wait for event
-    while (plChangedCount.get() <= tmp) {
-      Thread.sleep(100);
-      LOG.info("Peer count=" + tmp);
-    }
-    assertEquals(1, plChangedData.size());
-    assertTrue(plChangedData.contains("5"));
-
-    // clean up
-    //ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5");
-    rp.unregisterPeer("5");
-  }
-
-  @Test(timeout = 30000)
   public void testPeerNameControl() throws Exception {
     int exists = 0;
     int hyphen = 0;
@@ -222,21 +186,6 @@ public class TestReplicationTrackerZKImpl {
       rsRemovedCount.getAndIncrement();
       LOG.debug("Received regionServerRemoved event: " + regionServer);
     }
-
-    @Override
-    public void peerRemoved(String peerId) {
-      peerRemovedData = peerId;
-      peerRemovedCount.getAndIncrement();
-      LOG.debug("Received peerDisconnected event: " + peerId);
-    }
-
-    @Override
-    public void peerListChanged(List<String> peerIds) {
-      plChangedData.clear();
-      plChangedData.addAll(peerIds);
-      int count = plChangedCount.getAndIncrement();
-      LOG.debug("Received peerListChanged event " + count);
-    }
   }
 
   private class DummyServer implements Server {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b2ce999/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 057c64d..b5a7461 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
@@ -176,6 +176,12 @@ public abstract class TestReplicationSourceManager {
     replication = new Replication(new DummyServer(), fs, logDir, oldLogDir);
 
     managerOfCluster = getManagerFromCluster();
+    if (managerOfCluster != null) {
+      // After replication procedure, we need to add peer by hand (other than by receiving
+      // notification from zk)
+      managerOfCluster.addPeer(slaveId);
+    }
+
     manager = replication.getReplicationManager();
     manager.addSource(slaveId);
     if (managerOfCluster != null) {
@@ -535,18 +541,16 @@ public abstract class TestReplicationSourceManager {
       final int globalLogQueueSizeInitial = globalSource.getSizeOfLogQueue();
       final long sizeOfLatestPath = getSizeOfLatestPath();
       addPeerAndWait(peerId, peerConfig, true);
-      assertEquals(sizeOfLatestPath + globalLogQueueSizeInitial,
-          globalSource.getSizeOfLogQueue());
+      assertEquals(sizeOfLatestPath + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
       ReplicationSourceInterface source = manager.getSource(peerId);
       // Sanity check
       assertNotNull(source);
       final int sizeOfSingleLogQueue = source.getSourceMetrics().getSizeOfLogQueue();
       // Enqueue log and check if metrics updated
       source.enqueueLog(new Path("abc"));
-      assertEquals(1 + sizeOfSingleLogQueue,
-          source.getSourceMetrics().getSizeOfLogQueue());
-      assertEquals(source.getSourceMetrics().getSizeOfLogQueue()
-              + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
+      assertEquals(1 + sizeOfSingleLogQueue, source.getSourceMetrics().getSizeOfLogQueue());
+      assertEquals(source.getSourceMetrics().getSizeOfLogQueue() + globalLogQueueSizeInitial,
+        globalSource.getSizeOfLogQueue());
 
       // Removing the peer should reset the global metrics
       removePeerAndWait(peerId);
@@ -556,9 +560,8 @@ public abstract class TestReplicationSourceManager {
       addPeerAndWait(peerId, peerConfig, true);
       source = manager.getSource(peerId);
       assertNotNull(source);
-      assertEquals(sizeOfLatestPath, source.getSourceMetrics().getSizeOfLogQueue());
-      assertEquals(source.getSourceMetrics().getSizeOfLogQueue()
-          + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
+      assertEquals(source.getSourceMetrics().getSizeOfLogQueue() + globalLogQueueSizeInitial,
+        globalSource.getSizeOfLogQueue());
     } finally {
       removePeerAndWait(peerId);
     }
@@ -575,8 +578,14 @@ public abstract class TestReplicationSourceManager {
       final boolean waitForSource) throws Exception {
     final ReplicationPeers rp = manager.getReplicationPeers();
     rp.registerPeer(peerId, peerConfig);
+    try {
+      manager.addPeer(peerId);
+    } catch (Exception e) {
+      // ignore the failed exception, because we'll test both success & failed case.
+    }
     waitPeer(peerId, manager, waitForSource);
     if (managerOfCluster != null) {
+      managerOfCluster.addPeer(peerId);
       waitPeer(peerId, managerOfCluster, waitForSource);
     }
   }
@@ -609,6 +618,11 @@ public abstract class TestReplicationSourceManager {
     final ReplicationPeers rp = manager.getReplicationPeers();
     if (rp.getAllPeerIds().contains(peerId)) {
       rp.unregisterPeer(peerId);
+      try {
+        manager.removePeer(peerId);
+      } catch (Exception e) {
+        // ignore the failed exception and continue.
+      }
     }
     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
       @Override public boolean evaluate() throws Exception {


[25/48] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 6e27a21..d8f9625 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
@@ -21,13 +21,13 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -48,17 +48,18 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AtomicLongMap;
 
 /**
@@ -303,57 +304,53 @@ public class DumpReplicationQueues extends Configured implements Tool {
   }
 
   public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set<String> peerIds,
-                           boolean hdfs) throws Exception {
-    ReplicationQueuesClient queuesClient;
+      boolean hdfs) throws Exception {
+    ReplicationQueueStorage queueStorage;
     ReplicationPeers replicationPeers;
     ReplicationQueues replicationQueues;
     ReplicationTracker replicationTracker;
-    ReplicationQueuesClientArguments replicationArgs =
-        new ReplicationQueuesClientArguments(getConf(), new WarnOnlyAbortable(), zkw);
+    ReplicationQueuesArguments replicationArgs =
+        new ReplicationQueuesArguments(getConf(), new WarnOnlyAbortable(), zkw);
     StringBuilder sb = new StringBuilder();
 
-    queuesClient = ReplicationFactory.getReplicationQueuesClient(replicationArgs);
-    queuesClient.init();
+    queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
     replicationQueues = ReplicationFactory.getReplicationQueues(replicationArgs);
-    replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection);
+    replicationPeers =
+        ReplicationFactory.getReplicationPeers(zkw, getConf(), queueStorage, connection);
     replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
       new WarnOnlyAbortable(), new WarnOnlyStoppable());
-    List<String> liveRegionServers = replicationTracker.getListOfRegionServers();
+    Set<String> liveRegionServers = new HashSet<>(replicationTracker.getListOfRegionServers());
 
     // Loops each peer on each RS and dumps the queues
-    try {
-      List<String> regionservers = queuesClient.getListOfReplicators();
-      if (regionservers == null || regionservers.isEmpty()) {
-        return sb.toString();
+    List<ServerName> regionservers = queueStorage.getListOfReplicators();
+    if (regionservers == null || regionservers.isEmpty()) {
+      return sb.toString();
+    }
+    for (ServerName regionserver : regionservers) {
+      List<String> queueIds = queueStorage.getAllQueues(regionserver);
+      replicationQueues.init(regionserver.getServerName());
+      if (!liveRegionServers.contains(regionserver.getServerName())) {
+        deadRegionServers.add(regionserver.getServerName());
       }
-      for (String regionserver : regionservers) {
-        List<String> queueIds = queuesClient.getAllQueues(regionserver);
-        replicationQueues.init(regionserver);
-        if (!liveRegionServers.contains(regionserver)) {
-          deadRegionServers.add(regionserver);
-        }
-        for (String queueId : queueIds) {
-          ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-          List<String> wals = queuesClient.getLogsInQueue(regionserver, queueId);
-          if (!peerIds.contains(queueInfo.getPeerId())) {
-            deletedQueues.add(regionserver + "/" + queueId);
-            sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true,
-              hdfs));
-          } else {
-            sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false,
-              hdfs));
-          }
+      for (String queueId : queueIds) {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        List<String> wals = queueStorage.getWALsInQueue(regionserver, queueId);
+        if (!peerIds.contains(queueInfo.getPeerId())) {
+          deletedQueues.add(regionserver + "/" + queueId);
+          sb.append(
+            formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, hdfs));
+        } else {
+          sb.append(
+            formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, hdfs));
         }
       }
-    } catch (KeeperException ke) {
-      throw new IOException(ke);
     }
     return sb.toString();
   }
 
-  private String formatQueue(String regionserver, ReplicationQueues replicationQueues, ReplicationQueueInfo queueInfo,
-                           String queueId, List<String> wals, boolean isDeleted, boolean hdfs) throws Exception {
-
+  private String formatQueue(ServerName regionserver, ReplicationQueues replicationQueues,
+      ReplicationQueueInfo queueInfo, String queueId, List<String> wals, boolean isDeleted,
+      boolean hdfs) throws Exception {
     StringBuilder sb = new StringBuilder();
 
     List<ServerName> deadServers;
@@ -389,13 +386,14 @@ public class DumpReplicationQueues extends Configured implements Tool {
   /**
    *  return total size in bytes from a list of WALs
    */
-  private long getTotalWALSize(FileSystem fs, List<String> wals, String server) throws IOException {
+  private long getTotalWALSize(FileSystem fs, List<String> wals, ServerName server)
+      throws IOException {
     long size = 0;
     FileStatus fileStatus;
 
     for (String wal : wals) {
       try {
-        fileStatus = (new WALLink(getConf(), server, wal)).getFileStatus(fs);
+        fileStatus = (new WALLink(getConf(), server.getServerName(), wal)).getFileStatus(fs);
       } catch (IOException e) {
         if (e instanceof FileNotFoundException) {
           numWalsNotFound++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 839b5ad..85fa729 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
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.util.hbck;
 
 import java.io.IOException;
@@ -27,22 +26,23 @@ 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.ServerName;
 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.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
 
-/*
+/**
  * Check and fix undeleted replication queues for removed peerId.
  */
 @InterfaceAudience.Private
 public class ReplicationChecker {
   private final ErrorReporter errorReporter;
   // replicator with its queueIds for removed peers
-  private Map<String, List<String>> undeletedQueueIds = new HashMap<>();
+  private Map<ServerName, List<String>> undeletedQueueIds = new HashMap<>();
   // replicator with its undeleted queueIds for removed peers in hfile-refs queue
   private Set<String> undeletedHFileRefsQueueIds = new HashSet<>();
   private final ReplicationZKNodeCleaner cleaner;
@@ -60,8 +60,8 @@ public class ReplicationChecker {
 
   public void checkUnDeletedQueues() throws IOException {
     undeletedQueueIds = cleaner.getUnDeletedQueues();
-    for (Entry<String, List<String>> replicatorAndQueueIds : undeletedQueueIds.entrySet()) {
-      String replicator = replicatorAndQueueIds.getKey();
+    for (Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueueIds.entrySet()) {
+      ServerName replicator = replicatorAndQueueIds.getKey();
       for (String queueId : replicatorAndQueueIds.getValue()) {
         ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
         String msg = "Undeleted replication queue for removed peer found: "

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
index 28a7562..b28eaaf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -34,12 +35,16 @@ import java.util.Set;
 import java.util.concurrent.CompletionException;
 
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 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.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -56,8 +61,8 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
   private final String ID_ONE = "1";
   private final String KEY_ONE = "127.0.0.1:2181:/hbase";
-  private final String ID_SECOND = "2";
-  private final String KEY_SECOND = "127.0.0.1:2181:/hbase2";
+  private final String ID_TWO = "2";
+  private final String KEY_TWO = "127.0.0.1:2181:/hbase2";
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -65,21 +70,27 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 120000);
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
     TEST_UTIL.getConfiguration().setInt(START_LOG_ERRORS_AFTER_COUNT_KEY, 0);
+    TEST_UTIL.getConfiguration().setInt(ReadOnlyZKClient.RECOVERY_RETRY, 1);
     TEST_UTIL.startMiniCluster();
     ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
   }
 
   @After
-  public void cleanupPeer() {
+  public void clearPeerAndQueues() throws IOException, ReplicationException {
     try {
       admin.removeReplicationPeer(ID_ONE).join();
     } catch (Exception e) {
-      LOG.debug("Replication peer " + ID_ONE + " may already be removed");
     }
     try {
-      admin.removeReplicationPeer(ID_SECOND).join();
+      admin.removeReplicationPeer(ID_TWO).join();
     } catch (Exception e) {
-      LOG.debug("Replication peer " + ID_SECOND + " may already be removed");
+    }
+    ReplicationQueueStorage queueStorage = ReplicationStorageFactory
+        .getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), TEST_UTIL.getConfiguration());
+    for (ServerName serverName : queueStorage.getListOfReplicators()) {
+      for (String queue : queueStorage.getAllQueues(serverName)) {
+        queueStorage.removeQueue(serverName, queue);
+      }
     }
   }
 
@@ -88,7 +99,7 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
     rpc1.setClusterKey(KEY_ONE);
     ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
-    rpc2.setClusterKey(KEY_SECOND);
+    rpc2.setClusterKey(KEY_TWO);
     // Add a valid peer
     admin.addReplicationPeer(ID_ONE, rpc1).join();
     // try adding the same (fails)
@@ -101,19 +112,19 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     assertEquals(1, admin.listReplicationPeers().get().size());
     // Try to remove an inexisting peer
     try {
-      admin.removeReplicationPeer(ID_SECOND).join();
+      admin.removeReplicationPeer(ID_TWO).join();
       fail("Test case should fail as removing a inexisting peer.");
     } catch (CompletionException e) {
       // OK!
     }
     assertEquals(1, admin.listReplicationPeers().get().size());
     // Add a second since multi-slave is supported
-    admin.addReplicationPeer(ID_SECOND, rpc2).join();
+    admin.addReplicationPeer(ID_TWO, rpc2).join();
     assertEquals(2, admin.listReplicationPeers().get().size());
     // Remove the first peer we added
     admin.removeReplicationPeer(ID_ONE).join();
     assertEquals(1, admin.listReplicationPeers().get().size());
-    admin.removeReplicationPeer(ID_SECOND).join();
+    admin.removeReplicationPeer(ID_TWO).join();
     assertEquals(0, admin.listReplicationPeers().get().size());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 d612eab..56a6cfe 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
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterR
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -83,6 +84,7 @@ public class TestReplicationAdmin {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
+    TEST_UTIL.getConfiguration().setInt(ReadOnlyZKClient.RECOVERY_RETRY, 1);
     TEST_UTIL.startMiniCluster();
     admin = new ReplicationAdmin(TEST_UTIL.getConfiguration());
     hbaseAdmin = TEST_UTIL.getAdmin();

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 08b27ec..1e75959 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
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -24,16 +24,12 @@ import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.net.URLEncoder;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -51,7 +47,6 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -65,10 +60,11 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 @Category({MasterTests.class, MediumTests.class})
 public class TestLogsCleaner {
 
@@ -195,24 +191,6 @@ public class TestLogsCleaner {
     }
   }
 
-  @Test(timeout=5000)
-  public void testZnodeCversionChange() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
-    cleaner.setConf(conf);
-
-    ReplicationQueuesClientZKImpl rqcMock = Mockito.mock(ReplicationQueuesClientZKImpl.class);
-    Mockito.when(rqcMock.getQueuesZNodeCversion()).thenReturn(1, 2, 3, 4);
-
-    Field rqc = ReplicationLogCleaner.class.getDeclaredField("replicationQueues");
-    rqc.setAccessible(true);
-
-    rqc.set(cleaner, rqcMock);
-
-    // This should return eventually when cversion stabilizes
-    cleaner.getDeletableFiles(new LinkedList<>());
-  }
-
   /**
    * ReplicationLogCleaner should be able to ride over ZooKeeper errors without aborting.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 2948701..f83695f 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
@@ -1,12 +1,19 @@
-/*
- * 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.
+/**
+ * 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.cleaner;
 
@@ -17,14 +24,10 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,7 +48,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
@@ -63,10 +65,11 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 @Category({ MasterTests.class, SmallTests.class })
 public class TestReplicationHFileCleaner {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class);
@@ -188,32 +191,6 @@ public class TestReplicationHFileCleaner {
     assertTrue(deletableFilesIterator.next().getPath().equals(deletablefile));
   }
 
-  /*
-   * Test for HBASE-14621. This test will not assert directly anything. Without the fix the test
-   * will end up in a infinite loop, so it will timeout.
-   */
-  @Test(timeout = 15000)
-  public void testForDifferntHFileRefsZnodeVersion() throws Exception {
-    // 1. Create a file
-    Path file = new Path(root, "testForDifferntHFileRefsZnodeVersion");
-    fs.createNewFile(file);
-    // 2. Assert file is successfully created
-    assertTrue("Test file not created!", fs.exists(file));
-    ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner();
-    cleaner.setConf(conf);
-
-    ReplicationQueuesClient replicationQueuesClient = Mockito.mock(ReplicationQueuesClient.class);
-    //Return different znode version for each call
-    Mockito.when(replicationQueuesClient.getHFileRefsNodeChangeVersion()).thenReturn(1, 2);
-
-    Class<? extends ReplicationHFileCleaner> cleanerClass = cleaner.getClass();
-    Field rqc = cleanerClass.getDeclaredField("rqc");
-    rqc.setAccessible(true);
-    rqc.set(cleaner, replicationQueuesClient);
-
-    cleaner.isFileDeletable(fs.getFileStatus(file));
-  }
-
   /**
    * ReplicationHFileCleaner should be able to ride over ZooKeeper errors without aborting.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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 6aa59cb..8178266 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
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.cleaner;
 
 import static org.junit.Assert.assertEquals;
@@ -26,6 +25,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
@@ -43,9 +43,9 @@ public class TestReplicationZKNodeCleaner {
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private final String ID_ONE = "1";
-  private final String SERVER_ONE = "server1";
+  private final ServerName SERVER_ONE = ServerName.valueOf("server1", 8000, 1234);
   private final String ID_TWO = "2";
-  private final String SERVER_TWO = "server2";
+  private final ServerName SERVER_TWO = ServerName.valueOf("server2", 8000, 1234);
 
   private final Configuration conf;
   private final ZKWatcher zkw;
@@ -72,12 +72,12 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleaner() throws Exception {
-    repQueues.init(SERVER_ONE);
+    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
     repQueues.addLog(ID_ONE, "file1");
 
     ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
-    Map<String, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
+    Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
     assertEquals(1, undeletedQueues.size());
     assertTrue(undeletedQueues.containsKey(SERVER_ONE));
     assertEquals(1, undeletedQueues.get(SERVER_ONE).size());
@@ -100,7 +100,7 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleanerChore() throws Exception {
-    repQueues.init(SERVER_ONE);
+    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
     repQueues.addLog(ID_ONE, "file1");
     // add a recovery queue for ID_TWO which isn't exist

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
deleted file mode 100644
index 29c0930..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ /dev/null
@@ -1,378 +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.replication;
-
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.zookeeper.KeeperException;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * White box testing for replication state interfaces. Implementations should extend this class, and
- * initialize the interfaces properly.
- */
-public abstract class TestReplicationStateBasic {
-
-  protected ReplicationQueues rq1;
-  protected ReplicationQueues rq2;
-  protected ReplicationQueues rq3;
-  protected ReplicationQueuesClient rqc;
-  protected String server1 = ServerName.valueOf("hostname1.example.org", 1234, -1L).toString();
-  protected String server2 = ServerName.valueOf("hostname2.example.org", 1234, -1L).toString();
-  protected String server3 = ServerName.valueOf("hostname3.example.org", 1234, -1L).toString();
-  protected ReplicationPeers rp;
-  protected static final String ID_ONE = "1";
-  protected static final String ID_TWO = "2";
-  protected static String KEY_ONE;
-  protected static String KEY_TWO;
-
-  // For testing when we try to replicate to ourself
-  protected String OUR_ID = "3";
-  protected String OUR_KEY;
-
-  protected static int zkTimeoutCount;
-  protected static final int ZK_MAX_COUNT = 300;
-  protected static final int ZK_SLEEP_INTERVAL = 100; // millis
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class);
-
-  @Before
-  public void setUp() {
-    zkTimeoutCount = 0;
-  }
-
-  @Test
-  public void testReplicationQueuesClient() throws ReplicationException, KeeperException {
-    rqc.init();
-    // Test methods with empty state
-    assertEquals(0, rqc.getListOfReplicators().size());
-    assertNull(rqc.getLogsInQueue(server1, "qId1"));
-    assertNull(rqc.getAllQueues(server1));
-
-    /*
-     * Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each --
-     * server2: zero queues
-     */
-    rq1.init(server1);
-    rq2.init(server2);
-    rq1.addLog("qId1", "trash");
-    rq1.removeLog("qId1", "trash");
-    rq1.addLog("qId2", "filename1");
-    rq1.addLog("qId3", "filename2");
-    rq1.addLog("qId3", "filename3");
-    rq2.addLog("trash", "trash");
-    rq2.removeQueue("trash");
-
-    List<String> reps = rqc.getListOfReplicators();
-    assertEquals(2, reps.size());
-    assertTrue(server1, reps.contains(server1));
-    assertTrue(server2, reps.contains(server2));
-
-    assertNull(rqc.getLogsInQueue("bogus", "bogus"));
-    assertNull(rqc.getLogsInQueue(server1, "bogus"));
-    assertEquals(0, rqc.getLogsInQueue(server1, "qId1").size());
-    assertEquals(1, rqc.getLogsInQueue(server1, "qId2").size());
-    assertEquals("filename1", rqc.getLogsInQueue(server1, "qId2").get(0));
-
-    assertNull(rqc.getAllQueues("bogus"));
-    assertEquals(0, rqc.getAllQueues(server2).size());
-    List<String> list = rqc.getAllQueues(server1);
-    assertEquals(3, list.size());
-    assertTrue(list.contains("qId2"));
-    assertTrue(list.contains("qId3"));
-  }
-
-  @Test
-  public void testReplicationQueues() throws ReplicationException {
-    rq1.init(server1);
-    rq2.init(server2);
-    rq3.init(server3);
-    //Initialize ReplicationPeer so we can add peers (we don't transfer lone queues)
-    rp.init();
-
-    // 3 replicators should exist
-    assertEquals(3, rq1.getListOfReplicators().size());
-    rq1.removeQueue("bogus");
-    rq1.removeLog("bogus", "bogus");
-    rq1.removeAllQueues();
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
-    assertNull(rq1.getLogsInQueue("bogus"));
-    assertNull(rq1.getUnClaimedQueueIds(
-        ServerName.valueOf("bogus", 1234, -1L).toString()));
-
-    rq1.setLogPosition("bogus", "bogus", 5L);
-
-    populateQueues();
-
-    assertEquals(3, rq1.getListOfReplicators().size());
-    assertEquals(0, rq2.getLogsInQueue("qId1").size());
-    assertEquals(5, rq3.getLogsInQueue("qId5").size());
-    assertEquals(0, rq3.getLogPosition("qId1", "filename0"));
-    rq3.setLogPosition("qId5", "filename4", 354L);
-    assertEquals(354L, rq3.getLogPosition("qId5", "filename4"));
-
-    assertEquals(5, rq3.getLogsInQueue("qId5").size());
-    assertEquals(0, rq2.getLogsInQueue("qId1").size());
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(1, rq2.getAllQueues().size());
-    assertEquals(5, rq3.getAllQueues().size());
-
-    assertEquals(0, rq3.getUnClaimedQueueIds(server1).size());
-    rq3.removeReplicatorIfQueueIsEmpty(server1);
-    assertEquals(2, rq3.getListOfReplicators().size());
-
-    List<String> queues = rq2.getUnClaimedQueueIds(server3);
-    assertEquals(5, queues.size());
-    for(String queue: queues) {
-      rq2.claimQueue(server3, queue);
-    }
-    rq2.removeReplicatorIfQueueIsEmpty(server3);
-    assertEquals(1, rq2.getListOfReplicators().size());
-
-    // Try to claim our own queues
-    assertNull(rq2.getUnClaimedQueueIds(server2));
-    rq2.removeReplicatorIfQueueIsEmpty(server2);
-
-    assertEquals(6, rq2.getAllQueues().size());
-
-    rq2.removeAllQueues();
-
-    assertEquals(0, rq2.getListOfReplicators().size());
-  }
-
-  @Test
-  public void testInvalidClusterKeys() throws ReplicationException, KeeperException {
-    rp.init();
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"));
-      fail("Should throw an IllegalArgumentException because "
-            + "zookeeper.znode.parent is missing leading '/'.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"));
-      fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"));
-      fail("Should throw an IllegalArgumentException because "
-          + "hbase.zookeeper.property.clientPort is missing.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-  }
-
-  @Test
-  public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
-    rp.init();
-    rq1.init(server1);
-    rqc.init();
-
-    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
-    files1.add(new Pair<>(null, new Path("file_1")));
-    files1.add(new Pair<>(null, new Path("file_2")));
-    files1.add(new Pair<>(null, new Path("file_3")));
-    assertNull(rqc.getReplicableHFiles(ID_ONE));
-    assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rq1.addPeerToHFileRefs(ID_ONE);
-    rq1.addHFileRefs(ID_ONE, files1);
-    assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
-    assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
-    List<String> hfiles2 = new ArrayList<>(files1.size());
-    for (Pair<Path, Path> p : files1) {
-      hfiles2.add(p.getSecond().getName());
-    }
-    String removedString = hfiles2.remove(0);
-    rq1.removeHFileRefs(ID_ONE, hfiles2);
-    assertEquals(1, rqc.getReplicableHFiles(ID_ONE).size());
-    hfiles2 = new ArrayList<>(1);
-    hfiles2.add(removedString);
-    rq1.removeHFileRefs(ID_ONE, hfiles2);
-    assertEquals(0, rqc.getReplicableHFiles(ID_ONE).size());
-    rp.unregisterPeer(ID_ONE);
-  }
-
-  @Test
-  public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
-    rq1.init(server1);
-    rqc.init();
-
-    rp.init();
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rq1.addPeerToHFileRefs(ID_ONE);
-    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
-    rq1.addPeerToHFileRefs(ID_TWO);
-
-    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
-    files1.add(new Pair<>(null, new Path("file_1")));
-    files1.add(new Pair<>(null, new Path("file_2")));
-    files1.add(new Pair<>(null, new Path("file_3")));
-    rq1.addHFileRefs(ID_ONE, files1);
-    rq1.addHFileRefs(ID_TWO, files1);
-    assertEquals(2, rqc.getAllPeersFromHFileRefsQueue().size());
-    assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
-    assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
-
-    rp.unregisterPeer(ID_ONE);
-    rq1.removePeerFromHFileRefs(ID_ONE);
-    assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
-    assertNull(rqc.getReplicableHFiles(ID_ONE));
-    assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
-
-    rp.unregisterPeer(ID_TWO);
-    rq1.removePeerFromHFileRefs(ID_TWO);
-    assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
-    assertNull(rqc.getReplicableHFiles(ID_TWO));
-  }
-
-  @Test
-  public void testReplicationPeers() throws Exception {
-    rp.init();
-
-    // Test methods with non-existent peer ids
-    try {
-      rp.unregisterPeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
-    try {
-      rp.enablePeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
-    try {
-      rp.disablePeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
-    try {
-      rp.getStatusOfPeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
-    assertFalse(rp.peerConnected("bogus"));
-    rp.peerDisconnected("bogus");
-
-    assertNull(rp.getPeerConf("bogus"));
-    assertNumberOfPeers(0);
-
-    // Add some peers
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    assertNumberOfPeers(1);
-    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
-    assertNumberOfPeers(2);
-
-    // Test methods with a peer that is added but not connected
-    try {
-      rp.getStatusOfPeer(ID_ONE);
-      fail("There are no connected peers, should have thrown an IllegalArgumentException");
-    } catch (IllegalArgumentException e) {
-    }
-    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond()));
-    rp.unregisterPeer(ID_ONE);
-    rp.peerDisconnected(ID_ONE);
-    assertNumberOfPeers(1);
-
-    // Add one peer
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rp.peerConnected(ID_ONE);
-    assertNumberOfPeers(2);
-    assertTrue(rp.getStatusOfPeer(ID_ONE));
-    rp.disablePeer(ID_ONE);
-    assertConnectedPeerStatus(false, ID_ONE);
-    rp.enablePeer(ID_ONE);
-    assertConnectedPeerStatus(true, ID_ONE);
-
-    // Disconnect peer
-    rp.peerDisconnected(ID_ONE);
-    assertNumberOfPeers(2);
-    try {
-      rp.getStatusOfPeer(ID_ONE);
-      fail("There are no connected peers, should have thrown an IllegalArgumentException");
-    } catch (IllegalArgumentException e) {
-    }
-  }
-
-  protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
-    // we can first check if the value was changed in the store, if it wasn't then fail right away
-    if (status != rp.getStatusOfPeerFromBackingStore(peerId)) {
-      fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK");
-    }
-    while (true) {
-      if (status == rp.getStatusOfPeer(peerId)) {
-        return;
-      }
-      if (zkTimeoutCount < ZK_MAX_COUNT) {
-        LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status
-            + ", sleeping and trying again.");
-        Thread.sleep(ZK_SLEEP_INTERVAL);
-      } else {
-        fail("Timed out waiting for ConnectedPeerStatus to be " + status);
-      }
-    }
-  }
-
-  protected void assertNumberOfPeers(int total) {
-    assertEquals(total, rp.getAllPeerConfigs().size());
-    assertEquals(total, rp.getAllPeerIds().size());
-    assertEquals(total, rp.getAllPeerIds().size());
-  }
-
-  /*
-   * three replicators: rq1 has 0 queues, rq2 has 1 queue with no logs, rq3 has 5 queues with 1, 2,
-   * 3, 4, 5 log files respectively
-   */
-  protected void populateQueues() throws ReplicationException {
-    rq1.addLog("trash", "trash");
-    rq1.removeQueue("trash");
-
-    rq2.addLog("qId1", "trash");
-    rq2.removeLog("qId1", "trash");
-
-    for (int i = 1; i < 6; i++) {
-      for (int j = 0; j < i; j++) {
-        rq3.addLog("qId" + i, "filename" + j);
-      }
-      //Add peers for the corresponding queues so they are not orphans
-      rp.registerPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i));
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/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
deleted file mode 100644
index 231d655..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ /dev/null
@@ -1,227 +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.replication;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.ChoreService;
-import org.apache.hadoop.hbase.ClusterId;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
-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.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.zookeeper.KeeperException;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Category({ReplicationTests.class, MediumTests.class})
-public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateZKImpl.class);
-
-  private static Configuration conf;
-  private static HBaseTestingUtility utility;
-  private static ZKWatcher zkw;
-  private static String replicationZNode;
-  private ReplicationQueuesZKImpl rqZK;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    utility = new HBaseTestingUtility();
-    utility.startMiniZKCluster();
-    conf = utility.getConfiguration();
-    conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
-    zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
-    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
-    replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
-    KEY_ONE = initPeerClusterState("/hbase1");
-    KEY_TWO = initPeerClusterState("/hbase2");
-  }
-
-  private static String initPeerClusterState(String baseZKNode)
-      throws IOException, KeeperException {
-    // Add a dummy region server and set up the cluster id
-    Configuration testConf = new Configuration(conf);
-    testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode);
-    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());
-    return ZKConfig.getZooKeeperClusterKey(testConf);
-  }
-
-  @Before
-  @Override
-  public void setUp() {
-    super.setUp();
-    DummyServer ds1 = new DummyServer(server1);
-    DummyServer ds2 = new DummyServer(server2);
-    DummyServer ds3 = new DummyServer(server3);
-    try {
-      rq1 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds1, zkw));
-      rq2 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds2, zkw));
-      rq3 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds3, zkw));
-      rqc = ReplicationFactory.getReplicationQueuesClient(
-        new ReplicationQueuesClientArguments(conf, ds1, zkw));
-    } catch (Exception e) {
-      // This should not occur, because getReplicationQueues() only throws for
-      // TableBasedReplicationQueuesImpl
-      fail("ReplicationFactory.getReplicationQueues() threw an IO Exception");
-    }
-    rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
-    OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
-    rqZK = new ReplicationQueuesZKImpl(zkw, conf, ds1);
-  }
-
-  @After
-  public void tearDown() throws KeeperException, IOException {
-    ZKUtil.deleteNodeRecursively(zkw, replicationZNode);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    utility.shutdownMiniZKCluster();
-  }
-
-  @Test
-  public void testIsPeerPath_PathToParentOfPeerNode() {
-    assertFalse(rqZK.isPeerPath(rqZK.peersZNode));
-  }
-
-  @Test
-  public void testIsPeerPath_PathToChildOfPeerNode() {
-    String peerChild = ZNodePaths.joinZNode(ZNodePaths.joinZNode(rqZK.peersZNode, "1"), "child");
-    assertFalse(rqZK.isPeerPath(peerChild));
-  }
-
-  @Test
-  public void testIsPeerPath_ActualPeerPath() {
-    String peerPath = ZNodePaths.joinZNode(rqZK.peersZNode, "1");
-    assertTrue(rqZK.isPeerPath(peerPath));
-  }
-
-  static class DummyServer implements Server {
-    private String serverName;
-    private boolean isAborted = false;
-    private boolean isStopped = false;
-
-    public DummyServer(String serverName) {
-      this.serverName = serverName;
-    }
-
-    @Override
-    public Configuration getConfiguration() {
-      return conf;
-    }
-
-    @Override
-    public ZKWatcher getZooKeeper() {
-      return zkw;
-    }
-
-    @Override
-    public CoordinatedStateManager getCoordinatedStateManager() {
-      return null;
-    }
-
-    @Override
-    public ClusterConnection getConnection() {
-      return null;
-    }
-
-    @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
-    public ServerName getServerName() {
-      return ServerName.valueOf(this.serverName);
-    }
-
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.info("Aborting " + serverName);
-      this.isAborted = true;
-    }
-
-    @Override
-    public boolean isAborted() {
-      return this.isAborted;
-    }
-
-    @Override
-    public void stop(String why) {
-      this.isStopped = true;
-    }
-
-    @Override
-    public boolean isStopped() {
-      return this.isStopped;
-    }
-
-    @Override
-    public ChoreService getChoreService() {
-      return null;
-    }
-
-    @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
-    public FileSystem getFileSystem() {
-      return null;
-    }
-
-    @Override
-    public boolean isStopping() {
-      return false;
-    }
-
-    @Override
-    public Connection createConnection(Configuration conf) throws IOException {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/050617da/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
index b47a8d3..aeab8b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
@@ -1,34 +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.
-*/
+/**
+ * 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.replication.regionserver;
 
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -36,11 +36,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 /**
  * Tests the ReplicationSourceManager with ReplicationQueueZkImpl's and
  * ReplicationQueuesClientZkImpl. Also includes extra tests outside of those in
@@ -114,41 +109,4 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan
 
     server.stop("");
   }
-
-  @Test
-  public void testFailoverDeadServerCversionChange() throws Exception {
-    final Server s0 = new DummyServer("cversion-change0.example.org");
-    ReplicationQueues repQueues =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, s0,
-        s0.getZooKeeper()));
-    repQueues.init(s0.getServerName().toString());
-    // populate some znodes in the peer znode
-    files.add("log1");
-    files.add("log2");
-    for (String file : files) {
-      repQueues.addLog("1", file);
-    }
-    // simulate queue transfer
-    Server s1 = new DummyServer("cversion-change1.example.org");
-    ReplicationQueues rq1 =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1,
-        s1.getZooKeeper()));
-    rq1.init(s1.getServerName().toString());
-
-    ReplicationQueuesClientZKImpl client =
-      (ReplicationQueuesClientZKImpl)ReplicationFactory.getReplicationQueuesClient(
-        new ReplicationQueuesClientArguments(s1.getConfiguration(), s1, s1.getZooKeeper()));
-
-    int v0 = client.getQueuesZNodeCversion();
-    List<String> queues = rq1.getUnClaimedQueueIds(s0.getServerName().getServerName());
-    for(String queue : queues) {
-      rq1.claimQueue(s0.getServerName().getServerName(), queue);
-    }
-    rq1.removeReplicatorIfQueueIsEmpty(s0.getServerName().getServerName());
-    int v1 = client.getQueuesZNodeCversion();
-    // cversion should increase by 1 since a child node is deleted
-    assertEquals(v0 + 1, v1);
-
-    s0.stop("");
-  }
 }


[04/48] hbase git commit: HBASE-19752 RSGroupBasedLoadBalancer#getMisplacedRegions() should handle the case where rs group cannot be determined

Posted by zh...@apache.org.
HBASE-19752 RSGroupBasedLoadBalancer#getMisplacedRegions() should handle the case where rs group cannot be determined


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 9ed52ee3e5dd6cc7a3c02953e0afab11de8dd680
Parents: b662080
Author: tedyu <yu...@gmail.com>
Authored: Fri Jan 12 12:16:06 2018 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jan 12 12:16:06 2018 -0800

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupBasedLoadBalancer.java    | 17 ++++++++++++-----
 .../balancer/TestRSGroupBasedLoadBalancer.java     | 17 +++++++++++++++++
 2 files changed, 29 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9ed52ee3/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
index 1c70925..619de9e 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
@@ -300,7 +300,8 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
     return finalList;
   }
 
-  private Set<RegionInfo> getMisplacedRegions(
+  @VisibleForTesting
+  public Set<RegionInfo> getMisplacedRegions(
       Map<RegionInfo, ServerName> regions) throws IOException {
     Set<RegionInfo> misplacedRegions = new HashSet<>();
     for(Map.Entry<RegionInfo, ServerName> region : regions.entrySet()) {
@@ -308,10 +309,16 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
       ServerName assignedServer = region.getValue();
       RSGroupInfo info = rsGroupInfoManager.getRSGroup(rsGroupInfoManager.
               getRSGroupOfTable(regionInfo.getTable()));
-      if (assignedServer != null &&
-          (info == null || !info.containsServer(assignedServer.getAddress()))) {
-        RSGroupInfo otherInfo = null;
-        otherInfo = rsGroupInfoManager.getRSGroupOfServer(assignedServer.getAddress());
+      if (assignedServer == null) {
+        LOG.debug("There is no assigned server for {}", region);
+        continue;
+      }
+      RSGroupInfo otherInfo = rsGroupInfoManager.getRSGroupOfServer(assignedServer.getAddress());
+      if (info == null && otherInfo == null) {
+        LOG.warn("Couldn't obtain rs group information for {} on {}", region, assignedServer);
+        continue;
+      }
+      if ((info == null || !info.containsServer(assignedServer.getAddress()))) {
         LOG.debug("Found misplaced region: " + regionInfo.getRegionNameAsString() +
             " on server: " + assignedServer +
             " found in group: " +  otherInfo +

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ed52ee3/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
index e234438..68845d7 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
@@ -76,6 +76,7 @@ public class TestRSGroupBasedLoadBalancer {
 
   static String[]  groups = new String[] { RSGroupInfo.DEFAULT_GROUP, "dg2", "dg3",
       "dg4" };
+  static TableName table0 = TableName.valueOf("dt0");
   static TableName[] tables =
       new TableName[] { TableName.valueOf("dt1"),
           TableName.valueOf("dt2"),
@@ -225,6 +226,20 @@ public class TestRSGroupBasedLoadBalancer {
     assertClusterAsBalanced(loadMap);
   }
 
+  @Test
+  public void testGetMisplacedRegions() throws Exception {
+    // Test case where region is not considered misplaced if RSGroupInfo cannot be determined
+    Map<RegionInfo, ServerName> inputForTest = new HashMap<>();
+    RegionInfo ri = RegionInfoBuilder.newBuilder(table0)
+        .setStartKey(new byte[16])
+        .setEndKey(new byte[16])
+        .setSplit(false)
+        .setRegionId(regionId++)
+        .build();
+    inputForTest.put(ri, servers.iterator().next());
+    Set<RegionInfo> misplacedRegions = loadBalancer.getMisplacedRegions(inputForTest);
+    assertFalse(misplacedRegions.contains(ri));
+  }
   /**
    * Test the cluster startup bulk assignment which attempts to retain
    * assignment info.
@@ -552,6 +567,8 @@ public class TestRSGroupBasedLoadBalancer {
       tableMap.put(tables[i], groupName);
       tds.add(htd);
     }
+    tableMap.put(table0, "");
+    tds.add(new HTableDescriptor(table0));
     return tds;
   }
 


[38/48] hbase git commit: HBASE-19636 All rs should already start work with the new peer change when replication peer procedure is finished

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index fc978be..e087127 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -16,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
@@ -33,7 +31,7 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
@@ -70,27 +68,53 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
- * This class is responsible to manage all the replication
- * sources. There are two classes of sources:
+ * This class is responsible to manage all the replication sources. There are two classes of
+ * sources:
  * <ul>
- * <li> Normal sources are persistent and one per peer cluster</li>
- * <li> Old sources are recovered from a failed region server and our
- * only goal is to finish replicating the WAL queue it had up in ZK</li>
+ * <li>Normal sources are persistent and one per peer cluster</li>
+ * <li>Old sources are recovered from a failed region server and our only goal is to finish
+ * replicating the WAL queue it had</li>
+ * </ul>
+ * <p>
+ * When a region server dies, this class uses a watcher to get notified and it tries to grab a lock
+ * in order to transfer all the queues in a local old source.
+ * <p>
+ * Synchronization specification:
+ * <ul>
+ * <li>No need synchronized on {@link #sources}. {@link #sources} is a ConcurrentHashMap and there
+ * is a Lock for peer id in {@link PeerProcedureHandlerImpl}. So there is no race for peer
+ * operations.</li>
+ * <li>Need synchronized on {@link #walsById}. There are four methods which modify it,
+ * {@link #addPeer(String)}, {@link #removePeer(String)},
+ * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}. {@link #walsById}
+ * is a ConcurrentHashMap and there is a Lock for peer id in {@link PeerProcedureHandlerImpl}. So
+ * there is no race between {@link #addPeer(String)} and {@link #removePeer(String)}.
+ * {@link #cleanOldLogs(SortedSet, String, String)} is called by {@link ReplicationSourceInterface}.
+ * So no race with {@link #addPeer(String)}. {@link #removePeer(String)} will terminate the
+ * {@link ReplicationSourceInterface} firstly, then remove the wals from {@link #walsById}. So no
+ * race with {@link #removePeer(String)}. The only case need synchronized is
+ * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}.</li>
+ * <li>No need synchronized on {@link #walsByIdRecoveredQueues}. There are three methods which
+ * modify it, {@link #removePeer(String)} , {@link #cleanOldLogs(SortedSet, String, String)} and
+ * {@link ReplicationSourceManager.NodeFailoverWorker#run()}.
+ * {@link #cleanOldLogs(SortedSet, String, String)} is called by {@link ReplicationSourceInterface}.
+ * {@link #removePeer(String)} will terminate the {@link ReplicationSourceInterface} firstly, then
+ * remove the wals from {@link #walsByIdRecoveredQueues}. And
+ * {@link ReplicationSourceManager.NodeFailoverWorker#run()} will add the wals to
+ * {@link #walsByIdRecoveredQueues} firstly, then start up a {@link ReplicationSourceInterface}. So
+ * there is no race here. For {@link ReplicationSourceManager.NodeFailoverWorker#run()} and
+ * {@link #removePeer(String)}, there is already synchronized on {@link #oldsources}. So no need
+ * synchronized on {@link #walsByIdRecoveredQueues}.</li>
+ * <li>Need synchronized on {@link #latestPaths} to avoid the new open source miss new log.</li>
+ * <li>Need synchronized on {@link #oldsources} to avoid adding recovered source for the
+ * to-be-removed peer.</li>
  * </ul>
- *
- * When a region server dies, this class uses a watcher to get notified and it
- * tries to grab a lock in order to transfer all the queues in a local
- * old source.
- *
- * This class implements the ReplicationListener interface so that it can track changes in
- * replication state.
  */
 @InterfaceAudience.Private
 public class ReplicationSourceManager implements ReplicationListener {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ReplicationSourceManager.class);
-  // List of all the sources that read this RS's logs
-  private final List<ReplicationSourceInterface> sources;
+  private static final Logger LOG = LoggerFactory.getLogger(ReplicationSourceManager.class);
+  // all the sources that read this RS's logs and every peer only has one replication source
+  private final ConcurrentMap<String, ReplicationSourceInterface> sources;
   // List of all the sources we got from died RSs
   private final List<ReplicationSourceInterface> oldsources;
   private final ReplicationQueueStorage queueStorage;
@@ -100,11 +124,16 @@ public class ReplicationSourceManager implements ReplicationListener {
   private final UUID clusterId;
   // All about stopping
   private final Server server;
+
   // All logs we are currently tracking
-  // Index structure of the map is: peer_id->logPrefix/logGroup->logs
-  private final Map<String, Map<String, SortedSet<String>>> walsById;
+  // Index structure of the map is: queue_id->logPrefix/logGroup->logs
+  // For normal replication source, the peer id is same with the queue id
+  private final ConcurrentMap<String, Map<String, SortedSet<String>>> walsById;
   // Logs for recovered sources we are currently tracking
-  private final Map<String, Map<String, SortedSet<String>>> walsByIdRecoveredQueues;
+  // the map is: queue_id->logPrefix/logGroup->logs
+  // For recovered source, the queue id's format is peer_id-servername-*
+  private final ConcurrentMap<String, Map<String, SortedSet<String>>> walsByIdRecoveredQueues;
+
   private final Configuration conf;
   private final FileSystem fs;
   // The paths to the latest log of each wal group, for new coming peers
@@ -142,22 +171,22 @@ public class ReplicationSourceManager implements ReplicationListener {
       ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf,
       Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId,
       WALFileLengthProvider walFileLengthProvider) throws IOException {
-    //CopyOnWriteArrayList is thread-safe.
-    //Generally, reading is more than modifying.
-    this.sources = new CopyOnWriteArrayList<>();
+    // CopyOnWriteArrayList is thread-safe.
+    // Generally, reading is more than modifying.
+    this.sources = new ConcurrentHashMap<>();
     this.queueStorage = queueStorage;
     this.replicationPeers = replicationPeers;
     this.replicationTracker = replicationTracker;
     this.server = server;
-    this.walsById = new HashMap<>();
+    this.walsById = new ConcurrentHashMap<>();
     this.walsByIdRecoveredQueues = new ConcurrentHashMap<>();
-    this.oldsources = new CopyOnWriteArrayList<>();
+    this.oldsources = new ArrayList<>();
     this.conf = conf;
     this.fs = fs;
     this.logDir = logDir;
     this.oldLogDir = oldLogDir;
-    this.sleepBeforeFailover =
-        conf.getLong("replication.sleep.before.failover", 30000); // 30 seconds
+    this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 30000); // 30
+                                                                                         // seconds
     this.clusterId = clusterId;
     this.walFileLengthProvider = walFileLengthProvider;
     this.replicationTracker.registerListener(this);
@@ -166,89 +195,36 @@ public class ReplicationSourceManager implements ReplicationListener {
     int nbWorkers = conf.getInt("replication.executor.workers", 1);
     // use a short 100ms sleep since this could be done inline with a RS startup
     // even if we fail, other region servers can take care of it
-    this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers,
-        100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>());
+    this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers, 100, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<>());
     ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
     tfb.setNameFormat("ReplicationExecutor-%d");
     tfb.setDaemon(true);
     this.executor.setThreadFactory(tfb.build());
     this.latestPaths = new HashSet<Path>();
-    replicationForBulkLoadDataEnabled =
-        conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
-          HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
+    replicationForBulkLoadDataEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
     this.replicationWaitTime = conf.getLong(HConstants.REPLICATION_SERIALLY_WAITING_KEY,
-          HConstants.REPLICATION_SERIALLY_WAITING_DEFAULT);
+      HConstants.REPLICATION_SERIALLY_WAITING_DEFAULT);
     connection = ConnectionFactory.createConnection(conf);
   }
 
-  @FunctionalInterface
-  private interface ReplicationQueueOperation {
-    void exec() throws ReplicationException;
-  }
-
-  private void abortWhenFail(ReplicationQueueOperation op) {
-    try {
-      op.exec();
-    } catch (ReplicationException e) {
-      server.abort("Failed to operate on replication queue", e);
-    }
-  }
-
   /**
-   * Provide the id of the peer and a log key and this method will figure which
-   * wal it belongs to and will log, for this region server, the current
-   * position. It will also clean old logs from the queue.
-   * @param log Path to the log currently being replicated from
-   * replication status in zookeeper. It will also delete older entries.
-   * @param id id of the peer cluster
-   * @param position current location in the log
-   * @param queueRecovered indicates if this queue comes from another region server
-   * @param holdLogInZK if true then the log is retained in ZK
-   */
-  public void logPositionAndCleanOldLogs(Path log, String id, long position, boolean queueRecovered,
-      boolean holdLogInZK) {
-    String fileName = log.getName();
-    abortWhenFail(
-      () -> this.queueStorage.setWALPosition(server.getServerName(), id, fileName, position));
-    if (holdLogInZK) {
-      return;
-    }
-    cleanOldLogs(fileName, id, queueRecovered);
-  }
-
-  /**
-   * Cleans a log file and all older files from ZK. Called when we are sure that a
-   * log file is closed and has no more entries.
-   * @param key Path to the log
-   * @param id id of the peer cluster
-   * @param queueRecovered Whether this is a recovered queue
+   * Adds a normal source per registered peer cluster and tries to process all old region server wal
+   * queues
+   * <p>
+   * The returned future is for adoptAbandonedQueues task.
    */
-  public void cleanOldLogs(String key, String id, boolean queueRecovered) {
-    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(key);
-    if (queueRecovered) {
-      SortedSet<String> wals = walsByIdRecoveredQueues.get(id).get(logPrefix);
-      if (wals != null && !wals.first().equals(key)) {
-        cleanOldLogs(wals, key, id);
-      }
-    } else {
-      synchronized (this.walsById) {
-        SortedSet<String> wals = walsById.get(id).get(logPrefix);
-        if (wals != null && !wals.first().equals(key)) {
-          cleanOldLogs(wals, key, id);
-        }
+  Future<?> init() throws IOException {
+    for (String id : this.replicationPeers.getAllPeerIds()) {
+      addSource(id);
+      if (replicationForBulkLoadDataEnabled) {
+        // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
+        // when a peer was added before replication for bulk loaded data was enabled.
+        throwIOExceptionWhenFail(() -> this.queueStorage.addPeerToHFileRefs(id));
       }
     }
-  }
-
-  private void cleanOldLogs(SortedSet<String> wals, String key, String id) {
-    SortedSet<String> walSet = wals.headSet(key);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
-    }
-    for (String wal : walSet) {
-      abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), id, wal));
-    }
-    walSet.clear();
+    return this.executor.submit(this::adoptAbandonedQueues);
   }
 
   private void adoptAbandonedQueues() {
@@ -264,8 +240,8 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
     List<ServerName> otherRegionServers = replicationTracker.getListOfRegionServers().stream()
         .map(ServerName::valueOf).collect(Collectors.toList());
-    LOG.info("Current list of replicators: " + currentReplicators + " other RSs: "
-        + otherRegionServers);
+    LOG.info(
+      "Current list of replicators: " + currentReplicators + " other RSs: " + otherRegionServers);
 
     // Look if there's anything to process after a restart
     for (ServerName rs : currentReplicators) {
@@ -276,56 +252,112 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
-   * Adds a normal source per registered peer cluster and tries to process all old region server wal
-   * queues
-   * <p>
-   * The returned future is for adoptAbandonedQueues task.
+   * 1. Add peer to replicationPeers 2. Add the normal source and related replication queue 3. Add
+   * HFile Refs
+   * @param peerId the id of replication peer
    */
-  Future<?> init() throws IOException, ReplicationException {
-    for (String id : this.replicationPeers.getAllPeerIds()) {
-      addSource(id);
+  public void addPeer(String peerId) throws IOException {
+    boolean added = false;
+    try {
+      added = this.replicationPeers.addPeer(peerId);
+    } catch (ReplicationException e) {
+      throw new IOException(e);
+    }
+    if (added) {
+      addSource(peerId);
       if (replicationForBulkLoadDataEnabled) {
-        // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
-        // when a peer was added before replication for bulk loaded data was enabled.
-        this.queueStorage.addPeerToHFileRefs(id);
+        throwIOExceptionWhenFail(() -> this.queueStorage.addPeerToHFileRefs(peerId));
       }
     }
-    return this.executor.submit(this::adoptAbandonedQueues);
   }
 
   /**
-   * Add sources for the given peer cluster on this region server. For the newly added peer, we only
-   * need to enqueue the latest log of each wal group and do replication
-   * @param id the id of the peer cluster
+   * 1. Remove peer for replicationPeers 2. Remove all the recovered sources for the specified id
+   * and related replication queues 3. Remove the normal source and related replication queue 4.
+   * Remove HFile Refs
+   * @param peerId the id of the replication peer
+   */
+  public void removePeer(String peerId) {
+    replicationPeers.removePeer(peerId);
+    String terminateMessage = "Replication stream was removed by a user";
+    List<ReplicationSourceInterface> oldSourcesToDelete = new ArrayList<>();
+    // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
+    // see NodeFailoverWorker.run
+    synchronized (this.oldsources) {
+      // First close all the recovered sources for this peer
+      for (ReplicationSourceInterface src : oldsources) {
+        if (peerId.equals(src.getPeerId())) {
+          oldSourcesToDelete.add(src);
+        }
+      }
+      for (ReplicationSourceInterface src : oldSourcesToDelete) {
+        src.terminate(terminateMessage);
+        removeRecoveredSource(src);
+      }
+    }
+    LOG.info(
+      "Number of deleted recovered sources for " + peerId + ": " + oldSourcesToDelete.size());
+    // Now close the normal source for this peer
+    ReplicationSourceInterface srcToRemove = this.sources.get(peerId);
+    if (srcToRemove != null) {
+      srcToRemove.terminate(terminateMessage);
+      removeSource(srcToRemove);
+    } else {
+      // This only happened in unit test TestReplicationSourceManager#testPeerRemovalCleanup
+      // Delete queue from storage and memory and queue id is same with peer id for normal
+      // source
+      deleteQueue(peerId);
+      this.walsById.remove(peerId);
+    }
+
+    // Remove HFile Refs
+    abortWhenFail(() -> this.queueStorage.removePeerFromHFileRefs(peerId));
+  }
+
+  /**
+   * Factory method to create a replication source
+   * @param queueId the id of the replication queue
+   * @return the created source
+   */
+  private ReplicationSourceInterface createSource(String queueId, ReplicationPeer replicationPeer)
+      throws IOException {
+    ReplicationSourceInterface src = ReplicationSourceFactory.create(conf, queueId);
+
+    MetricsSource metrics = new MetricsSource(queueId);
+    // init replication source
+    src.init(conf, fs, this, queueStorage, replicationPeer, server, queueId, clusterId,
+      walFileLengthProvider, metrics);
+    return src;
+  }
+
+  /**
+   * Add a normal source for the given peer on this region server. Meanwhile, add new replication
+   * queue to storage. For the newly added peer, we only need to enqueue the latest log of each wal
+   * group and do replication
+   * @param peerId the id of the replication peer
    * @return the source that was created
    */
   @VisibleForTesting
-  ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException {
-    ReplicationPeer peer = replicationPeers.getPeer(id);
-    ReplicationSourceInterface src = getReplicationSource(id, peer);
-    synchronized (this.walsById) {
-      this.sources.add(src);
+  ReplicationSourceInterface addSource(String peerId) throws IOException {
+    ReplicationPeer peer = replicationPeers.getPeer(peerId);
+    ReplicationSourceInterface src = createSource(peerId, peer);
+    // synchronized on latestPaths to avoid missing the new log
+    synchronized (this.latestPaths) {
+      this.sources.put(peerId, src);
       Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
-      this.walsById.put(id, walsByGroup);
+      this.walsById.put(peerId, walsByGroup);
       // Add the latest wal to that source's queue
-      synchronized (latestPaths) {
-        if (this.latestPaths.size() > 0) {
-          for (Path logPath : latestPaths) {
-            String name = logPath.getName();
-            String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(name);
-            SortedSet<String> logs = new TreeSet<>();
-            logs.add(name);
-            walsByGroup.put(walPrefix, logs);
-            try {
-              this.queueStorage.addWAL(server.getServerName(), id, name);
-            } catch (ReplicationException e) {
-              String message = "Cannot add log to queue when creating a new source, queueId=" + id +
-                ", filename=" + name;
-              server.stop(message);
-              throw e;
-            }
-            src.enqueueLog(logPath);
-          }
+      if (this.latestPaths.size() > 0) {
+        for (Path logPath : latestPaths) {
+          String name = logPath.getName();
+          String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(name);
+          SortedSet<String> logs = new TreeSet<>();
+          logs.add(name);
+          walsByGroup.put(walPrefix, logs);
+          // Abort RS and throw exception to make add peer failed
+          abortAndThrowIOExceptionWhenFail(
+            () -> this.queueStorage.addWAL(server.getServerName(), peerId, name));
+          src.enqueueLog(logPath);
         }
       }
     }
@@ -333,87 +365,217 @@ public class ReplicationSourceManager implements ReplicationListener {
     return src;
   }
 
-  @VisibleForTesting
-  int getSizeOfLatestPath() {
-    synchronized (latestPaths) {
-      return latestPaths.size();
-    }
-  }
-
   /**
-   * Delete a complete queue of wals associated with a peer cluster
-   * @param peerId Id of the peer cluster queue of wals to delete
+   * Close the previous replication sources of this peer id and open new sources to trigger the new
+   * replication state changes or new replication config changes. Here we don't need to change
+   * replication queue storage and only to enqueue all logs to the new replication source
+   * @param peerId the id of the replication peer
+   * @throws IOException
    */
-  public void deleteSource(String peerId, boolean closeConnection) {
-    abortWhenFail(() -> this.queueStorage.removeQueue(server.getServerName(), peerId));
-    if (closeConnection) {
-      this.replicationPeers.removePeer(peerId);
+  public void refreshSources(String peerId) throws IOException {
+    String terminateMessage = "Peer " + peerId +
+      " state or config changed. Will close the previous replication source and open a new one";
+    ReplicationPeer peer = replicationPeers.getPeer(peerId);
+    ReplicationSourceInterface src = createSource(peerId, peer);
+    // synchronized on latestPaths to avoid missing the new log
+    synchronized (this.latestPaths) {
+      ReplicationSourceInterface toRemove = this.sources.put(peerId, src);
+      if (toRemove != null) {
+        LOG.info("Terminate replication source for " + toRemove.getPeerId());
+        toRemove.terminate(terminateMessage);
+      }
+      for (SortedSet<String> walsByGroup : walsById.get(peerId).values()) {
+        walsByGroup.forEach(wal -> src.enqueueLog(new Path(this.logDir, wal)));
+      }
     }
-  }
+    LOG.info("Startup replication source for " + src.getPeerId());
+    src.startup();
 
-  /**
-   * Terminate the replication on this region server
-   */
-  public void join() {
-    this.executor.shutdown();
-    for (ReplicationSourceInterface source : this.sources) {
-      source.terminate("Region server is closing");
+    List<ReplicationSourceInterface> toStartup = new ArrayList<>();
+    // synchronized on oldsources to avoid race with NodeFailoverWorker
+    synchronized (this.oldsources) {
+      List<String> previousQueueIds = new ArrayList<>();
+      for (ReplicationSourceInterface oldSource : this.oldsources) {
+        if (oldSource.getPeerId().equals(peerId)) {
+          previousQueueIds.add(oldSource.getQueueId());
+          oldSource.terminate(terminateMessage);
+          this.oldsources.remove(oldSource);
+        }
+      }
+      for (String queueId : previousQueueIds) {
+        ReplicationSourceInterface replicationSource = createSource(queueId, peer);
+        this.oldsources.add(replicationSource);
+        for (SortedSet<String> walsByGroup : walsByIdRecoveredQueues.get(queueId).values()) {
+          walsByGroup.forEach(wal -> src.enqueueLog(new Path(wal)));
+        }
+        toStartup.add(replicationSource);
+      }
+    }
+    for (ReplicationSourceInterface replicationSource : oldsources) {
+      replicationSource.startup();
     }
   }
 
   /**
-   * Get a copy of the wals of the first source on this rs
-   * @return a sorted set of wal names
+   * Clear the metrics and related replication queue of the specified old source
+   * @param src source to clear
    */
-  @VisibleForTesting
-  Map<String, Map<String, SortedSet<String>>> getWALs() {
-    return Collections.unmodifiableMap(walsById);
+  void removeRecoveredSource(ReplicationSourceInterface src) {
+    LOG.info("Done with the recovered queue " + src.getQueueId());
+    src.getSourceMetrics().clear();
+    this.oldsources.remove(src);
+    // Delete queue from storage and memory
+    deleteQueue(src.getQueueId());
+    this.walsByIdRecoveredQueues.remove(src.getQueueId());
   }
 
   /**
-   * Get a copy of the wals of the recovered sources on this rs
-   * @return a sorted set of wal names
+   * Clear the metrics and related replication queue of the specified old source
+   * @param src source to clear
    */
-  @VisibleForTesting
-  Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
-    return Collections.unmodifiableMap(walsByIdRecoveredQueues);
+  void removeSource(ReplicationSourceInterface src) {
+    LOG.info("Done with the queue " + src.getQueueId());
+    src.getSourceMetrics().clear();
+    this.sources.remove(src.getPeerId());
+    // Delete queue from storage and memory
+    deleteQueue(src.getQueueId());
+    this.walsById.remove(src.getQueueId());
   }
 
   /**
-   * Get a list of all the normal sources of this rs
-   * @return lis of all sources
+   * Delete a complete queue of wals associated with a replication source
+   * @param queueId the id of replication queue to delete
    */
-  public List<ReplicationSourceInterface> getSources() {
-    return this.sources;
+  private void deleteQueue(String queueId) {
+    abortWhenFail(() -> this.queueStorage.removeQueue(server.getServerName(), queueId));
+  }
+
+  @FunctionalInterface
+  private interface ReplicationQueueOperation {
+    void exec() throws ReplicationException;
+  }
+
+  private void abortWhenFail(ReplicationQueueOperation op) {
+    try {
+      op.exec();
+    } catch (ReplicationException e) {
+      server.abort("Failed to operate on replication queue", e);
+    }
+  }
+
+  private void throwIOExceptionWhenFail(ReplicationQueueOperation op) throws IOException {
+    try {
+      op.exec();
+    } catch (ReplicationException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private void abortAndThrowIOExceptionWhenFail(ReplicationQueueOperation op) throws IOException {
+    try {
+      op.exec();
+    } catch (ReplicationException e) {
+      server.abort("Failed to operate on replication queue", e);
+      throw new IOException(e);
+    }
   }
 
   /**
-   * Get a list of all the old sources of this rs
-   * @return list of all old sources
+   * This method will log the current position to storage. And also clean old logs from the
+   * replication queue.
+   * @param log Path to the log currently being replicated
+   * @param queueId id of the replication queue
+   * @param position current location in the log
+   * @param queueRecovered indicates if this queue comes from another region server
    */
-  public List<ReplicationSourceInterface> getOldSources() {
-    return this.oldsources;
+  public void logPositionAndCleanOldLogs(Path log, String queueId, long position,
+      boolean queueRecovered) {
+    String fileName = log.getName();
+    abortWhenFail(
+      () -> this.queueStorage.setWALPosition(server.getServerName(), queueId, fileName, position));
+    cleanOldLogs(fileName, queueId, queueRecovered);
   }
 
   /**
-   * Get the normal source for a given peer
-   * @param peerId
-   * @return the normal source for the give peer if it exists, otherwise null.
+   * Cleans a log file and all older logs from replication queue. Called when we are sure that a log
+   * file is closed and has no more entries.
+   * @param log Path to the log
+   * @param queueId id of the replication queue
+   * @param queueRecovered Whether this is a recovered queue
    */
-  public ReplicationSourceInterface getSource(String peerId) {
-    return getSources().stream().filter(s -> s.getPeerId().equals(peerId)).findFirst().orElse(null);
+  @VisibleForTesting
+  void cleanOldLogs(String log, String queueId, boolean queueRecovered) {
+    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log);
+    if (queueRecovered) {
+      SortedSet<String> wals = walsByIdRecoveredQueues.get(queueId).get(logPrefix);
+      if (wals != null && !wals.first().equals(log)) {
+        cleanOldLogs(wals, log, queueId);
+      }
+    } else {
+      // synchronized on walsById to avoid race with preLogRoll
+      synchronized (this.walsById) {
+        SortedSet<String> wals = walsById.get(queueId).get(logPrefix);
+        if (wals != null && !wals.first().equals(log)) {
+          cleanOldLogs(wals, log, queueId);
+        }
+      }
+    }
   }
 
-  @VisibleForTesting
-  List<String> getAllQueues() throws ReplicationException {
-    return queueStorage.getAllQueues(server.getServerName());
+  private void cleanOldLogs(SortedSet<String> wals, String key, String id) {
+    SortedSet<String> walSet = wals.headSet(key);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
+    }
+    for (String wal : walSet) {
+      abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), id, wal));
+    }
+    walSet.clear();
   }
 
   void preLogRoll(Path newLog) throws IOException {
-    recordLog(newLog);
     String logName = newLog.getName();
     String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(logName);
-    synchronized (latestPaths) {
+    // synchronized on latestPaths to avoid the new open source miss the new log
+    synchronized (this.latestPaths) {
+      // Add log to queue storage
+      for (ReplicationSourceInterface source : this.sources.values()) {
+        // If record log to queue storage failed, abort RS and throw exception to make log roll
+        // failed
+        abortAndThrowIOExceptionWhenFail(
+          () -> this.queueStorage.addWAL(server.getServerName(), source.getQueueId(), logName));
+      }
+
+      // synchronized on walsById to avoid race with cleanOldLogs
+      synchronized (this.walsById) {
+        // Update walsById map
+        for (Map.Entry<String, Map<String, SortedSet<String>>> entry : this.walsById.entrySet()) {
+          String peerId = entry.getKey();
+          Map<String, SortedSet<String>> walsByPrefix = entry.getValue();
+          boolean existingPrefix = false;
+          for (Map.Entry<String, SortedSet<String>> walsEntry : walsByPrefix.entrySet()) {
+            SortedSet<String> wals = walsEntry.getValue();
+            if (this.sources.isEmpty()) {
+              // If there's no slaves, don't need to keep the old wals since
+              // we only consider the last one when a new slave comes in
+              wals.clear();
+            }
+            if (logPrefix.equals(walsEntry.getKey())) {
+              wals.add(logName);
+              existingPrefix = true;
+            }
+          }
+          if (!existingPrefix) {
+            // The new log belongs to a new group, add it into this peer
+            LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId);
+            SortedSet<String> wals = new TreeSet<>();
+            wals.add(logName);
+            walsByPrefix.put(logPrefix, wals);
+          }
+        }
+      }
+
+      // Add to latestPaths
       Iterator<Path> iterator = latestPaths.iterator();
       while (iterator.hasNext()) {
         Path path = iterator.next();
@@ -426,87 +588,21 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
   }
 
-  /**
-   * Check and enqueue the given log to the correct source. If there's still no source for the
-   * group to which the given log belongs, create one
-   * @param logPath the log path to check and enqueue
-   * @throws IOException
-   */
-  private void recordLog(Path logPath) throws IOException {
-    String logName = logPath.getName();
-    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(logName);
-    // update replication queues on ZK
-    // synchronize on replicationPeers to avoid adding source for the to-be-removed peer
-    synchronized (replicationPeers) {
-      for (String id : replicationPeers.getAllPeerIds()) {
-        try {
-          this.queueStorage.addWAL(server.getServerName(), id, logName);
-        } catch (ReplicationException e) {
-          throw new IOException("Cannot add log to replication queue"
-              + " when creating a new source, queueId=" + id + ", filename=" + logName, e);
-        }
-      }
-    }
-    // update walsById map
-    synchronized (walsById) {
-      for (Map.Entry<String, Map<String, SortedSet<String>>> entry : this.walsById.entrySet()) {
-        String peerId = entry.getKey();
-        Map<String, SortedSet<String>> walsByPrefix = entry.getValue();
-        boolean existingPrefix = false;
-        for (Map.Entry<String, SortedSet<String>> walsEntry : walsByPrefix.entrySet()) {
-          SortedSet<String> wals = walsEntry.getValue();
-          if (this.sources.isEmpty()) {
-            // If there's no slaves, don't need to keep the old wals since
-            // we only consider the last one when a new slave comes in
-            wals.clear();
-          }
-          if (logPrefix.equals(walsEntry.getKey())) {
-            wals.add(logName);
-            existingPrefix = true;
-          }
-        }
-        if (!existingPrefix) {
-          // The new log belongs to a new group, add it into this peer
-          LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId);
-          SortedSet<String> wals = new TreeSet<>();
-          wals.add(logName);
-          walsByPrefix.put(logPrefix, wals);
-        }
-      }
-    }
-  }
-
   void postLogRoll(Path newLog) throws IOException {
     // This only updates the sources we own, not the recovered ones
-    for (ReplicationSourceInterface source : this.sources) {
+    for (ReplicationSourceInterface source : this.sources.values()) {
       source.enqueueLog(newLog);
     }
   }
 
-  @VisibleForTesting
-  public AtomicLong getTotalBufferUsed() {
-    return totalBufferUsed;
-  }
-
-  /**
-   * Factory method to create a replication source
-   * @param peerId the id of the peer cluster
-   * @return the created source
-   */
-  private ReplicationSourceInterface getReplicationSource(String peerId,
-      ReplicationPeer replicationPeer) throws IOException {
-    ReplicationSourceInterface src = ReplicationSourceFactory.create(conf, peerId);
-
-    MetricsSource metrics = new MetricsSource(peerId);
-    // init replication source
-    src.init(conf, fs, this, queueStorage, replicationPeer, server, peerId, clusterId,
-      walFileLengthProvider, metrics);
-    return src;
+  @Override
+  public void regionServerRemoved(String regionserver) {
+    transferQueues(ServerName.valueOf(regionserver));
   }
 
   /**
    * Transfer all the queues of the specified to this region server. First it tries to grab a lock
-   * and if it works it will move the znodes and finally will delete the old znodes.
+   * and if it works it will move the old queues and finally will delete the old queues.
    * <p>
    * It creates one old source for any type of source of the old rs.
    */
@@ -524,102 +620,8 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
-   * Clear the references to the specified old source
-   * @param src source to clear
-   */
-  public void closeRecoveredQueue(ReplicationSourceInterface src) {
-    LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
-    if (src instanceof ReplicationSource) {
-      ((ReplicationSource) src).getSourceMetrics().clear();
-    }
-    this.oldsources.remove(src);
-    deleteSource(src.getPeerClusterZnode(), false);
-    this.walsByIdRecoveredQueues.remove(src.getPeerClusterZnode());
-  }
-
-  /**
-   * Clear the references to the specified old source
-   * @param src source to clear
-   */
-  public void closeQueue(ReplicationSourceInterface src) {
-    LOG.info("Done with the queue " + src.getPeerClusterZnode());
-    src.getSourceMetrics().clear();
-    this.sources.remove(src);
-    deleteSource(src.getPeerClusterZnode(), true);
-    this.walsById.remove(src.getPeerClusterZnode());
-  }
-
-  public void addPeer(String id) throws ReplicationException, IOException {
-    LOG.info("Trying to add peer, peerId: " + id);
-    boolean added = this.replicationPeers.addPeer(id);
-    if (added) {
-      LOG.info("Peer " + id + " connected success, trying to start the replication source thread.");
-      addSource(id);
-      if (replicationForBulkLoadDataEnabled) {
-        this.queueStorage.addPeerToHFileRefs(id);
-      }
-    }
-  }
-
-  /**
-   * Thie method first deletes all the recovered sources for the specified
-   * id, then deletes the normal source (deleting all related data in ZK).
-   * @param id The id of the peer cluster
-   */
-  public void removePeer(String id) {
-    LOG.info("Closing the following queue " + id + ", currently have "
-        + sources.size() + " and another "
-        + oldsources.size() + " that were recovered");
-    String terminateMessage = "Replication stream was removed by a user";
-    List<ReplicationSourceInterface> oldSourcesToDelete = new ArrayList<>();
-    // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
-    // see NodeFailoverWorker.run
-    synchronized (oldsources) {
-      // First close all the recovered sources for this peer
-      for (ReplicationSourceInterface src : oldsources) {
-        if (id.equals(src.getPeerId())) {
-          oldSourcesToDelete.add(src);
-        }
-      }
-      for (ReplicationSourceInterface src : oldSourcesToDelete) {
-        src.terminate(terminateMessage);
-        closeRecoveredQueue(src);
-      }
-    }
-    LOG.info("Number of deleted recovered sources for " + id + ": "
-        + oldSourcesToDelete.size());
-    // Now look for the one on this cluster
-    List<ReplicationSourceInterface> srcToRemove = new ArrayList<>();
-    // synchronize on replicationPeers to avoid adding source for the to-be-removed peer
-    synchronized (this.replicationPeers) {
-      for (ReplicationSourceInterface src : this.sources) {
-        if (id.equals(src.getPeerId())) {
-          srcToRemove.add(src);
-        }
-      }
-      if (srcToRemove.isEmpty()) {
-        LOG.error("The peer we wanted to remove is missing a ReplicationSourceInterface. " +
-            "This could mean that ReplicationSourceInterface initialization failed for this peer " +
-            "and that replication on this peer may not be caught up. peerId=" + id);
-      }
-      for (ReplicationSourceInterface toRemove : srcToRemove) {
-        toRemove.terminate(terminateMessage);
-        closeQueue(toRemove);
-      }
-      deleteSource(id, true);
-    }
-    // Remove HFile Refs znode from zookeeper
-    abortWhenFail(() -> this.queueStorage.removePeerFromHFileRefs(id));
-  }
-
-  @Override
-  public void regionServerRemoved(String regionserver) {
-    transferQueues(ServerName.valueOf(regionserver));
-  }
-
-  /**
-   * Class responsible to setup new ReplicationSources to take care of the
-   * queues from dead region servers.
+   * Class responsible to setup new ReplicationSources to take care of the queues from dead region
+   * servers.
    */
   class NodeFailoverWorker extends Thread {
 
@@ -649,10 +651,10 @@ public class ReplicationSourceManager implements ReplicationListener {
       }
       Map<String, Set<String>> newQueues = new HashMap<>();
       try {
-        List<String> peers = queueStorage.getAllQueues(deadRS);
-        while (!peers.isEmpty()) {
+        List<String> queues = queueStorage.getAllQueues(deadRS);
+        while (!queues.isEmpty()) {
           Pair<String, SortedSet<String>> peer = queueStorage.claimQueue(deadRS,
-            peers.get(ThreadLocalRandom.current().nextInt(peers.size())), server.getServerName());
+            queues.get(ThreadLocalRandom.current().nextInt(queues.size())), server.getServerName());
           long sleep = sleepBeforeFailover / 2;
           if (!peer.getSecond().isEmpty()) {
             newQueues.put(peer.getFirst(), peer.getSecond());
@@ -664,9 +666,9 @@ public class ReplicationSourceManager implements ReplicationListener {
             LOG.warn("Interrupted while waiting before transferring a queue.");
             Thread.currentThread().interrupt();
           }
-          peers = queueStorage.getAllQueues(deadRS);
+          queues = queueStorage.getAllQueues(deadRS);
         }
-        if (!peers.isEmpty()) {
+        if (queues.isEmpty()) {
           queueStorage.removeReplicatorIfQueueIsEmpty(deadRS);
         }
       } catch (ReplicationException e) {
@@ -681,23 +683,23 @@ public class ReplicationSourceManager implements ReplicationListener {
       }
 
       for (Map.Entry<String, Set<String>> entry : newQueues.entrySet()) {
-        String peerId = entry.getKey();
+        String queueId = entry.getKey();
         Set<String> walsSet = entry.getValue();
         try {
           // there is not an actual peer defined corresponding to peerId for the failover.
-          ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
+          ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queueId);
           String actualPeerId = replicationQueueInfo.getPeerId();
 
           ReplicationPeer peer = replicationPeers.getPeer(actualPeerId);
           if (peer == null) {
-            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS
-                + ", peer is null");
-            abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
+            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS +
+              ", peer is null");
+            abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), queueId));
             continue;
           }
           // track sources in walsByIdRecoveredQueues
           Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
-          walsByIdRecoveredQueues.put(peerId, walsByGroup);
+          walsByIdRecoveredQueues.put(queueId, walsByGroup);
           for (String wal : walsSet) {
             String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal);
             SortedSet<String> wals = walsByGroup.get(walPrefix);
@@ -708,14 +710,12 @@ public class ReplicationSourceManager implements ReplicationListener {
             wals.add(wal);
           }
 
-          // enqueue sources
-          ReplicationSourceInterface src = getReplicationSource(peerId, peer);
+          ReplicationSourceInterface src = createSource(queueId, peer);
           // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
-          // see removePeer
           synchronized (oldsources) {
             if (!replicationPeers.getAllPeerIds().contains(src.getPeerId())) {
               src.terminate("Recovered queue doesn't belong to any current peer");
-              closeRecoveredQueue(src);
+              removeRecoveredSource(src);
               continue;
             }
             oldsources.add(src);
@@ -733,6 +733,82 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
+   * Terminate the replication on this region server
+   */
+  public void join() {
+    this.executor.shutdown();
+    for (ReplicationSourceInterface source : this.sources.values()) {
+      source.terminate("Region server is closing");
+    }
+  }
+
+  /**
+   * Get a copy of the wals of the normal sources on this rs
+   * @return a sorted set of wal names
+   */
+  @VisibleForTesting
+  Map<String, Map<String, SortedSet<String>>> getWALs() {
+    return Collections.unmodifiableMap(walsById);
+  }
+
+  /**
+   * Get a copy of the wals of the recovered sources on this rs
+   * @return a sorted set of wal names
+   */
+  @VisibleForTesting
+  Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
+    return Collections.unmodifiableMap(walsByIdRecoveredQueues);
+  }
+
+  /**
+   * Get a list of all the normal sources of this rs
+   * @return list of all normal sources
+   */
+  public List<ReplicationSourceInterface> getSources() {
+    return new ArrayList<>(this.sources.values());
+  }
+
+  /**
+   * Get a list of all the recovered sources of this rs
+   * @return list of all recovered sources
+   */
+  public List<ReplicationSourceInterface> getOldSources() {
+    return this.oldsources;
+  }
+
+  /**
+   * Get the normal source for a given peer
+   * @return the normal source for the give peer if it exists, otherwise null.
+   */
+  @VisibleForTesting
+  public ReplicationSourceInterface getSource(String peerId) {
+    return this.sources.get(peerId);
+  }
+
+  @VisibleForTesting
+  List<String> getAllQueues() throws IOException {
+    List<String> allQueues = Collections.emptyList();
+    try {
+      allQueues = queueStorage.getAllQueues(server.getServerName());
+    } catch (ReplicationException e) {
+      throw new IOException(e);
+    }
+    return allQueues;
+  }
+
+  @VisibleForTesting
+  int getSizeOfLatestPath() {
+    synchronized (latestPaths) {
+      return latestPaths.size();
+    }
+  }
+
+  @VisibleForTesting
+  public AtomicLong getTotalBufferUsed() {
+    return totalBufferUsed;
+  }
+
+  /**
    * Get the directory where wals are archived
    * @return the directory where wals are archived
    */
@@ -764,28 +840,30 @@ public class ReplicationSourceManager implements ReplicationListener {
    * Get the ReplicationPeers used by this ReplicationSourceManager
    * @return the ReplicationPeers used by this ReplicationSourceManager
    */
-  public ReplicationPeers getReplicationPeers() {return this.replicationPeers;}
+  public ReplicationPeers getReplicationPeers() {
+    return this.replicationPeers;
+  }
 
   /**
    * Get a string representation of all the sources' metrics
    */
   public String getStats() {
     StringBuilder stats = new StringBuilder();
-    for (ReplicationSourceInterface source : sources) {
+    for (ReplicationSourceInterface source : this.sources.values()) {
       stats.append("Normal source for cluster " + source.getPeerId() + ": ");
       stats.append(source.getStats() + "\n");
     }
     for (ReplicationSourceInterface oldSource : oldsources) {
-      stats.append("Recovered source for cluster/machine(s) " + oldSource.getPeerId()+": ");
-      stats.append(oldSource.getStats()+ "\n");
+      stats.append("Recovered source for cluster/machine(s) " + oldSource.getPeerId() + ": ");
+      stats.append(oldSource.getStats() + "\n");
     }
     return stats.toString();
   }
 
   public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
-      throws ReplicationException {
-    for (ReplicationSourceInterface source : this.sources) {
-      source.addHFileRefs(tableName, family, pairs);
+      throws IOException {
+    for (ReplicationSourceInterface source : this.sources.values()) {
+      throwIOExceptionWhenFail(() -> source.addHFileRefs(tableName, family, pairs));
     }
   }
 
@@ -798,11 +876,10 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
-   * Whether an entry can be pushed to the peer or not right now.
-   * If we enable serial replication, we can not push the entry until all entries in its region
-   * whose sequence numbers are smaller than this entry have been pushed.
-   * For each ReplicationSource, we need only check the first entry in each region, as long as it
-   * can be pushed, we can push all in this ReplicationSource.
+   * Whether an entry can be pushed to the peer or not right now. If we enable serial replication,
+   * we can not push the entry until all entries in its region whose sequence numbers are smaller
+   * than this entry have been pushed. For each ReplicationSource, we need only check the first
+   * entry in each region, as long as it can be pushed, we can push all in this ReplicationSource.
    * This method will be blocked until we can push.
    * @return the first barrier of entry's region, or -1 if there is no barrier. It is used to
    *         prevent saving positions in the region of no barrier.
@@ -813,22 +890,18 @@ public class ReplicationSourceManager implements ReplicationListener {
     /**
      * There are barriers for this region and position for this peer. N barriers form N intervals,
      * (b1,b2) (b2,b3) ... (bn,max). Generally, there is no logs whose seq id is not greater than
-     * the first barrier and the last interval is start from the last barrier.
-     *
-     * There are several conditions that we can push now, otherwise we should block:
-     * 1) "Serial replication" is not enabled, we can push all logs just like before. This case
-     *    should not call this method.
-     * 2) There is no barriers for this region, or the seq id is smaller than the first barrier.
-     *    It is mainly because we alter REPLICATION_SCOPE = 2. We can not guarantee the
-     *    order of logs that is written before altering.
-     * 3) This entry is in the first interval of barriers. We can push them because it is the
-     *    start of a region. But if the region is created by region split, we should check
-     *    if the parent regions are fully pushed.
-     * 4) If the entry's seq id and the position are in same section, or the pos is the last
-     *    number of previous section. Because when open a region we put a barrier the number
-     *    is the last log's id + 1.
-     * 5) Log's seq is smaller than pos in meta, we are retrying. It may happen when a RS crashes
-     *    after save replication meta and before save zk offset.
+     * the first barrier and the last interval is start from the last barrier. There are several
+     * conditions that we can push now, otherwise we should block: 1) "Serial replication" is not
+     * enabled, we can push all logs just like before. This case should not call this method. 2)
+     * There is no barriers for this region, or the seq id is smaller than the first barrier. It is
+     * mainly because we alter REPLICATION_SCOPE = 2. We can not guarantee the order of logs that is
+     * written before altering. 3) This entry is in the first interval of barriers. We can push them
+     * because it is the start of a region. But if the region is created by region split, we should
+     * check if the parent regions are fully pushed. 4) If the entry's seq id and the position are
+     * in same section, or the pos is the last number of previous section. Because when open a
+     * region we put a barrier the number is the last log's id + 1. 5) Log's seq is smaller than pos
+     * in meta, we are retrying. It may happen when a RS crashes after save replication meta and
+     * before save zk offset.
      */
     List<Long> barriers = MetaTableAccessor.getReplicationBarriers(connection, encodedName);
     if (barriers.isEmpty() || seq <= barriers.get(0)) {
@@ -842,8 +915,8 @@ public class ReplicationSourceManager implements ReplicationListener {
     if (interval == 1) {
       // Case 3
       // Check if there are parent regions
-      String parentValue = MetaTableAccessor.getSerialReplicationParentRegion(connection,
-          encodedName);
+      String parentValue =
+          MetaTableAccessor.getSerialReplicationParentRegion(connection, encodedName);
       if (parentValue == null) {
         // This region has no parent or the parent's log entries are fully pushed.
         return;
@@ -855,16 +928,17 @@ public class ReplicationSourceManager implements ReplicationListener {
           byte[] region = Bytes.toBytes(parent);
           long pos = MetaTableAccessor.getReplicationPositionForOnePeer(connection, region, peerId);
           List<Long> parentBarriers = MetaTableAccessor.getReplicationBarriers(connection, region);
-          if (parentBarriers.size() > 0
-              && parentBarriers.get(parentBarriers.size() - 1) - 1 > pos) {
+          if (parentBarriers.size() > 0 &&
+            parentBarriers.get(parentBarriers.size() - 1) - 1 > pos) {
             allParentDone = false;
             // For a closed region, we will write a close event marker to WAL whose sequence id is
             // larger than final barrier but still smaller than next region's openSeqNum.
             // So if the pos is larger than last barrier, we can say we have read the event marker
             // which means the parent region has been fully pushed.
-            LOG.info(Bytes.toString(encodedName) + " can not start pushing because parent region's"
-                + " log has not been fully pushed: parent=" + Bytes.toString(region) + " pos=" + pos
-                + " barriers=" + Arrays.toString(barriers.toArray()));
+            LOG.info(
+              Bytes.toString(encodedName) + " can not start pushing because parent region's" +
+                " log has not been fully pushed: parent=" + Bytes.toString(region) + " pos=" + pos +
+                " barriers=" + Arrays.toString(barriers.toArray()));
             break;
           }
         }
@@ -878,7 +952,8 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
 
     while (true) {
-      long pos = MetaTableAccessor.getReplicationPositionForOnePeer(connection, encodedName, peerId);
+      long pos =
+          MetaTableAccessor.getReplicationPositionForOnePeer(connection, encodedName, peerId);
       if (seq <= pos) {
         // Case 5
       }
@@ -893,9 +968,9 @@ public class ReplicationSourceManager implements ReplicationListener {
         }
       }
 
-      LOG.info(Bytes.toString(encodedName) + " can not start pushing to peer " + peerId
-          + " because previous log has not been pushed: sequence=" + seq + " pos=" + pos
-          + " barriers=" + Arrays.toString(barriers.toArray()));
+      LOG.info(Bytes.toString(encodedName) + " can not start pushing to peer " + peerId +
+        " because previous log has not been pushed: sequence=" + seq + " pos=" + pos +
+        " barriers=" + Arrays.toString(barriers.toArray()));
       Thread.sleep(replicationWaitTime);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
index ea98cda..808f738 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
@@ -277,8 +277,8 @@ public class ReplicationSourceShipper extends Thread {
   }
 
   protected void updateLogPosition(long lastReadPosition) {
-    source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getPeerClusterZnode(),
-      lastReadPosition, false, false);
+    source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getQueueId(),
+      lastReadPosition, false);
     lastLoggedPosition = lastReadPosition;
   }
 
@@ -295,7 +295,7 @@ public class ReplicationSourceShipper extends Thread {
   public void startup(UncaughtExceptionHandler handler) {
     String name = Thread.currentThread().getName();
     Threads.setDaemonThreadRunning(this, name + ".replicationSource." + walGroupId + ","
-        + source.getPeerClusterZnode(), handler);
+        + source.getQueueId(), handler);
   }
 
   public PriorityBlockingQueue<Path> getLogQueue() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index 4643a22..90a421d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -115,7 +115,7 @@ public class ReplicationSourceWALReader extends Thread {
         this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per
     this.eofAutoRecovery = conf.getBoolean("replication.source.eof.autorecovery", false);
     this.entryBatchQueue = new LinkedBlockingQueue<>(batchCount);
-    LOG.info("peerClusterZnode=" + source.getPeerClusterZnode()
+    LOG.info("peerClusterZnode=" + source.getQueueId()
         + ", ReplicationSourceWALReaderThread : " + source.getPeerId()
         + " inited, replicationBatchSizeCapacity=" + replicationBatchSizeCapacity
         + ", replicationBatchCountCapacity=" + replicationBatchCountCapacity

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index 38ec598..ff20ddc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -89,7 +89,7 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   }
 
   @Override
-  public String getPeerClusterZnode() {
+  public String getQueueId() {
     return peerClusterId;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java
index ed71123..40a955e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java
@@ -30,12 +30,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -43,6 +42,8 @@ import org.apache.hadoop.hbase.client.Get;
 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.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
@@ -68,9 +69,6 @@ public class TestNamespaceReplication extends TestReplicationBase {
 
   private static final byte[] val = Bytes.toBytes("myval");
 
-  private static HTableDescriptor tabA;
-  private static HTableDescriptor tabB;
-
   private static Connection connection1;
   private static Connection connection2;
   private static Admin admin1;
@@ -90,23 +88,21 @@ public class TestNamespaceReplication extends TestReplicationBase {
     admin2.createNamespace(NamespaceDescriptor.create(ns1).build());
     admin2.createNamespace(NamespaceDescriptor.create(ns2).build());
 
-    tabA = new HTableDescriptor(tabAName);
-    HColumnDescriptor fam = new HColumnDescriptor(f1Name);
-    fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
-    tabA.addFamily(fam);
-    fam = new HColumnDescriptor(f2Name);
-    fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
-    tabA.addFamily(fam);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tabAName);
+    builder.addColumnFamily(ColumnFamilyDescriptorBuilder
+      .newBuilder(f1Name).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build());
+    builder.addColumnFamily(ColumnFamilyDescriptorBuilder
+      .newBuilder(f2Name).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build());
+    TableDescriptor tabA = builder.build();
     admin1.createTable(tabA);
     admin2.createTable(tabA);
 
-    tabB = new HTableDescriptor(tabBName);
-    fam = new HColumnDescriptor(f1Name);
-    fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
-    tabB.addFamily(fam);
-    fam = new HColumnDescriptor(f2Name);
-    fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
-    tabB.addFamily(fam);
+    builder = TableDescriptorBuilder.newBuilder(tabBName);
+    builder.addColumnFamily(ColumnFamilyDescriptorBuilder
+      .newBuilder(f1Name).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build());
+    builder.addColumnFamily(ColumnFamilyDescriptorBuilder
+      .newBuilder(f2Name).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build());
+    TableDescriptor tabB = builder.build();
     admin1.createTable(tabB);
     admin2.createTable(tabB);
   }
@@ -134,22 +130,24 @@ public class TestNamespaceReplication extends TestReplicationBase {
 
   @Test
   public void testNamespaceReplication() throws Exception {
+    String peerId = "2";
+
     Table htab1A = connection1.getTable(tabAName);
     Table htab2A = connection2.getTable(tabAName);
 
     Table htab1B = connection1.getTable(tabBName);
     Table htab2B = connection2.getTable(tabBName);
 
-    ReplicationPeerConfig rpc = admin.getPeerConfig("2");
-    rpc.setReplicateAllUserTables(false);
-    admin.updatePeerConfig("2", rpc);
+    ReplicationPeerConfig rpc = admin1.getReplicationPeerConfig(peerId);
+    admin1.updateReplicationPeerConfig(peerId,
+      ReplicationPeerConfig.newBuilder(rpc).setReplicateAllUserTables(false).build());
 
     // add ns1 to peer config which replicate to cluster2
-    rpc = admin.getPeerConfig("2");
+    rpc = admin1.getReplicationPeerConfig(peerId);
     Set<String> namespaces = new HashSet<>();
     namespaces.add(ns1);
-    rpc.setNamespaces(namespaces);
-    admin.updatePeerConfig("2", rpc);
+    admin1.updateReplicationPeerConfig(peerId,
+      ReplicationPeerConfig.newBuilder(rpc).setNamespaces(namespaces).build());
     LOG.info("update peer config");
 
     // Table A can be replicated to cluster2
@@ -163,15 +161,14 @@ public class TestNamespaceReplication extends TestReplicationBase {
     ensureRowNotExisted(htab2B, row, f1Name, f2Name);
 
     // add ns1:TA => 'f1' and ns2 to peer config which replicate to cluster2
-    rpc = admin.getPeerConfig("2");
+    rpc = admin1.getReplicationPeerConfig(peerId);
     namespaces = new HashSet<>();
     namespaces.add(ns2);
-    rpc.setNamespaces(namespaces);
     Map<TableName, List<String>> tableCfs = new HashMap<>();
     tableCfs.put(tabAName, new ArrayList<>());
     tableCfs.get(tabAName).add("f1");
-    rpc.setTableCFsMap(tableCfs);
-    admin.updatePeerConfig("2", rpc);
+    admin1.updateReplicationPeerConfig(peerId, ReplicationPeerConfig.newBuilder(rpc)
+        .setNamespaces(namespaces).setTableCFsMap(tableCfs).build());
     LOG.info("update peer config");
 
     // Only family f1 of Table A can replicated to cluster2
@@ -186,7 +183,7 @@ public class TestNamespaceReplication extends TestReplicationBase {
     delete(htab1B, row, f1Name, f2Name);
     ensureRowNotExisted(htab2B, row, f1Name, f2Name);
 
-    admin.removePeer("2");
+    admin1.removeReplicationPeer(peerId);
   }
 
   private void put(Table source, byte[] row, byte[]... families)

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/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 1001aa5..33216cb 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
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -95,12 +94,12 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
-
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
+
 /**
  * An abstract class that tests ReplicationSourceManager. Classes that extend this class should
  * set up the proper config for this class and initialize the proper cluster using
@@ -313,7 +312,7 @@ public abstract class TestReplicationSourceManager {
     wal.rollWriter();
 
     manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
-        "1", 0, false, false);
+        "1", 0, false);
 
     wal.append(hri,
         new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f0bfe01/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
index c6d9eef..490c4b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
-
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;