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 2017/12/26 13:11:42 UTC

[01/17] hbase git commit: HBASE-19618 Remove replicationQueuesClient.class/replicationQueues.class config and remove table based ReplicationQueuesClient/ReplicationQueues implementation [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19397 092291e33 -> aa3bc1d9b (forced update)


HBASE-19618 Remove replicationQueuesClient.class/replicationQueues.class config and remove table based ReplicationQueuesClient/ReplicationQueues implementation


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

Branch: refs/heads/HBASE-19397
Commit: 2ce5dc892710666c9a382fdeece412ecbb8559bb
Parents: 38472e1
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Dec 25 11:44:18 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Tue Dec 26 14:39:41 2017 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |  17 +-
 .../TableBasedReplicationQueuesClientImpl.java  | 113 -----
 .../TableBasedReplicationQueuesImpl.java        | 448 -----------------
 .../org/apache/hadoop/hbase/master/HMaster.java |  17 +-
 .../replication/TestMultiSlaveReplication.java  |   2 -
 .../TestReplicationStateHBaseImpl.java          | 495 -------------------
 .../replication/TestReplicationTableBase.java   | 109 ----
 ...tTableBasedReplicationSourceManagerImpl.java |  63 ---
 8 files changed, 12 insertions(+), 1252 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2ce5dc89/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 3ff6914..9f4ad18 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
@@ -31,21 +31,16 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static final Class defaultReplicationQueueClass = ReplicationQueuesZKImpl.class;
-
   public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args)
       throws Exception {
-    Class<?> classToBuild = args.getConf().getClass("hbase.region.replica." +
-        "replication.replicationQueues.class", defaultReplicationQueueClass);
-    return (ReplicationQueues) ConstructorUtils.invokeConstructor(classToBuild, args);
+    return (ReplicationQueues) ConstructorUtils.invokeConstructor(ReplicationQueuesZKImpl.class,
+      args);
   }
 
-  public static ReplicationQueuesClient getReplicationQueuesClient(
-      ReplicationQueuesClientArguments args) throws Exception {
-    Class<?> classToBuild = args.getConf().getClass(
-      "hbase.region.replica.replication.replicationQueuesClient.class",
-      ReplicationQueuesClientZKImpl.class);
-    return (ReplicationQueuesClient) ConstructorUtils.invokeConstructor(classToBuild, 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,

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ce5dc89/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesClientImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesClientImpl.java
deleted file mode 100644
index 0a8ed31..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesClientImpl.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import org.apache.commons.lang3.NotImplementedException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-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.zookeeper.KeeperException;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-/**
- * Implements the ReplicationQueuesClient interface on top of the Replication Table. It utilizes
- * the ReplicationTableBase to access the Replication Table.
- */
-@InterfaceAudience.Private
-public class TableBasedReplicationQueuesClientImpl extends ReplicationTableBase
-  implements ReplicationQueuesClient {
-
-  public TableBasedReplicationQueuesClientImpl(ReplicationQueuesClientArguments args)
-    throws IOException {
-    super(args.getConf(), args.getAbortable());
-  }
-  public TableBasedReplicationQueuesClientImpl(Configuration conf,
-                                               Abortable abortable) throws IOException {
-    super(conf, abortable);
-  }
-
-  @Override
-  public void init() throws ReplicationException{
-    // no-op
-  }
-
-  @Override
-  public List<String> getListOfReplicators() {
-    return super.getListOfReplicators();
-  }
-
-  @Override
-  public List<String> getLogsInQueue(String serverName, String queueId) {
-    return super.getLogsInQueue(serverName, queueId);
-  }
-
-  @Override
-  public List<String> getAllQueues(String serverName) {
-    return super.getAllQueues(serverName);
-  }
-
-  @Override
-  public Set<String> getAllWALs() {
-    Set<String> allWals = new HashSet<>();
-    ResultScanner allQueues = null;
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      allQueues = replicationTable.getScanner(new Scan());
-      for (Result queue : allQueues) {
-        for (String wal : readWALsFromResult(queue)) {
-          allWals.add(wal);
-        }
-      }
-    } catch (IOException e) {
-      String errMsg = "Failed getting all WAL's in Replication Table";
-      abortable.abort(errMsg, e);
-    } finally {
-      if (allQueues != null) {
-        allQueues.close();
-      }
-    }
-    return allWals;
-  }
-
-  @Override
-  public int getHFileRefsNodeChangeVersion() throws KeeperException {
-    // TODO
-    throw new NotImplementedException(HConstants.NOT_IMPLEMENTED);
-  }
-
-  @Override
-  public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
-    // TODO
-    throw new NotImplementedException(HConstants.NOT_IMPLEMENTED);
-  }
-
-  @Override
-  public List<String> getReplicableHFiles(String peerId) throws KeeperException {
-    // TODO
-    throw new NotImplementedException(HConstants.NOT_IMPLEMENTED);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ce5dc89/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
deleted file mode 100644
index b6c849c..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
+++ /dev/null
@@ -1,448 +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.commons.lang3.NotImplementedException;
-import org.apache.hadoop.conf.Configuration;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.CompareOperator;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hadoop.hbase.client.Delete;
-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.ResultScanner;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-/**
- * This class provides an implementation of the ReplicationQueues interface using an HBase table
- * "Replication Table". It utilizes the ReplicationTableBase to access the Replication Table.
- */
-@InterfaceAudience.Private
-public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
-  implements ReplicationQueues {
-
-  private static final Logger LOG = LoggerFactory.getLogger(TableBasedReplicationQueuesImpl.class);
-
-  // Common byte values used in replication offset tracking
-  private static final byte[] INITIAL_OFFSET_BYTES = Bytes.toBytes(0L);
-  private static final byte[] EMPTY_STRING_BYTES = Bytes.toBytes("");
-
-  private String serverName = null;
-  private byte[] serverNameBytes = null;
-
-  // TODO: Only use this variable temporarily. Eventually we want to use HBase to store all
-  // TODO: replication information
-  private ReplicationStateZKBase replicationState;
-
-  public TableBasedReplicationQueuesImpl(ReplicationQueuesArguments args) throws IOException {
-    this(args.getConf(), args.getAbortable(), args.getZk());
-  }
-
-  public TableBasedReplicationQueuesImpl(Configuration conf, Abortable abort, ZKWatcher zkw)
-    throws IOException {
-    super(conf, abort);
-    replicationState = new ReplicationStateZKBase(zkw, conf, abort) {};
-  }
-
-  @Override
-  public void init(String serverName) throws ReplicationException {
-    this.serverName = serverName;
-    this.serverNameBytes = Bytes.toBytes(serverName);
-  }
-
-  @Override
-  public List<String> getListOfReplicators() {
-    return super.getListOfReplicators();
-  }
-
-  @Override
-  public void removeQueue(String queueId) {
-    try {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      if (checkQueueExists(queueId)) {
-        Delete deleteQueue = new Delete(rowKey);
-        safeQueueUpdate(deleteQueue);
-      } else {
-        LOG.info("No logs were registered for queue id=" + queueId + " so no rows were removed " +
-            "from the replication table while removing the queue");
-      }
-    } catch (IOException | ReplicationException e) {
-      String errMsg = "Failed removing queue queueId=" + queueId;
-      abortable.abort(errMsg, e);
-    }
-  }
-
-  @Override
-  public void addLog(String queueId, String filename) throws ReplicationException {
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      if (!checkQueueExists(queueId)) {
-        // Each queue will have an Owner, OwnerHistory, and a collection of [WAL:offset] key values
-        Put putNewQueue = new Put(Bytes.toBytes(buildQueueRowKey(queueId)));
-        putNewQueue.addColumn(CF_QUEUE, COL_QUEUE_OWNER, serverNameBytes);
-        putNewQueue.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY, EMPTY_STRING_BYTES);
-        putNewQueue.addColumn(CF_QUEUE, Bytes.toBytes(filename), INITIAL_OFFSET_BYTES);
-        replicationTable.put(putNewQueue);
-      } else {
-        // Otherwise simply add the new log and offset as a new column
-        Put putNewLog = new Put(queueIdToRowKey(queueId));
-        putNewLog.addColumn(CF_QUEUE, Bytes.toBytes(filename), INITIAL_OFFSET_BYTES);
-        safeQueueUpdate(putNewLog);
-      }
-    } catch (IOException | ReplicationException e) {
-      String errMsg = "Failed adding log queueId=" + queueId + " filename=" + filename;
-      abortable.abort(errMsg, e);
-    }
-  }
-
-  @Override
-  public void removeLog(String queueId, String filename) {
-    try {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      Delete delete = new Delete(rowKey);
-      delete.addColumns(CF_QUEUE, Bytes.toBytes(filename));
-      safeQueueUpdate(delete);
-    } catch (IOException | ReplicationException e) {
-      String errMsg = "Failed removing log queueId=" + queueId + " filename=" + filename;
-      abortable.abort(errMsg, e);
-    }
-  }
-
-  @Override
-  public void setLogPosition(String queueId, String filename, long position) {
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      // Check that the log exists. addLog() must have been called before setLogPosition().
-      Get checkLogExists = new Get(rowKey);
-      checkLogExists.addColumn(CF_QUEUE, Bytes.toBytes(filename));
-      if (!replicationTable.exists(checkLogExists)) {
-        String errMsg = "Could not set position of non-existent log from queueId=" + queueId +
-          ", filename=" + filename;
-        abortable.abort(errMsg, new ReplicationException(errMsg));
-        return;
-      }
-      // Update the log offset if it exists
-      Put walAndOffset = new Put(rowKey);
-      walAndOffset.addColumn(CF_QUEUE, Bytes.toBytes(filename), Bytes.toBytes(position));
-      safeQueueUpdate(walAndOffset);
-    } catch (IOException | ReplicationException e) {
-      String errMsg = "Failed writing log position queueId=" + queueId + "filename=" +
-        filename + " position=" + position;
-      abortable.abort(errMsg, e);
-    }
-  }
-
-  @Override
-  public long getLogPosition(String queueId, String filename) throws ReplicationException {
-    try {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      Get getOffset = new Get(rowKey);
-      getOffset.addColumn(CF_QUEUE, Bytes.toBytes(filename));
-      Result result = getResultIfOwner(getOffset);
-      if (result == null || !result.containsColumn(CF_QUEUE, Bytes.toBytes(filename))) {
-        throw new ReplicationException("Could not read empty result while getting log position " +
-          "queueId=" + queueId + ", filename=" + filename);
-      }
-      return Bytes.toLong(result.getValue(CF_QUEUE, Bytes.toBytes(filename)));
-    } catch (IOException e) {
-      throw new ReplicationException("Could not get position in log for queueId=" + queueId +
-        ", filename=" + filename);
-    }
-  }
-
-  @Override
-  public void removeAllQueues() {
-    List<String> myQueueIds = getAllQueues();
-    for (String queueId : myQueueIds) {
-      removeQueue(queueId);
-    }
-  }
-
-  @Override
-  public List<String> getLogsInQueue(String queueId) {
-    String errMsg = "Failed getting logs in queue queueId=" + queueId;
-    byte[] rowKey = queueIdToRowKey(queueId);
-    List<String> logs = new ArrayList<>();
-    try {
-      Get getQueue = new Get(rowKey);
-      Result queue = getResultIfOwner(getQueue);
-      if (queue == null || queue.isEmpty()) {
-        String errMsgLostOwnership = "Failed getting logs for queue queueId=" +
-            Bytes.toString(rowKey) + " because the queue was missing or we lost ownership";
-        abortable.abort(errMsg, new ReplicationException(errMsgLostOwnership));
-        return null;
-      }
-      Map<byte[], byte[]> familyMap = queue.getFamilyMap(CF_QUEUE);
-      for(byte[] cQualifier : familyMap.keySet()) {
-        if (Arrays.equals(cQualifier, COL_QUEUE_OWNER) || Arrays.equals(cQualifier,
-            COL_QUEUE_OWNER_HISTORY)) {
-          continue;
-        }
-        logs.add(Bytes.toString(cQualifier));
-      }
-    } catch (IOException e) {
-      abortable.abort(errMsg, e);
-      return null;
-    }
-    return logs;
-  }
-
-  @Override
-  public List<String> getAllQueues() {
-    return getAllQueues(serverName);
-  }
-
-  @Override public List<String> getUnClaimedQueueIds(String regionserver) {
-    if (isThisOurRegionServer(regionserver)) {
-      return null;
-    }
-    try (ResultScanner queuesToClaim = getQueuesBelongingToServer(regionserver)) {
-      List<String> res = new ArrayList<>();
-      for (Result queue : queuesToClaim) {
-        String rowKey = Bytes.toString(queue.getRow());
-        res.add(rowKey);
-      }
-      return res.isEmpty() ? null : res;
-    } catch (IOException e) {
-      String errMsg = "Failed getUnClaimedQueueIds";
-      abortable.abort(errMsg, e);
-    }
-    return null;
-  }
-
-  @Override public void removeReplicatorIfQueueIsEmpty(String regionserver) {
-    // Do nothing here
-  }
-
-  @Override
-  public Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId) {
-    if (isThisOurRegionServer(regionserver)) {
-      return null;
-    }
-
-    try (ResultScanner queuesToClaim = getQueuesBelongingToServer(regionserver)){
-      for (Result queue : queuesToClaim) {
-        String rowKey = Bytes.toString(queue.getRow());
-        if (!rowKey.equals(queueId)){
-          continue;
-        }
-        if (attemptToClaimQueue(queue, regionserver)) {
-          ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(rowKey);
-          if (replicationState.peerExists(replicationQueueInfo.getPeerId())) {
-            SortedSet<String> sortedLogs = new TreeSet<>();
-            List<String> logs = getLogsInQueue(queue.getRow());
-            for (String log : logs) {
-              sortedLogs.add(log);
-            }
-            LOG.info(serverName + " has claimed queue " + rowKey + " from " + regionserver);
-            return new Pair<>(rowKey, sortedLogs);
-          } else {
-            // Delete orphaned queues
-            removeQueue(Bytes.toString(queue.getRow()));
-            LOG.info(serverName + " has deleted abandoned queue " + queueId + " from " +
-              regionserver);
-          }
-        }
-      }
-    } catch (IOException | KeeperException e) {
-      String errMsg = "Failed claiming queues for regionserver=" + regionserver;
-      abortable.abort(errMsg, e);
-    }
-    return null;
-  }
-
-  @Override
-  public boolean isThisOurRegionServer(String regionserver) {
-    return this.serverName.equals(regionserver);
-  }
-
-  @Override
-  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
-    // TODO
-    throw new NotImplementedException(HConstants.NOT_IMPLEMENTED);
-  }
-
-  @Override
-  public void removePeerFromHFileRefs(String peerId) {
-    // TODO
-    throw new NotImplementedException(HConstants.NOT_IMPLEMENTED);
-  }
-
-  @Override
-  public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
-      throws ReplicationException {
-    // TODO
-    throw new NotImplementedException(HConstants.NOT_IMPLEMENTED);
-  }
-
-  @Override
-  public void removeHFileRefs(String peerId, List<String> files) {
-    // TODO
-    throw new NotImplementedException(HConstants.NOT_IMPLEMENTED);
-  }
-
-  private String buildQueueRowKey(String queueId) {
-    return buildQueueRowKey(serverName, queueId);
-  }
-
-  /**
-   * Convenience method that gets the row key of the queue specified by queueId
-   * @param queueId queueId of a queue in this server
-   * @return the row key of the queue in the Replication Table
-   */
-  private byte[] queueIdToRowKey(String queueId) {
-    return queueIdToRowKey(serverName, queueId);
-  }
-
-  /**
-   * See safeQueueUpdate(RowMutations mutate)
-   *
-   * @param put Row mutation to perform on the queue
-   */
-  private void safeQueueUpdate(Put put) throws ReplicationException, IOException {
-    RowMutations mutations = new RowMutations(put.getRow());
-    mutations.add(put);
-    safeQueueUpdate(mutations);
-  }
-
-  /**
-   * See safeQueueUpdate(RowMutations mutate)
-   *
-   * @param delete Row mutation to perform on the queue
-   */
-  private void safeQueueUpdate(Delete delete) throws ReplicationException,
-    IOException{
-    RowMutations mutations = new RowMutations(delete.getRow());
-    mutations.add(delete);
-    safeQueueUpdate(mutations);
-  }
-
-  /**
-   * Attempt to mutate a given queue in the Replication Table with a checkAndPut on the OWNER column
-   * of the queue. Abort the server if this checkAndPut fails: which means we have somehow lost
-   * ownership of the column or an IO Exception has occurred during the transaction.
-   *
-   * @param mutate Mutation to perform on a given queue
-   */
-  private void safeQueueUpdate(RowMutations mutate) throws ReplicationException, IOException{
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      boolean updateSuccess = replicationTable.checkAndMutate(mutate.getRow(), CF_QUEUE)
-          .qualifier(COL_QUEUE_OWNER).ifEquals(serverNameBytes).thenMutate(mutate);
-      if (!updateSuccess) {
-        throw new ReplicationException("Failed to update Replication Table because we lost queue " +
-            " ownership");
-      }
-    }
-  }
-
-  /**
-   * Check if the queue specified by queueId is stored in HBase
-   *
-   * @param queueId Either raw or reclaimed format of the queueId
-   * @return Whether the queue is stored in HBase
-   * @throws IOException
-   */
-  private boolean checkQueueExists(String queueId) throws IOException {
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      return replicationTable.exists(new Get(rowKey));
-    }
-  }
-
-  /**
-   * Attempt to claim the given queue with a checkAndPut on the OWNER column. We check that the
-   * recently killed server is still the OWNER before we claim it.
-   *
-   * @param queue The queue that we are trying to claim
-   * @param originalServer The server that originally owned the queue
-   * @return Whether we successfully claimed the queue
-   * @throws IOException
-   */
-  private boolean attemptToClaimQueue (Result queue, String originalServer) throws IOException{
-    Put putQueueNameAndHistory = new Put(queue.getRow());
-    putQueueNameAndHistory.addColumn(CF_QUEUE, COL_QUEUE_OWNER, Bytes.toBytes(serverName));
-    String newOwnerHistory = buildClaimedQueueHistory(Bytes.toString(queue.getValue(CF_QUEUE,
-      COL_QUEUE_OWNER_HISTORY)), originalServer);
-    putQueueNameAndHistory.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY,
-        Bytes.toBytes(newOwnerHistory));
-    RowMutations claimAndRenameQueue = new RowMutations(queue.getRow());
-    claimAndRenameQueue.add(putQueueNameAndHistory);
-    // Attempt to claim ownership for this queue by checking if the current OWNER is the original
-    // server. If it is not then another RS has already claimed it. If it is we set ourselves as the
-    // new owner and update the queue's history
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      boolean success = replicationTable.checkAndMutate(queue.getRow(), CF_QUEUE)
-          .qualifier(COL_QUEUE_OWNER).ifEquals(Bytes.toBytes(originalServer))
-          .thenMutate(claimAndRenameQueue);
-      return success;
-    }
-  }
-
-  /**
-   * Attempts to run a Get on some queue. Will only return a non-null result if we currently own
-   * the queue.
-   *
-   * @param get The Get that we want to query
-   * @return The result of the Get if this server is the owner of the queue. Else it returns null.
-   * @throws IOException
-   */
-  private Result getResultIfOwner(Get get) throws IOException {
-    Scan scan = new Scan(get);
-    // Check if the Get currently contains all columns or only specific columns
-    if (scan.getFamilyMap().size() > 0) {
-      // Add the OWNER column if the scan is already only over specific columns
-      scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
-    }
-    scan.setMaxResultSize(1);
-    SingleColumnValueFilter checkOwner = new SingleColumnValueFilter(CF_QUEUE, COL_QUEUE_OWNER,
-    CompareOperator.EQUAL, serverNameBytes);
-    scan.setFilter(checkOwner);
-    ResultScanner scanner = null;
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      scanner = replicationTable.getScanner(scan);
-      Result result = scanner.next();
-      return (result == null || result.isEmpty()) ? null : result;
-    } finally {
-      if (scanner != null) {
-        scanner.close();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ce5dc89/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 cf95030..efa17a4 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
@@ -157,10 +157,8 @@ import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
 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.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
@@ -1148,15 +1146,12 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
 
     // Start replication zk node cleaner
-    if (conf.getClass("hbase.region.replica.replication.replicationQueues.class",
-      ReplicationFactory.defaultReplicationQueueClass) == ReplicationQueuesZKImpl.class) {
-      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);
-      }
+    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);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ce5dc89/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 9da0745..c57d9bb 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
@@ -94,8 +94,6 @@ public class TestMultiSlaveReplication {
     conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
         "org.apache.hadoop.hbase.replication.TestMasterReplication$CoprocessorCounter");
     conf1.setInt("hbase.master.cleaner.interval", 5 * 1000);
-    conf1.setClass("hbase.region.replica.replication.replicationQueues.class",
-        ReplicationQueuesZKImpl.class, ReplicationQueues.class);
 
     utility1 = new HBaseTestingUtility(conf1);
     utility1.startMiniZKCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ce5dc89/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
deleted file mode 100644
index 1ef525f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
+++ /dev/null
@@ -1,495 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.fs.FileSystem;
-import org.apache.hadoop.hbase.ChoreService;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-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.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.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.IOException;
-import java.util.List;
-
-import static junit.framework.TestCase.assertNull;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-@Category({ReplicationTests.class, MediumTests.class})
-public class TestReplicationStateHBaseImpl {
-
-  private static Configuration conf;
-  private static HBaseTestingUtility utility;
-  private static ZKWatcher zkw;
-  private static String replicationZNode;
-
-  private static ReplicationQueues rq1;
-  private static ReplicationQueues rq2;
-  private static ReplicationQueues rq3;
-  private static ReplicationQueuesClient rqc;
-  private static ReplicationPeers rp;
-
-
-  private static final String server0 = ServerName.valueOf("hostname0.example.org", 1234, -1L)
-    .toString();
-  private static final String server1 = ServerName.valueOf("hostname1.example.org", 1234, 1L)
-    .toString();
-  private static final String server2 = ServerName.valueOf("hostname2.example.org", 1234, 1L)
-    .toString();
-  private static final String server3 = ServerName.valueOf("hostname3.example.org", 1234, 1L)
-    .toString();
-
-  private static DummyServer ds0;
-  private static DummyServer ds1;
-  private static DummyServer ds2;
-  private static DummyServer ds3;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    utility = new HBaseTestingUtility();
-    conf = utility.getConfiguration();
-    conf.setClass("hbase.region.replica.replication.replicationQueues.class",
-      TableBasedReplicationQueuesImpl.class, ReplicationQueues.class);
-    conf.setClass("hbase.region.replica.replication.replicationQueuesClient.class",
-        TableBasedReplicationQueuesClientImpl.class, ReplicationQueuesClient.class);
-    utility.startMiniCluster();
-    zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
-    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
-    replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
-  }
-
-  @Before
-  public void setUp() {
-    try {
-      ds0 = new DummyServer(server0);
-      rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(
-        conf, ds0));
-      ds1 = new DummyServer(server1);
-      rq1 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds1, zkw));
-      rq1.init(server1);
-      ds2 = new DummyServer(server2);
-      rq2 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds2, zkw));
-      rq2.init(server2);
-      ds3 = new DummyServer(server3);
-      rq3 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds3, zkw));
-      rq3.init(server3);
-      rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
-      rp.init();
-    } catch (Exception e) {
-      fail("testReplicationStateHBaseConstruction received an exception" + e.getMessage());
-    }
-  }
-
-  @Test
-  public void checkNamingSchema() throws Exception {
-    assertTrue(rq1.isThisOurRegionServer(server1));
-    assertTrue(!rq1.isThisOurRegionServer(server1 + "a"));
-    assertTrue(!rq1.isThisOurRegionServer(null));
-  }
-
-  @Test
-  public void testSingleReplicationQueuesHBaseImpl() {
-    try {
-      // Test adding in WAL files
-      assertEquals(0, rq1.getAllQueues().size());
-      rq1.addLog("Queue1", "WALLogFile1.1");
-      assertEquals(1, rq1.getAllQueues().size());
-      rq1.addLog("Queue1", "WALLogFile1.2");
-      rq1.addLog("Queue1", "WALLogFile1.3");
-      rq1.addLog("Queue1", "WALLogFile1.4");
-      rq1.addLog("Queue2", "WALLogFile2.1");
-      rq1.addLog("Queue3", "WALLogFile3.1");
-      assertEquals(3, rq1.getAllQueues().size());
-      assertEquals(4, rq1.getLogsInQueue("Queue1").size());
-      assertEquals(1, rq1.getLogsInQueue("Queue2").size());
-      assertEquals(1, rq1.getLogsInQueue("Queue3").size());
-      // Make sure that abortCount is still 0
-      assertEquals(0, ds1.getAbortCount());
-      // Make sure that getting a log from a non-existent queue triggers an abort
-      assertNull(rq1.getLogsInQueue("Queue4"));
-      assertEquals(1, ds1.getAbortCount());
-    } catch (ReplicationException e) {
-      e.printStackTrace();
-      fail("testAddLog received a ReplicationException");
-    }
-    try {
-
-      // Test updating the log positions
-      assertEquals(0L, rq1.getLogPosition("Queue1", "WALLogFile1.1"));
-      rq1.setLogPosition("Queue1", "WALLogFile1.1", 123L);
-      assertEquals(123L, rq1.getLogPosition("Queue1", "WALLogFile1.1"));
-      rq1.setLogPosition("Queue1", "WALLogFile1.1", 123456789L);
-      assertEquals(123456789L, rq1.getLogPosition("Queue1", "WALLogFile1.1"));
-      rq1.setLogPosition("Queue2", "WALLogFile2.1", 242L);
-      assertEquals(242L, rq1.getLogPosition("Queue2", "WALLogFile2.1"));
-      rq1.setLogPosition("Queue3", "WALLogFile3.1", 243L);
-      assertEquals(243L, rq1.getLogPosition("Queue3", "WALLogFile3.1"));
-
-      // Test that setting log positions in non-existing logs will cause an abort
-      assertEquals(1, ds1.getAbortCount());
-      rq1.setLogPosition("NotHereQueue", "WALLogFile3.1", 243L);
-      assertEquals(2, ds1.getAbortCount());
-      rq1.setLogPosition("NotHereQueue", "NotHereFile", 243L);
-      assertEquals(3, ds1.getAbortCount());
-      rq1.setLogPosition("Queue1", "NotHereFile", 243l);
-      assertEquals(4, ds1.getAbortCount());
-
-      // Test reading log positions for non-existent queues and WAL's
-      try {
-        rq1.getLogPosition("Queue1", "NotHereWAL");
-        fail("Replication queue should have thrown a ReplicationException for reading from a " +
-          "non-existent WAL");
-      } catch (ReplicationException e) {
-      }
-      try {
-        rq1.getLogPosition("NotHereQueue", "NotHereWAL");
-        fail("Replication queue should have thrown a ReplicationException for reading from a " +
-          "non-existent queue");
-      } catch (ReplicationException e) {
-      }
-      // Test removing logs
-      rq1.removeLog("Queue1", "WALLogFile1.1");
-      assertEquals(3, rq1.getLogsInQueue("Queue1").size());
-      // Test removing queues
-      rq1.removeQueue("Queue2");
-      assertEquals(2, rq1.getAllQueues().size());
-      assertNull(rq1.getLogsInQueue("Queue2"));
-      // Test that getting logs from a non-existent queue aborts
-      assertEquals(5, ds1.getAbortCount());
-      // Test removing all queues for a Region Server
-      rq1.removeAllQueues();
-      assertEquals(0, rq1.getAllQueues().size());
-      assertNull(rq1.getLogsInQueue("Queue1"));
-      // Test that getting logs from a non-existent queue aborts
-      assertEquals(6, ds1.getAbortCount());
-      // Test removing a non-existent queue does not cause an abort. This is because we can
-      // attempt to remove a queue that has no corresponding Replication Table row (if we never
-      // registered a WAL for it)
-      rq1.removeQueue("NotHereQueue");
-      assertEquals(6, ds1.getAbortCount());
-    } catch (ReplicationException e) {
-      e.printStackTrace();
-      fail("testAddLog received a ReplicationException");
-    }
-  }
-
-  @Test
-  public void TestMultipleReplicationQueuesHBaseImpl () {
-    try {
-      rp.registerPeer("Queue1", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus1"));
-      rp.registerPeer("Queue2", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus2"));
-      rp.registerPeer("Queue3", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus3"));
-    } catch (ReplicationException e) {
-      fail("Failed to add peers to ReplicationPeers");
-    }
-    try {
-      // Test adding in WAL files
-      rq1.addLog("Queue1", "WALLogFile1.1");
-      rq1.addLog("Queue1", "WALLogFile1.2");
-      rq1.addLog("Queue1", "WALLogFile1.3");
-      rq1.addLog("Queue1", "WALLogFile1.4");
-      rq1.addLog("Queue2", "WALLogFile2.1");
-      rq1.addLog("Queue3", "WALLogFile3.1");
-      rq2.addLog("Queue1", "WALLogFile1.1");
-      rq2.addLog("Queue1", "WALLogFile1.2");
-      rq2.addLog("Queue2", "WALLogFile2.1");
-      rq3.addLog("Queue1", "WALLogFile1.1");
-      // Test adding logs to replication queues
-      assertEquals(3, rq1.getAllQueues().size());
-      assertEquals(2, rq2.getAllQueues().size());
-      assertEquals(1, rq3.getAllQueues().size());
-      assertEquals(4, rq1.getLogsInQueue("Queue1").size());
-      assertEquals(1, rq1.getLogsInQueue("Queue2").size());
-      assertEquals(1, rq1.getLogsInQueue("Queue3").size());
-      assertEquals(2, rq2.getLogsInQueue("Queue1").size());
-      assertEquals(1, rq2.getLogsInQueue("Queue2").size());
-      assertEquals(1, rq3.getLogsInQueue("Queue1").size());
-    } catch (ReplicationException e) {
-      e.printStackTrace();
-      fail("testAddLogs received a ReplicationException");
-    }
-    try {
-      // Test setting and reading offset in queues
-      rq1.setLogPosition("Queue1", "WALLogFile1.1", 1l);
-      rq1.setLogPosition("Queue1", "WALLogFile1.2", 2l);
-      rq1.setLogPosition("Queue1", "WALLogFile1.3", 3l);
-      rq1.setLogPosition("Queue2", "WALLogFile2.1", 4l);
-      rq1.setLogPosition("Queue2", "WALLogFile2.2", 5l);
-      rq1.setLogPosition("Queue3", "WALLogFile3.1", 6l);
-      rq2.setLogPosition("Queue1", "WALLogFile1.1", 7l);
-      rq2.setLogPosition("Queue2", "WALLogFile2.1", 8l);
-      rq3.setLogPosition("Queue1", "WALLogFile1.1", 9l);
-      assertEquals(1l, rq1.getLogPosition("Queue1", "WALLogFile1.1"));
-      assertEquals(2l, rq1.getLogPosition("Queue1", "WALLogFile1.2"));
-      assertEquals(4l, rq1.getLogPosition("Queue2", "WALLogFile2.1"));
-      assertEquals(6l, rq1.getLogPosition("Queue3", "WALLogFile3.1"));
-      assertEquals(7l, rq2.getLogPosition("Queue1", "WALLogFile1.1"));
-      assertEquals(8l, rq2.getLogPosition("Queue2", "WALLogFile2.1"));
-      assertEquals(9l, rq3.getLogPosition("Queue1", "WALLogFile1.1"));
-      assertEquals(rq1.getListOfReplicators().size(), 3);
-      assertEquals(rq2.getListOfReplicators().size(), 3);
-      assertEquals(rq3.getListOfReplicators().size(), 3);
-    } catch (ReplicationException e) {
-      fail("testAddLogs threw a ReplicationException");
-    }
-    try {
-      // Test claiming queues
-      List<String> claimedQueuesFromRq2 = rq1.getUnClaimedQueueIds(server2);
-      // Check to make sure that list of peers with outstanding queues is decremented by one
-      // after claimQueues
-      // Check to make sure that we claimed the proper number of queues
-      assertEquals(2, claimedQueuesFromRq2.size());
-      assertTrue(claimedQueuesFromRq2.contains("Queue1-" + server2));
-      assertTrue(claimedQueuesFromRq2.contains("Queue2-" + server2));
-      assertEquals(2, rq1.claimQueue(server2, "Queue1-" + server2).getSecond().size());
-      assertEquals(1, rq1.claimQueue(server2, "Queue2-" + server2).getSecond().size());
-      rq1.removeReplicatorIfQueueIsEmpty(server2);
-      assertEquals(rq1.getListOfReplicators().size(), 2);
-      assertEquals(rq2.getListOfReplicators().size(), 2);
-      assertEquals(rq3.getListOfReplicators().size(), 2);
-      assertEquals(5, rq1.getAllQueues().size());
-      // Check that all the logs in the other queue were claimed
-      assertEquals(2, rq1.getLogsInQueue("Queue1-" + server2).size());
-      assertEquals(1, rq1.getLogsInQueue("Queue2-" + server2).size());
-      // Check that the offsets of the claimed queues are the same
-      assertEquals(7l, rq1.getLogPosition("Queue1-" + server2, "WALLogFile1.1"));
-      assertEquals(8l, rq1.getLogPosition("Queue2-" + server2, "WALLogFile2.1"));
-      // Check that the queues were properly removed from rq2
-      assertEquals(0, rq2.getAllQueues().size());
-      assertNull(rq2.getLogsInQueue("Queue1"));
-      assertNull(rq2.getLogsInQueue("Queue2"));
-      // Check that non-existent peer queues are not claimed
-      rq1.addLog("UnclaimableQueue", "WALLogFile1.1");
-      rq1.addLog("UnclaimableQueue", "WALLogFile1.2");
-      assertEquals(6, rq1.getAllQueues().size());
-      List<String> claimedQueuesFromRq1 = rq3.getUnClaimedQueueIds(server1);
-      for(String queue : claimedQueuesFromRq1) {
-        rq3.claimQueue(server1, queue);
-      }
-      rq3.removeReplicatorIfQueueIsEmpty(server1);
-      assertEquals(rq1.getListOfReplicators().size(), 1);
-      assertEquals(rq2.getListOfReplicators().size(), 1);
-      assertEquals(rq3.getListOfReplicators().size(), 1);
-      // Note that we do not pick up the queue: UnclaimableQueue which was not registered in
-      // Replication Peers
-      assertEquals(6, rq3.getAllQueues().size());
-      // Test claiming non-existing queues
-      List<String> noQueues = rq3.getUnClaimedQueueIds("NotARealServer");
-      assertNull(noQueues);
-      assertEquals(6, rq3.getAllQueues().size());
-      // Test claiming own queues
-      noQueues = rq3.getUnClaimedQueueIds(server3);
-      Assert.assertNull(noQueues);
-      assertEquals(6, rq3.getAllQueues().size());
-      // Check that rq3 still remain on list of replicators
-      assertEquals(1, rq3.getListOfReplicators().size());
-    } catch (ReplicationException e) {
-      fail("testClaimQueue threw a ReplicationException");
-    }
-  }
-
-  @Test
-  public void TestReplicationQueuesClient() throws Exception{
-
-    // Test ReplicationQueuesClient log tracking
-    rq1.addLog("Queue1", "WALLogFile1.1");
-    assertEquals(1, rqc.getLogsInQueue(server1, "Queue1").size());
-    rq1.removeLog("Queue1", "WALLogFile1.1");
-    assertEquals(0, rqc.getLogsInQueue(server1, "Queue1").size());
-    rq2.addLog("Queue2", "WALLogFile2.1");
-    rq2.addLog("Queue2", "WALLogFile2.2");
-    assertEquals(2, rqc.getLogsInQueue(server2, "Queue2").size());
-    rq3.addLog("Queue1", "WALLogFile1.1");
-    rq3.addLog("Queue3", "WALLogFile3.1");
-    rq3.addLog("Queue3", "WALLogFile3.2");
-
-    // Test ReplicationQueueClient log tracking for faulty cases
-    assertEquals(0, ds0.getAbortCount());
-    assertNull(rqc.getLogsInQueue("NotHereServer", "NotHereQueue"));
-    assertNull(rqc.getLogsInQueue(server1, "NotHereQueue"));
-    assertNull(rqc.getLogsInQueue("NotHereServer", "WALLogFile1.1"));
-    assertEquals(3, ds0.getAbortCount());
-    // Test ReplicationQueueClient replicators
-    List<String> replicators = rqc.getListOfReplicators();
-    assertEquals(3, replicators.size());
-    assertTrue(replicators.contains(server1));
-    assertTrue(replicators.contains(server2));
-    rq1.removeQueue("Queue1");
-    assertEquals(2, rqc.getListOfReplicators().size());
-
-    // Test ReplicationQueuesClient queue tracking
-    assertEquals(0, rqc.getAllQueues(server1).size());
-    rq1.addLog("Queue2", "WALLogFile2.1");
-    rq1.addLog("Queue3", "WALLogFile3.1");
-    assertEquals(2, rqc.getAllQueues(server1).size());
-    rq1.removeAllQueues();
-    assertEquals(0, rqc.getAllQueues(server1).size());
-
-    // Test ReplicationQueuesClient queue tracking for faulty cases
-    assertEquals(0, rqc.getAllQueues("NotHereServer").size());
-
-    // Test ReplicationQueuesClient get all WAL's
-    assertEquals(5 , rqc.getAllWALs().size());
-    rq3.removeLog("Queue1", "WALLogFile1.1");
-    assertEquals(4, rqc.getAllWALs().size());
-    rq3.removeAllQueues();
-    assertEquals(2, rqc.getAllWALs().size());
-    rq2.removeAllQueues();
-    assertEquals(0, rqc.getAllWALs().size());
-  }
-
-  @After
-  public void clearQueues() throws Exception{
-    rq1.removeAllQueues();
-    rq2.removeAllQueues();
-    rq3.removeAllQueues();
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(0, rq2.getAllQueues().size());
-    assertEquals(0, rq3.getAllQueues().size());
-    ds0.resetAbortCount();
-    ds1.resetAbortCount();
-    ds2.resetAbortCount();
-    ds3.resetAbortCount();
-  }
-
-  @After
-  public void tearDown() throws KeeperException, IOException {
-    ZKUtil.deleteNodeRecursively(zkw, replicationZNode);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    utility.shutdownMiniCluster();
-    utility.shutdownMiniZKCluster();
-  }
-
-  static class DummyServer implements Server {
-    private String serverName;
-    private boolean isAborted = false;
-    private boolean isStopped = false;
-    private int abortCount = 0;
-
-    public DummyServer(String serverName) {
-      this.serverName = serverName;
-    }
-
-    @Override
-    public Configuration getConfiguration() {
-      return conf;
-    }
-
-    @Override
-    public ZKWatcher getZooKeeper() {
-      return null;
-    }
-
-    @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) {
-      abortCount++;
-      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() {
-      return null;
-    }
-
-    public int getAbortCount() {
-      return abortCount;
-    }
-
-    public void resetAbortCount() {
-      abortCount = 0;
-    }
-
-    @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/2ce5dc89/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.java
deleted file mode 100644
index 665eedb..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTableBase.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.replication;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests ReplicationTableBase behavior when the Master startup is delayed. The table initialization
- * should be non-blocking, but any method calls that access the table should be blocking.
- */
-@Category({ReplicationTests.class, MediumTests.class})
-public class TestReplicationTableBase {
-
-  private static long SLEEP_MILLIS = 5000;
-  private static long TIME_OUT_MILLIS = 3000;
-  private static Configuration conf;
-  private static HBaseTestingUtility utility;
-  private static ZKWatcher zkw;
-  private static ReplicationTableBase rb;
-  private static ReplicationQueues rq;
-  private static ReplicationQueuesClient rqc;
-  private volatile boolean asyncRequestSuccess = false;
-
-  @Test
-  public void testSlowStartup() throws Exception{
-    utility = new HBaseTestingUtility();
-    utility.startMiniZKCluster();
-    conf = utility.getConfiguration();
-    conf.setClass("hbase.region.replica.replication.replicationQueues.class",
-      TableBasedReplicationQueuesImpl.class, ReplicationQueues.class);
-    conf.setClass("hbase.region.replica.replication.replicationQueuesClient.class",
-      TableBasedReplicationQueuesClientImpl.class, ReplicationQueuesClient.class);
-    zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
-    utility.waitFor(0, TIME_OUT_MILLIS, new Waiter.ExplainingPredicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        rb = new ReplicationTableBase(conf, zkw) {};
-        rq = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(
-          conf, zkw, zkw));
-        rqc = ReplicationFactory.getReplicationQueuesClient(
-          new ReplicationQueuesClientArguments(conf, zkw, zkw));
-        return true;
-      }
-      @Override
-      public String explainFailure() throws Exception {
-        return "Failed to initialize ReplicationTableBase, TableBasedReplicationQueuesClient and " +
-          "TableBasedReplicationQueues after a timeout=" + TIME_OUT_MILLIS +
-          " ms. Their initialization " + "should be non-blocking";
-      }
-    });
-    final RequestReplicationQueueData async = new RequestReplicationQueueData();
-    async.start();
-    Thread.sleep(SLEEP_MILLIS);
-    // Test that the Replication Table has not been assigned and the methods are blocking
-    assertFalse(rb.getInitializationStatus());
-    assertFalse(asyncRequestSuccess);
-    utility.startMiniCluster();
-    // Test that the methods do return the correct results after getting the table
-    utility.waitFor(0, TIME_OUT_MILLIS, new Waiter.ExplainingPredicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        async.join();
-        return true;
-      }
-      @Override
-      public String explainFailure() throws Exception {
-        return "ReplicationQueue failed to return list of replicators even after Replication Table "
-          + "was initialized timeout=" + TIME_OUT_MILLIS + " ms";
-      }
-    });
-    assertTrue(asyncRequestSuccess);
-  }
-
-  public class RequestReplicationQueueData extends Thread {
-    @Override
-    public void run() {
-      assertEquals(0, rq.getListOfReplicators().size());
-      asyncRequestSuccess = true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ce5dc89/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java
deleted file mode 100644
index 19457e2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java
+++ /dev/null
@@ -1,63 +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.regionserver;
-
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
-import org.apache.hadoop.hbase.replication.TableBasedReplicationQueuesClientImpl;
-import org.apache.hadoop.hbase.replication.TableBasedReplicationQueuesImpl;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.junit.BeforeClass;
-import org.junit.experimental.categories.Category;
-
-/**
- * Tests the ReplicationSourceManager with TableBasedReplicationQueue's and
- * TableBasedReplicationQueuesClient
- */
-@Category({ReplicationTests.class, MediumTests.class})
-public class TestTableBasedReplicationSourceManagerImpl extends TestReplicationSourceManager {
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    conf = HBaseConfiguration.create();
-    conf.set("replication.replicationsource.implementation",
-      ReplicationSourceDummy.class.getCanonicalName());
-    conf.setLong("replication.sleep.before.failover", 2000);
-    conf.setInt("replication.source.maxretriesmultiplier", 10);
-
-    conf.setClass("hbase.region.replica.replication.replicationQueues.class",
-      TableBasedReplicationQueuesImpl.class, ReplicationQueues.class);
-    conf.setClass("hbase.region.replica.replication.replicationQueuesClient.class",
-      TableBasedReplicationQueuesClientImpl.class, ReplicationQueuesClient.class);
-    utility = new HBaseTestingUtility(conf);
-    utility.startMiniCluster();
-    Waiter.waitFor(conf, 3 * 1000,
-      () -> utility.getMiniHBaseCluster().getMaster().isInitialized());
-    utility.waitUntilAllRegionsAssigned(TableName.valueOf(
-        NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication"));
-    setupZkAndReplication();
-  }
-
-}


[02/17] hbase git commit: HBASE-19621 Revisit the methods in ReplicationPeerConfigBuilder

Posted by zh...@apache.org.
HBASE-19621 Revisit the methods in ReplicationPeerConfigBuilder


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

Branch: refs/heads/HBASE-19397
Commit: 1556939236016bb51e45ffa1e8038c74e0f0db75
Parents: 2ce5dc8
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Dec 25 14:29:39 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Tue Dec 26 14:40:31 2017 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfigUtil.java  |  9 ++----
 .../replication/ReplicationPeerConfig.java      | 10 +++----
 .../ReplicationPeerConfigBuilder.java           | 14 +++++++--
 .../replication/ReplicationPeersZKImpl.java     | 14 ++-------
 .../replication/TestReplicationAdmin.java       | 31 ++++++++++----------
 5 files changed, 38 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/15569392/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 a50d48f..012b309 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
@@ -26,7 +26,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.StringUtils;
@@ -277,17 +276,13 @@ public final class ReplicationPeerConfigUtil {
       builder.setReplicationEndpointImpl(peer.getReplicationEndpointImpl());
     }
 
-    Map<byte[], byte[]> peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     for (HBaseProtos.BytesBytesPair pair : peer.getDataList()) {
-      peerData.put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
+      builder.putPeerData(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
     }
-    builder.setPeerData(peerData);
 
-    Map<String, String> configuration = new HashMap<>();
     for (HBaseProtos.NameStringPair pair : peer.getConfigurationList()) {
-      configuration.put(pair.getName(), pair.getValue());
+      builder.putConfiguration(pair.getName(), pair.getValue());
     }
-    builder.setConfiguration(configuration);
 
     Map<TableName, List<String>> tableCFsMap = convert2Map(
       peer.getTableCfsList().toArray(new ReplicationProtos.TableCF[peer.getTableCfsCount()]));

http://git-wip-us.apache.org/repos/asf/hbase/blob/15569392/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 8f6b938..ab75dff 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
@@ -219,7 +219,7 @@ public class ReplicationPeerConfig {
     ReplicationPeerConfigBuilderImpl builder = new ReplicationPeerConfigBuilderImpl();
     builder.setClusterKey(peerConfig.getClusterKey())
         .setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl())
-        .setPeerData(peerConfig.getPeerData()).setConfiguration(peerConfig.getConfiguration())
+        .putAllPeerData(peerConfig.getPeerData()).putAllConfiguration(peerConfig.getConfiguration())
         .setTableCFsMap(peerConfig.getTableCFsMap()).setNamespaces(peerConfig.getNamespaces())
         .setReplicateAllUserTables(peerConfig.replicateAllUserTables())
         .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap())
@@ -264,14 +264,14 @@ public class ReplicationPeerConfig {
     }
 
     @Override
-    public ReplicationPeerConfigBuilder setPeerData(Map<byte[], byte[]> peerData) {
-      this.peerData = peerData;
+    public ReplicationPeerConfigBuilder putConfiguration(String key, String value) {
+      this.configuration.put(key, value);
       return this;
     }
 
     @Override
-    public ReplicationPeerConfigBuilder setConfiguration(Map<String, String> configuration) {
-      this.configuration = configuration;
+    public ReplicationPeerConfigBuilder putPeerData(byte[] key, byte[] value) {
+      this.peerData.put(key, value);
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/15569392/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
index b642acf..f3639b1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
@@ -43,9 +43,19 @@ public interface ReplicationPeerConfigBuilder {
    */
   ReplicationPeerConfigBuilder setReplicationEndpointImpl(String replicationEndpointImpl);
 
-  ReplicationPeerConfigBuilder setPeerData(Map<byte[], byte[]> peerData);
+  ReplicationPeerConfigBuilder putConfiguration(String key, String value);
 
-  ReplicationPeerConfigBuilder setConfiguration(Map<String, String> configuration);
+  default ReplicationPeerConfigBuilder putAllConfiguration(Map<String, String> configuration) {
+    configuration.forEach(this::putConfiguration);
+    return this;
+  }
+
+  ReplicationPeerConfigBuilder putPeerData(byte[] key, byte[] value);
+
+  default ReplicationPeerConfigBuilder putAllPeerData(Map<byte[], byte[]> peerData) {
+    peerData.forEach(this::putPeerData);
+    return this;
+  }
 
   ReplicationPeerConfigBuilder
       setTableCFsMap(Map<TableName, List<String>> tableCFsMap);

http://git-wip-us.apache.org/repos/asf/hbase/blob/15569392/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 ff6c07b..2f6d52c 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
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.replication;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -38,7 +37,6 @@ 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.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -363,17 +361,11 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
           + 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
+    // 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(newConfig);
-    Map<byte[], byte[]> peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    existingConfig.getPeerData().forEach(peerData::put);
-    newConfig.getPeerData().forEach(peerData::put);
-    builder.setPeerData(peerData);
-    Map<String, String> configuration = new HashMap<>();
-    existingConfig.getConfiguration().forEach(configuration::put);
-    newConfig.getConfiguration().forEach(configuration::put);
-    builder.setConfiguration(configuration);
+    builder.putAllConfiguration(existingConfig.getConfiguration());
+    builder.putAllPeerData(existingConfig.getPeerData());
 
     try {
       ZKUtil.setData(this.zookeeper, getPeerNode(id),

http://git-wip-us.apache.org/repos/asf/hbase/blob/15569392/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 dd33564..d2a16aa 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
@@ -36,6 +36,7 @@ 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;
@@ -121,39 +122,39 @@ public class TestReplicationAdmin {
    */
   @Test
   public void testAddRemovePeer() throws Exception {
-    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    ReplicationPeerConfigBuilder rpc1 = ReplicationPeerConfig.newBuilder();
     rpc1.setClusterKey(KEY_ONE);
-    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+    ReplicationPeerConfigBuilder rpc2 = ReplicationPeerConfig.newBuilder();
     rpc2.setClusterKey(KEY_SECOND);
     // Add a valid peer
-    admin.addPeer(ID_ONE, rpc1, null);
+    hbaseAdmin.addReplicationPeer(ID_ONE, rpc1.build());
     // try adding the same (fails)
     try {
-      admin.addPeer(ID_ONE, rpc1, null);
+      hbaseAdmin.addReplicationPeer(ID_ONE, rpc1.build());
     } catch (Exception e) {
       // OK!
     }
-    assertEquals(1, admin.getPeersCount());
+    assertEquals(1, hbaseAdmin.listReplicationPeers().size());
     // Try to remove an inexisting peer
     try {
-      admin.removePeer(ID_SECOND);
+      hbaseAdmin.removeReplicationPeer(ID_SECOND);
       fail();
-    } catch (Exception iae) {
+    } catch (Exception e) {
       // OK!
     }
-    assertEquals(1, admin.getPeersCount());
+    assertEquals(1, hbaseAdmin.listReplicationPeers().size());
     // Add a second since multi-slave is supported
     try {
-      admin.addPeer(ID_SECOND, rpc2, null);
-    } catch (Exception iae) {
+      hbaseAdmin.addReplicationPeer(ID_SECOND, rpc2.build());
+    } catch (Exception e) {
       fail();
     }
-    assertEquals(2, admin.getPeersCount());
+    assertEquals(2, hbaseAdmin.listReplicationPeers().size());
     // Remove the first peer we added
-    admin.removePeer(ID_ONE);
-    assertEquals(1, admin.getPeersCount());
-    admin.removePeer(ID_SECOND);
-    assertEquals(0, admin.getPeersCount());
+    hbaseAdmin.removeReplicationPeer(ID_ONE);
+    assertEquals(1, hbaseAdmin.listReplicationPeers().size());
+    hbaseAdmin.removeReplicationPeer(ID_SECOND);
+    assertEquals(0, hbaseAdmin.listReplicationPeers().size());
   }
 
   @Test


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

Branch: refs/heads/HBASE-19397
Commit: f60be0ea03991dbe64413ed63a7361a9cd17f471
Parents: 724ee67
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 21 21:59:46 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:03:19 2017 +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/f60be0ea/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/f60be0ea/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/f60be0ea/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/f60be0ea/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/f60be0ea/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/f60be0ea/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/f60be0ea/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;
 


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

Branch: refs/heads/HBASE-19397
Commit: a3395659d37eeb8eb560bc3dce1933aef27a4ded
Parents: d422629
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 25 18:49:56 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:07:55 2017 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |  19 +-
 .../replication/ReplicationPeersZKImpl.java     |  24 +-
 .../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         | 108 ++----
 .../master/ReplicationLogCleaner.java           |  35 +-
 .../regionserver/DumpReplicationQueues.java     |  77 ++--
 .../hbase/util/hbck/ReplicationChecker.java     |  14 +-
 .../client/TestAsyncReplicationAdminApi.java    |  31 +-
 .../replication/TestReplicationAdmin.java       |   2 +
 .../hbase/master/cleaner/TestLogsCleaner.java   |  29 +-
 .../cleaner/TestReplicationHFileCleaner.java    |  58 +--
 .../cleaner/TestReplicationZKNodeCleaner.java   |  12 +-
 .../replication/TestReplicationStateBasic.java  | 378 -------------------
 .../replication/TestReplicationStateZKImpl.java | 227 -----------
 .../TestReplicationSourceManagerZkImpl.java     |  84 ++---
 25 files changed, 907 insertions(+), 1323 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3395659/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/a3395659/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 06cdbe5..3581a05 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
@@ -32,10 +32,10 @@ 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;
@@ -48,6 +48,8 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
 /**
  * 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
@@ -79,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
@@ -506,14 +508,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)) {
@@ -524,7 +528,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/a3395659/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/a3395659/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/a3395659/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/a3395659/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 e85b42a..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.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.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/a3395659/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/a3395659/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/a3395659/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/a3395659/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/a3395659/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/a3395659/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/a3395659/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/a3395659/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 4e9d67a..39a6ba6 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,44 @@
-/*
- * 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.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-import org.apache.hadoop.hbase.shaded.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.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
+import org.apache.hadoop.hbase.shaded.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 +47,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 +62,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 +84,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 +110,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 +148,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/a3395659/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 773e10e..e8d6fa3 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.hadoop.hbase.shaded.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;
     }
@@ -112,9 +109,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);
     }
@@ -134,18 +129,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;
-    }
-  }
 }


[11/17] 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/d422629e/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/d422629e/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/d422629e/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/d422629e/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 fb29e9e..c2fcd8c 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;
@@ -31,18 +37,17 @@ 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.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;
@@ -50,15 +55,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
@@ -66,8 +62,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();
 
@@ -102,16 +96,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);
     }
   }
 
@@ -201,32 +196,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();
   }
 
   /**
@@ -422,7 +414,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/d422629e/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 540a67c..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/d422629e/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 6119e43..eaace03 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/d422629e/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");


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

Branch: refs/heads/HBASE-19397
Commit: e10313c4c6283b27fbab9e0db3e4b5cf183cc89e
Parents: a339565
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Dec 23 21:04:27 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:07:55 2017 +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/e10313c4/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/e10313c4/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)
 


[14/17] 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/a3395659/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 93b8649..1faaae3 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,6 +21,7 @@ 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;
@@ -48,17 +49,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.hadoop.hbase.shaded.com.google.common.util.concurrent.AtomicLongMap;
 
 /**
@@ -303,57 +305,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 +387,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/a3395659/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/a3395659/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/a3395659/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 c2fcd8c..8bb3230 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
@@ -48,6 +48,7 @@ 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.MediumTests;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -82,6 +83,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/a3395659/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 2d517c4..e678e1b 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,13 @@ 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.hadoop.hbase.shaded.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 +48,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 +61,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.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 @Category({MasterTests.class, MediumTests.class})
 public class TestLogsCleaner {
 
@@ -195,24 +192,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/a3395659/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 2f23301..cfad645 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,10 +24,7 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 
-import org.apache.hadoop.hbase.shaded.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;
@@ -45,7 +49,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 +66,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.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 @Category({ MasterTests.class, SmallTests.class })
 public class TestReplicationHFileCleaner {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class);
@@ -188,32 +192,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/a3395659/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/a3395659/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/a3395659/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/a3395659/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("");
-  }
 }


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

Branch: refs/heads/HBASE-19397
Commit: aa3bc1d9bb0ea2dd38a574bbf4d1d9d17289302f
Parents: 7459c8c
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 26 21:10:00 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:10:00 2017 +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/aa3bc1d9/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/aa3bc1d9/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 8bb3230..a198d20 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
@@ -112,6 +112,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


[03/17] hbase git commit: HBASE-19550 Wrap the cell passed via Mutation#add(Cell) to be of ExtendedCell

Posted by zh...@apache.org.
HBASE-19550 Wrap the cell passed via Mutation#add(Cell) to be of ExtendedCell


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

Branch: refs/heads/HBASE-19397
Commit: 7ce1943ef3ef6865d0292364d5d9b51adf3e9827
Parents: 1556939
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Tue Dec 26 16:39:51 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Tue Dec 26 16:39:51 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Append.java  |  19 +-
 .../org/apache/hadoop/hbase/client/Delete.java  |  17 +-
 .../apache/hadoop/hbase/client/Increment.java   |   9 +-
 .../apache/hadoop/hbase/client/Mutation.java    | 208 ++++++++++
 .../org/apache/hadoop/hbase/client/Put.java     |  25 +-
 .../TestPassCustomCellViaRegionObserver.java    | 403 +++++++++++++++++++
 6 files changed, 628 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7ce1943e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
index 0cb51a2..b2995ed 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
@@ -17,12 +17,12 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.UUID;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.security.access.Permission;
@@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Performs Append operations on a single row.
@@ -44,6 +46,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Public
 public class Append extends Mutation {
+  private static final Logger LOG = LoggerFactory.getLogger(Append.class);
   private static final long HEAP_OVERHEAD = ClassSize.REFERENCE + ClassSize.TIMERANGE;
   private TimeRange tr = new TimeRange();
 
@@ -176,14 +179,12 @@ public class Append extends Mutation {
    */
   @SuppressWarnings("unchecked")
   public Append add(final Cell cell) {
-    // Presume it is KeyValue for now.
-    byte [] family = CellUtil.cloneFamily(cell);
-
-    // Get cell list for the family
-    List<Cell> list = getCellList(family);
-
-    // find where the new entry should be placed in the List
-    list.add(cell);
+    try {
+      super.add(cell);
+    } catch (IOException e) {
+      // we eat the exception of wrong row for BC..
+      LOG.error(e.toString(), e);
+    }
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7ce1943e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index 57f5648..b5a0b93 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -25,7 +25,6 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.UUID;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.security.access.Permission;
@@ -170,22 +169,12 @@ public class Delete extends Mutation implements Comparable<Row> {
 
   /**
    * Add an existing delete marker to this Delete object.
-   * @param kv An existing KeyValue of type "delete".
+   * @param cell An existing cell of type "delete".
    * @return this for invocation chaining
    * @throws IOException
    */
-  public Delete add(Cell kv) throws IOException {
-    if (!CellUtil.isDelete(kv)) {
-      throw new IOException("The recently added KeyValue is not of type "
-          + "delete. Rowkey: " + Bytes.toStringBinary(this.row));
-    }
-    if (!CellUtil.matchingRows(kv, this.row)) {
-      throw new WrongRowIOException("The row in " + kv.toString() +
-        " doesn't match the original one " +  Bytes.toStringBinary(this.row));
-    }
-    byte [] family = CellUtil.cloneFamily(kv);
-    List<Cell> list = getCellList(family);
-    list.add(kv);
+  public Delete add(Cell cell) throws IOException {
+    super.add(cell);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7ce1943e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
index e9ae8fb..1ccc7e9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
@@ -98,14 +98,7 @@ public class Increment extends Mutation implements Comparable<Row> {
    * @throws java.io.IOException e
    */
   public Increment add(Cell cell) throws IOException{
-    byte [] family = CellUtil.cloneFamily(cell);
-    List<Cell> list = getCellList(family);
-    //Checking that the row of the kv is the same as the put
-    if (!CellUtil.matchingRows(cell, this.row)) {
-      throw new WrongRowIOException("The row in " + cell +
-        " doesn't match the original one " +  Bytes.toStringBinary(this.row));
-    }
-    list.add(cell);
+    super.add(cell);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7ce1943e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index 3983f35..9472d70 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -18,24 +18,31 @@
 
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Optional;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ExtendedCell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.RawCell;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.io.HeapSize;
@@ -53,6 +60,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteArrayDataInput;
 import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteArrayDataOutput;
 import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteStreams;
@@ -757,4 +765,204 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
           HConstants.MAX_ROW_LENGTH);
     }
   }
+
+  Mutation add(Cell cell) throws IOException {
+    //Checking that the row of the kv is the same as the mutation
+    // TODO: It is fraught with risk if user pass the wrong row.
+    // Throwing the IllegalArgumentException is more suitable I'd say.
+    if (!CellUtil.matchingRows(cell, this.row)) {
+      throw new WrongRowIOException("The row in " + cell.toString() +
+        " doesn't match the original one " +  Bytes.toStringBinary(this.row));
+    }
+
+    if (cell.getFamilyArray() == null || cell.getFamilyLength() == 0) {
+      throw new IllegalArgumentException("Family cannot be null");
+    }
+
+    byte[] family = CellUtil.cloneFamily(cell);
+    if (cell instanceof ExtendedCell) {
+      getCellList(family).add(cell);
+    } else {
+      getCellList(family).add(new CellWrapper(cell));
+    }
+    return this;
+  }
+
+  private static final class CellWrapper implements ExtendedCell {
+    private static final long FIXED_OVERHEAD = ClassSize.align(
+      ClassSize.OBJECT              // object header
+        + KeyValue.TIMESTAMP_SIZE       // timestamp
+        + Bytes.SIZEOF_LONG             // sequence id
+        + 1 * ClassSize.REFERENCE);     // references to cell
+    private final Cell cell;
+    private long sequenceId;
+    private long timestamp;
+
+    CellWrapper(Cell cell) {
+      assert !(cell instanceof ExtendedCell);
+      this.cell = cell;
+      this.sequenceId = cell.getSequenceId();
+      this.timestamp = cell.getTimestamp();
+    }
+
+    @Override
+    public void setSequenceId(long seqId) {
+      sequenceId = seqId;
+    }
+
+    @Override
+    public void setTimestamp(long ts) {
+      timestamp = ts;
+    }
+
+    @Override
+    public void setTimestamp(byte[] ts) {
+      timestamp = Bytes.toLong(ts);
+    }
+
+    @Override
+    public long getSequenceId() {
+      return sequenceId;
+    }
+
+    @Override
+    public byte[] getValueArray() {
+      return cell.getValueArray();
+    }
+
+    @Override
+    public int getValueOffset() {
+      return cell.getValueOffset();
+    }
+
+    @Override
+    public int getValueLength() {
+      return cell.getValueLength();
+    }
+
+    @Override
+    public byte[] getTagsArray() {
+      return cell.getTagsArray();
+    }
+
+    @Override
+    public int getTagsOffset() {
+      return cell.getTagsOffset();
+    }
+
+    @Override
+    public int getTagsLength() {
+      return cell.getTagsLength();
+    }
+
+    @Override
+    public byte[] getRowArray() {
+      return cell.getRowArray();
+    }
+
+    @Override
+    public int getRowOffset() {
+      return cell.getRowOffset();
+    }
+
+    @Override
+    public short getRowLength() {
+      return cell.getRowLength();
+    }
+
+    @Override
+    public byte[] getFamilyArray() {
+      return cell.getFamilyArray();
+    }
+
+    @Override
+    public int getFamilyOffset() {
+      return cell.getFamilyOffset();
+    }
+
+    @Override
+    public byte getFamilyLength() {
+      return cell.getFamilyLength();
+    }
+
+    @Override
+    public byte[] getQualifierArray() {
+      return cell.getQualifierArray();
+    }
+
+    @Override
+    public int getQualifierOffset() {
+      return cell.getQualifierOffset();
+    }
+
+    @Override
+    public int getQualifierLength() {
+      return cell.getQualifierLength();
+    }
+
+    @Override
+    public long getTimestamp() {
+      return timestamp;
+    }
+
+    @Override
+    public byte getTypeByte() {
+      return cell.getTypeByte();
+    }
+
+    @Override
+    public Optional<Tag> getTag(byte type) {
+      if (cell instanceof RawCell) {
+        return ((RawCell) cell).getTag(type);
+      }
+      int length = getTagsLength();
+      int offset = getTagsOffset();
+      int pos = offset;
+      while (pos < offset + length) {
+        int tagLen = Bytes.readAsInt(getTagsArray(), pos, TAG_LENGTH_SIZE);
+        if (getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
+          return Optional.of(new ArrayBackedTag(getTagsArray(), pos,
+            tagLen + TAG_LENGTH_SIZE));
+        }
+        pos += TAG_LENGTH_SIZE + tagLen;
+      }
+      return Optional.empty();
+    }
+
+    @Override
+    public List<Tag> getTags() {
+      if (cell instanceof RawCell) {
+        return ((RawCell) cell).getTags();
+      }
+      return Lists.newArrayList(PrivateCellUtil.tagsIterator(cell));
+    }
+
+    @Override
+    public byte[] cloneTags() {
+      if (cell instanceof RawCell) {
+        return ((RawCell) cell).cloneTags();
+      } else {
+        return PrivateCellUtil.cloneTags(cell);
+      }
+    }
+
+    private long heapOverhead() {
+      return FIXED_OVERHEAD
+        + ClassSize.ARRAY // row
+        + getFamilyLength() == 0 ? 0 : ClassSize.ARRAY
+        + getQualifierLength() == 0 ? 0 : ClassSize.ARRAY
+        + getValueLength() == 0 ? 0 : ClassSize.ARRAY
+        + getTagsLength() == 0 ? 0 : ClassSize.ARRAY;
+    }
+
+    @Override
+    public long heapSize() {
+      return heapOverhead()
+        + ClassSize.align(getRowLength())
+        + ClassSize.align(getFamilyLength())
+        + ClassSize.align(getQualifierLength())
+        + ClassSize.align(getValueLength())
+        + ClassSize.align(getTagsLength());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7ce1943e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
index 1a1176f..34ddf08 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
@@ -26,7 +26,6 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.UUID;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.IndividualBytesFieldCell;
 import org.apache.hadoop.hbase.KeyValue;
@@ -275,30 +274,12 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
    * Add the specified KeyValue to this Put operation.  Operation assumes that
    * the passed KeyValue is immutable and its backing array will not be modified
    * for the duration of this Put.
-   * @param kv individual KeyValue
+   * @param cell individual cell
    * @return this
    * @throws java.io.IOException e
    */
-  public Put add(Cell kv) throws IOException {
-    // Family can not be null, otherwise NullPointerException is thrown when putting
-    // the cell into familyMap
-    if (kv.getFamilyArray() == null) {
-      throw new IllegalArgumentException("Family cannot be null");
-    }
-
-    // Check timestamp
-    if (ts < 0) {
-      throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + ts);
-    }
-
-    byte [] family = CellUtil.cloneFamily(kv);
-    List<Cell> list = getCellList(family);
-    //Checking that the row of the kv is the same as the put
-    if (!CellUtil.matchingRows(kv, this.row)) {
-      throw new WrongRowIOException("The row in " + kv.toString() +
-        " doesn't match the original one " +  Bytes.toStringBinary(this.row));
-    }
-    list.add(kv);
+  public Put add(Cell cell) throws IOException {
+    super.add(cell);
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7ce1943e/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java
new file mode 100644
index 0000000..53abfa0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPassCustomCellViaRegionObserver.java
@@ -0,0 +1,403 @@
+/**
+ *
+ * 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.coprocessor;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+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.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.filter.ByteArrayComparable;
+import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ CoprocessorTests.class, MediumTests.class })
+public class TestPassCustomCellViaRegionObserver {
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TestPassCustomCellViaRegionObserver.class);
+
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+    withTimeout(this.getClass()).withLookingForStuckThread(true).build();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  private TableName tableName;
+  private Table table = null;
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static final byte[] ROW = Bytes.toBytes("ROW");
+  private static final byte[] FAMILY = Bytes.toBytes("FAMILY");
+  private static final byte[] QUALIFIER = Bytes.toBytes("QUALIFIER");
+  private static final byte[] VALUE = Bytes.toBytes(10L);
+  private static final byte[] APPEND_VALUE = Bytes.toBytes("MB");
+
+  private static final byte[] QUALIFIER_FROM_CP = Bytes.toBytes("QUALIFIER_FROM_CP");
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    // small retry number can speed up the failed tests.
+    UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
+    UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void clearTable() throws IOException {
+    RegionObserverImpl.COUNT.set(0);
+    tableName = TableName.valueOf(testName.getMethodName());
+    if (table != null) {
+      table.close();
+    }
+    try (Admin admin = UTIL.getAdmin()) {
+      for (TableName name : admin.listTableNames()) {
+        try {
+          admin.disableTable(name);
+        } catch (IOException e) {
+        }
+        admin.deleteTable(name);
+      }
+      table = UTIL.createTable(TableDescriptorBuilder.newBuilder(tableName)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY))
+        .addCoprocessor(RegionObserverImpl.class.getName())
+        .build(), null);
+    }
+  }
+
+  @Test
+  public void testMutation() throws Exception {
+
+    Put put = new Put(ROW);
+    put.addColumn(FAMILY, QUALIFIER, VALUE);
+    table.put(put);
+    byte[] value = VALUE;
+    assertResult(table.get(new Get(ROW)), value, value);
+    assertObserverHasExecuted();
+
+    Increment inc = new Increment(ROW);
+    inc.addColumn(FAMILY, QUALIFIER, 10L);
+    table.increment(inc);
+    // QUALIFIER -> 10 (put) + 10 (increment)
+    // QUALIFIER_FROM_CP -> 10 (from cp's put) + 10 (from cp's increment)
+    value = Bytes.toBytes(20L);
+    assertResult(table.get(new Get(ROW)), value, value);
+    assertObserverHasExecuted();
+
+    Append append = new Append(ROW);
+    append.addColumn(FAMILY, QUALIFIER, APPEND_VALUE);
+    table.append(append);
+    // 10L + "MB"
+    value = ByteBuffer.wrap(new byte[value.length + APPEND_VALUE.length])
+      .put(value)
+      .put(APPEND_VALUE)
+      .array();
+    assertResult(table.get(new Get(ROW)), value, value);
+    assertObserverHasExecuted();
+
+    Delete delete = new Delete(ROW);
+    delete.addColumns(FAMILY, QUALIFIER);
+    table.delete(delete);
+    assertTrue(Arrays.asList(table.get(new Get(ROW)).rawCells()).toString(),
+      table.get(new Get(ROW)).isEmpty());
+    assertObserverHasExecuted();
+
+    assertTrue(table.checkAndPut(ROW, FAMILY, QUALIFIER, null, put));
+    assertObserverHasExecuted();
+
+    assertTrue(table.checkAndDelete(ROW, FAMILY, QUALIFIER, VALUE, delete));
+    assertObserverHasExecuted();
+
+    assertTrue(table.get(new Get(ROW)).isEmpty());
+  }
+
+  @Test
+  public void testMultiPut() throws Exception {
+    List<Put> puts = IntStream.range(0, 10)
+      .mapToObj(i -> new Put(ROW).addColumn(FAMILY, Bytes.toBytes(i), VALUE))
+      .collect(Collectors.toList());
+    table.put(puts);
+    assertResult(table.get(new Get(ROW)), VALUE);
+    assertObserverHasExecuted();
+
+    List<Delete> deletes = IntStream.range(0, 10)
+      .mapToObj(i -> new Delete(ROW).addColumn(FAMILY, Bytes.toBytes(i)))
+      .collect(Collectors.toList());
+    table.delete(deletes);
+    assertTrue(table.get(new Get(ROW)).isEmpty());
+    assertObserverHasExecuted();
+  }
+
+  private static void assertObserverHasExecuted() {
+    assertTrue(RegionObserverImpl.COUNT.getAndSet(0) > 0);
+  }
+
+  private static void assertResult(Result result, byte[] expectedValue) {
+    assertFalse(result.isEmpty());
+    for (Cell c : result.rawCells()) {
+      assertTrue(c.toString(), Bytes.equals(ROW, CellUtil.cloneRow(c)));
+      assertTrue(c.toString(), Bytes.equals(FAMILY, CellUtil.cloneFamily(c)));
+      assertTrue(c.toString(), Bytes.equals(expectedValue, CellUtil.cloneValue(c)));
+    }
+  }
+
+  private static void assertResult(Result result, byte[] expectedValue, byte[] expectedFromCp) {
+    assertFalse(result.isEmpty());
+    for (Cell c : result.rawCells()) {
+      assertTrue(c.toString(), Bytes.equals(ROW, CellUtil.cloneRow(c)));
+      assertTrue(c.toString(), Bytes.equals(FAMILY, CellUtil.cloneFamily(c)));
+      if (Bytes.equals(QUALIFIER, CellUtil.cloneQualifier(c))) {
+        assertTrue(c.toString(), Bytes.equals(expectedValue, CellUtil.cloneValue(c)));
+      } else if (Bytes.equals(QUALIFIER_FROM_CP, CellUtil.cloneQualifier(c))) {
+        assertTrue(c.toString(), Bytes.equals(expectedFromCp, CellUtil.cloneValue(c)));
+      } else {
+        fail("No valid qualifier");
+      }
+    }
+  }
+
+  private static Cell createCustomCell(byte[] row, byte[] family, byte[] qualifier,
+    Cell.DataType type, byte[] value) {
+    return new Cell() {
+
+      private byte[] getArray(byte[] array) {
+        return array == null ? HConstants.EMPTY_BYTE_ARRAY : array;
+      }
+
+      private int length(byte[] array) {
+        return array == null ? 0 : array.length;
+      }
+
+      @Override
+      public byte[] getRowArray() {
+        return getArray(row);
+      }
+
+      @Override
+      public int getRowOffset() {
+        return 0;
+      }
+
+      @Override
+      public short getRowLength() {
+        return (short) length(row);
+      }
+
+      @Override
+      public byte[] getFamilyArray() {
+        return getArray(family);
+      }
+
+      @Override
+      public int getFamilyOffset() {
+        return 0;
+      }
+
+      @Override
+      public byte getFamilyLength() {
+        return (byte) length(family);
+      }
+
+      @Override
+      public byte[] getQualifierArray() {
+        return getArray(qualifier);
+      }
+
+      @Override
+      public int getQualifierOffset() {
+        return 0;
+      }
+
+      @Override
+      public int getQualifierLength() {
+        return length(qualifier);
+      }
+
+      @Override
+      public long getTimestamp() {
+        return HConstants.LATEST_TIMESTAMP;
+      }
+
+      @Override
+      public byte getTypeByte() {
+        return type.getCode();
+      }
+
+      @Override
+      public long getSequenceId() {
+        return 0;
+      }
+
+      @Override
+      public byte[] getValueArray() {
+        return getArray(value);
+      }
+
+      @Override
+      public int getValueOffset() {
+        return 0;
+      }
+
+      @Override
+      public int getValueLength() {
+        return length(value);
+      }
+
+      @Override
+      public byte[] getTagsArray() {
+        return getArray(null);
+      }
+
+      @Override
+      public int getTagsOffset() {
+        return 0;
+      }
+
+      @Override
+      public int getTagsLength() {
+        return length(null);
+      }
+
+      @Override
+      public DataType getType() {
+        return type;
+      }
+    };
+  }
+
+  private static Cell createCustomCell(Put put) {
+    return createCustomCell(put.getRow(), FAMILY, QUALIFIER_FROM_CP, Cell.DataType.Put, VALUE);
+  }
+
+  private static Cell createCustomCell(Append append) {
+    return createCustomCell(append.getRow(), FAMILY, QUALIFIER_FROM_CP, Cell.DataType.Put,
+      APPEND_VALUE);
+  }
+
+  private static Cell createCustomCell(Increment inc) {
+    return createCustomCell(inc.getRow(), FAMILY, QUALIFIER_FROM_CP, Cell.DataType.Put, VALUE);
+  }
+
+  private static Cell createCustomCell(Delete delete) {
+    return createCustomCell(delete.getRow(), FAMILY, QUALIFIER_FROM_CP,
+      Cell.DataType.DeleteColumn, null);
+  }
+
+  public static class RegionObserverImpl implements RegionCoprocessor, RegionObserver {
+    static final AtomicInteger COUNT = new AtomicInteger(0);
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit,
+      Durability durability) throws IOException {
+      put.add(createCustomCell(put));
+      COUNT.incrementAndGet();
+    }
+
+    @Override
+    public void preDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete,
+      WALEdit edit, Durability durability) throws IOException {
+      delete.add(createCustomCell(delete));
+      COUNT.incrementAndGet();
+    }
+
+    @Override
+    public boolean preCheckAndPut(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
+      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, Put put,
+      boolean result) throws IOException {
+      put.add(createCustomCell(put));
+      COUNT.incrementAndGet();
+      return result;
+    }
+
+    @Override
+    public boolean preCheckAndDelete(ObserverContext<RegionCoprocessorEnvironment> c, byte[] row,
+      byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator,
+      Delete delete, boolean result) throws IOException {
+      delete.add(createCustomCell(delete));
+      COUNT.incrementAndGet();
+      return result;
+    }
+
+    @Override
+    public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
+      throws IOException {
+      append.add(createCustomCell(append));
+      COUNT.incrementAndGet();
+      return null;
+    }
+
+
+    @Override
+    public Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> c, Increment increment)
+      throws IOException {
+      increment.add(createCustomCell(increment));
+      COUNT.incrementAndGet();
+      return null;
+    }
+
+  }
+
+}


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

Branch: refs/heads/HBASE-19397
Commit: f6c697d331dcb502e567cbfd5989cece7fa533d7
Parents: 4faf7df
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 19 15:50:57 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:03:19 2017 +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/f6c697d3/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 ff2722e..cf8e198 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
@@ -2463,7 +2463,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)
@@ -2474,7 +2474,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
    */
@@ -2482,6 +2482,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
@@ -2489,6 +2520,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
@@ -2496,6 +2539,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
@@ -2503,6 +2558,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
@@ -2513,13 +2580,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/f6c697d3/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 af3916d..5998fb2 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
@@ -40,6 +40,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;
@@ -199,7 +200,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;
 
 /**
@@ -3779,6 +3785,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 {
@@ -3851,50 +3876,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
@@ -3913,14 +3970,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/f6c697d3/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 dac83f3..e8ab786 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
@@ -38,6 +38,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;
@@ -1522,47 +1523,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
@@ -1581,13 +1569,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
@@ -2546,6 +2532,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) -> {


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

Branch: refs/heads/HBASE-19397
Commit: d422629e865bc1a14281a2d31f9f63070376f4be
Parents: 99c9bb8
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 22 14:37:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:03:53 2017 +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       |  62 ++-
 .../hbase/master/MockNoopMasterServices.java    |  10 +-
 .../hbase/master/TestMasterNoCluster.java       |   4 +-
 .../TestReplicationDisableInactivePeer.java     |   6 +-
 26 files changed, 1749 insertions(+), 311 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d422629e/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/d422629e/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index ab22199..4e3cea0 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/d422629e/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/d422629e/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/d422629e/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 ad970c6..edbbd4b 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/d422629e/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/d422629e/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/d422629e/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/d422629e/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/d422629e/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/d422629e/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/d422629e/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 4d3b265..468ff89 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
@@ -39,6 +39,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.CountDownLatch;
 import java.util.concurrent.ExecutionException;
@@ -135,7 +136,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;
@@ -327,7 +328,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
@@ -698,8 +699,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);
@@ -717,7 +718,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();
@@ -762,9 +763,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 {
     activeMaster = true;
     Thread zombieDetector = new Thread(new InitializationMonitor(this),
         "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
@@ -3351,18 +3351,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
@@ -3381,7 +3382,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);
     }
@@ -3531,7 +3533,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/d422629e/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 43df8b1..67baf14 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/d422629e/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/d422629e/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/d422629e/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/d422629e/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/d422629e/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/d422629e/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


[10/17] 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/99c9bb8e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/99c9bb8e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/99c9bb8e

Branch: refs/heads/HBASE-19397
Commit: 99c9bb8e4454715618c91fbb9634faa158f7b454
Parents: f60be0e
Author: huzheng <op...@gmail.com>
Authored: Wed Dec 20 10:47:18 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:03:53 2017 +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      | 180 ++++-----------
 .../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 ++-
 .../replication/TestReplicationAdmin.java       |   2 +-
 .../TestReplicationAdminUsingProcedure.java     | 226 +++++++++++++++++++
 .../replication/DummyModifyPeerProcedure.java   |  48 ----
 .../TestDummyModifyPeerProcedure.java           |  80 -------
 .../TestReplicationTrackerZKImpl.java           |  51 -----
 .../TestReplicationSourceManager.java           |  32 ++-
 22 files changed, 533 insertions(+), 472 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/99c9bb8e/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 267dc7a..d5285dc 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;
@@ -199,7 +201,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);
   }
 
   /**
@@ -224,10 +226,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/99c9bb8e/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 17b1141..8954d04 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;
@@ -282,7 +284,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);
   }
 
   /**
@@ -307,10 +309,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);
     }
   }
 
@@ -1962,7 +1965,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/99c9bb8e/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/99c9bb8e/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/99c9bb8e/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..5d051a0 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,6 +20,7 @@ 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;
@@ -33,12 +34,10 @@ 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;
 
@@ -47,14 +46,13 @@ 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/99c9bb8e/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 3309a4b..06cdbe5 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
@@ -495,21 +495,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/99c9bb8e/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/99c9bb8e/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/99c9bb8e/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/99c9bb8e/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 c390d09..2057057 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/99c9bb8e/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/99c9bb8e/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/99c9bb8e/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/99c9bb8e/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 d8212e9..571ee75 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/99c9bb8e/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 f4f35ae..19ea240 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/99c9bb8e/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 07c53e1..a263fc3 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/99c9bb8e/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 d2a16aa..fb29e9e 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
@@ -764,4 +764,4 @@ public class TestReplicationAdmin {
     assertEquals(2097152, admin.getPeerConfig(ID_ONE).getBandwidth());
     admin.removePeer(ID_ONE);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/99c9bb8e/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/99c9bb8e/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/99c9bb8e/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/99c9bb8e/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/99c9bb8e/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 307ea7f..9f234a8 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 {


[05/17] 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/054e74fc
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/054e74fc
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/054e74fc

Branch: refs/heads/HBASE-19397
Commit: 054e74fc197b5492eb1fbcdf9a7db1fad9e615db
Parents: 7ce1943
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 15 21:06:44 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:03:19 2017 +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 |  90 ++++----
 .../master/replication/ModifyPeerProcedure.java | 127 +++++++++++
 .../master/replication/RefreshPeerCallable.java |  67 ++++++
 .../replication/RefreshPeerProcedure.java       | 197 ++++++++++++++++
 .../hbase/procedure2/RSProcedureCallable.java   |  43 ++++
 .../hbase/regionserver/HRegionServer.java       |  69 +++++-
 .../hbase/regionserver/RSRpcServices.java       |  56 +++--
 .../handler/RSProcedureHandler.java             |  51 +++++
 .../assignment/TestAssignmentManager.java       |  20 +-
 .../replication/DummyModifyPeerProcedure.java   |  41 ++++
 .../TestDummyModifyPeerProcedure.java           |  80 +++++++
 .../security/access/TestAccessController.java   |   6 +-
 24 files changed, 1109 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/054e74fc/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/054e74fc/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 54f2b08..e9a6906 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/054e74fc/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/054e74fc/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/054e74fc/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/054e74fc/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/054e74fc/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/054e74fc/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 efa17a4..9d0555f 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
@@ -140,6 +139,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;
@@ -328,8 +328,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,
@@ -3525,4 +3524,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/054e74fc/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 2e4b5e1..1d6949c 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/054e74fc/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 6e671c8..43e75ed 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/054e74fc/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 c60de5c..dd85f5c 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/054e74fc/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/054e74fc/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 72e0846..22b70f1 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
@@ -37,6 +37,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
@@ -49,6 +50,7 @@ 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;
 
 /**
  * A remote procecdure dispatcher for regionservers.
@@ -222,7 +224,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 +236,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 +288,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 +312,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 +326,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 +368,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 +389,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 +438,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 +476,11 @@ public class RSProcedureDispatcher
         submitTask(new CloseRegionRemoteCall(serverName, op));
       }
     }
+
+    @Override
+    public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) {
+      throw new UnsupportedOperationException();
+    }
   }
 
   // ==========================================================================
@@ -489,13 +488,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/054e74fc/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/054e74fc/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/054e74fc/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/054e74fc/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/054e74fc/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 8ead08c..0799ca6 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;
@@ -51,6 +48,10 @@ 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.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -117,6 +118,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 +129,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;
@@ -175,6 +178,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
@@ -206,6 +210,7 @@ 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;
@@ -1932,6 +1937,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);
@@ -3725,4 +3732,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/054e74fc/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 584d0a2..2ad8fc8 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
@@ -99,6 +98,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;
@@ -171,6 +171,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;
@@ -3434,23 +3435,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 =
@@ -3467,4 +3451,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/054e74fc/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/054e74fc/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/054e74fc/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/054e74fc/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/054e74fc/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 138a40e..cade419 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;
@@ -119,8 +120,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
@@ -136,6 +135,9 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
+
 /**
  * Performs authorization checks for common operations, according to different
  * levels of authorized users.


[08/17] 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/4faf7dfc
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4faf7dfc
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4faf7dfc

Branch: refs/heads/HBASE-19397
Commit: 4faf7dfc63da77b759e4f1d503c9813a02d4e61a
Parents: 054e74f
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 18 15:22:36 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:03:19 2017 +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 | 100 ++++++++-----------
 .../hadoop/hbase/master/MasterRpcServices.java  |   4 +-
 .../hadoop/hbase/master/MasterServices.java     |  26 +++--
 .../assignment/RegionTransitionProcedure.java   |  11 +-
 .../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       |   6 +-
 .../regionserver/RefreshPeerCallable.java       |  70 +++++++++++++
 .../hbase/master/MockNoopMasterServices.java    |  23 +++--
 .../replication/DummyModifyPeerProcedure.java   |  13 ++-
 24 files changed, 736 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4faf7dfc/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 e9a6906..1235b33 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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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 2f6d52c..3309a4b 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
@@ -526,7 +526,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);
           }
         }
@@ -534,7 +534,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/4faf7dfc/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 9d0555f..4d3b265 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.master;
 
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
@@ -49,10 +50,12 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
+
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 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.ClusterMetrics.Option;
@@ -127,7 +130,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;
@@ -140,6 +149,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;
@@ -168,7 +178,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;
@@ -329,15 +338,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;
@@ -1193,7 +1202,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));
@@ -2313,11 +2321,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;
           }
@@ -2743,7 +2748,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return initialized;
   }
 
@@ -2762,7 +2767,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureExecutor.getEnvironment().setEventReady(serverCrashProcessingEnabled, b);
   }
 
-  public ProcedureEvent getServerCrashProcessingEnabledEvent() {
+  public ProcedureEvent<?> getServerCrashProcessingEnabledEvent() {
     return serverCrashProcessingEnabled;
   }
 
@@ -3313,54 +3318,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
@@ -3379,17 +3366,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
@@ -3542,10 +3523,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/4faf7dfc/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 1d6949c..700b363 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
@@ -72,6 +72,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/4faf7dfc/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 6b3c212..43df8b1 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.hadoop.hbase.shaded.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/4faf7dfc/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 43e75ed..5714cae 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,7 +16,6 @@
  * 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;
@@ -33,13 +32,15 @@ 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.hadoop.hbase.shaded.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.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;
+
 /**
  * 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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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/4faf7dfc/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 0799ca6..49489d8 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
@@ -148,6 +148,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;
@@ -178,7 +179,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
@@ -3742,7 +3742,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/4faf7dfc/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/4faf7dfc/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 413abe3..540a67c 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;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/4faf7dfc/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/17] 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/4b81af3d
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4b81af3d
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4b81af3d

Branch: refs/heads/HBASE-19397
Commit: 4b81af3d274fdaf035f37f7ebdf45ca28f0e677f
Parents: f6c697d
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 20 20:57:37 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:03:19 2017 +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/4b81af3d/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 700b363..9f71bab 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/4b81af3d/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() {


[06/17] 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/724ee671
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/724ee671
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/724ee671

Branch: refs/heads/HBASE-19397
Commit: 724ee671471d59042000174a806f44fa285a4bbd
Parents: 4b81af3
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Dec 20 16:43:38 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:03:19 2017 +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/724ee671/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 dd85f5c..5f4665c 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/724ee671/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/724ee671/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();
+      }
+    }
+  }
 }


[16/17] 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/7459c8cf
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7459c8cf
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7459c8cf

Branch: refs/heads/HBASE-19397
Commit: 7459c8cf5ad170503d6d8a6f3327416c737df582
Parents: e10313c
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 26 11:39:34 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Dec 26 21:07:55 2017 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/7459c8cf/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 01df2bd..da231e6 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
@@ -49,7 +49,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.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -330,7 +329,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() {
@@ -351,9 +349,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/7459c8cf/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/7459c8cf/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..ed9fe14
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
@@ -0,0 +1,170 @@
+/*
+ *
+ * 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/7459c8cf/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 5d051a0..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.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-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;
-
-@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/7459c8cf/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/7459c8cf/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 3581a05..70d72d4 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.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 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);
   }
 
@@ -420,7 +420,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);
     }
   }
 
@@ -437,7 +437,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       return false;
     }
 
-    ReplicationPeerZKImpl peer = null;
+    ReplicationPeerImpl peer = null;
     try {
       peer = createPeer(peerId);
     } catch (Exception e) {
@@ -446,8 +446,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 {
@@ -490,19 +489,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/7459c8cf/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/7459c8cf/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();
   }
 }