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/25 10:51:27 UTC

[01/22] hbase git commit: HBASE-19148 Reevaluate default values of configurations; minor ADDENDUM2 Fix a description. [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19397 557f5d4d8 -> 0e53efd00 (forced update)


HBASE-19148 Reevaluate default values of configurations; minor ADDENDUM2
Fix a description.


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

Branch: refs/heads/HBASE-19397
Commit: 8ed3d4b9cbf79088e5b9b80477d476cb84a68129
Parents: c0598dc
Author: Michael Stack <st...@apache.org>
Authored: Fri Dec 22 08:37:39 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Dec 22 08:37:39 2017 -0800

----------------------------------------------------------------------
 hbase-common/src/main/resources/hbase-default.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8ed3d4b9/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index b42ce3f..8758741 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -133,14 +133,14 @@ possible configurations would overwhelm and obscure the important.
   <property>
     <name>hbase.master.logcleaner.ttl</name>
     <value>600000</value>
-    <description>How long a WAL remain in the {hbase.rootdir}/oldWALs directory,
+    <description>How long a WAL remain in the archive ({hbase.rootdir}/oldWALs) directory,
     after which it will be cleaned by a Master thread. The value is in milliseconds.</description>
   </property>
   <property>
     <name>hbase.master.procedurewalcleaner.ttl</name>
     <value>604800000</value>
     <description>How long a Procedure WAL will remain in the
-    {hbase.rootdir}/MasterProcedureWALs directory, after which it will be cleaned
+    archive directory, after which it will be cleaned
     by a Master thread. The value is in milliseconds.</description>
   </property>
   <property>


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

Branch: refs/heads/HBASE-19397
Commit: 0e428210f2e8a3840f805fc98b54f4d4e17f005b
Parents: 8a90228
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Dec 20 16:43:38 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:28:54 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/0e428210/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/0e428210/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/0e428210/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();
+      }
+    }
+  }
 }


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

Branch: refs/heads/HBASE-19397
Commit: c2184942a8a2e44682a008e4079b06c2e87bf499
Parents: 0e42821
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 21 21:59:46 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:28:54 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/c2184942/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/c2184942/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/c2184942/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/c2184942/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/c2184942/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/c2184942/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/c2184942/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;
 


[08/22] hbase git commit: HBASE-19601 Fixed Checkstyle errors in hbase-rsgroup and enabled Checkstyle to fail on violations

Posted by zh...@apache.org.
HBASE-19601 Fixed Checkstyle errors in hbase-rsgroup and enabled Checkstyle to fail on violations


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

Branch: refs/heads/HBASE-19397
Commit: c24cf2d55ecd479c89b0613b6ebbdaba4eb793ad
Parents: ab6571e
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Sat Dec 23 00:52:08 2017 +0100
Committer: Jan Hentschel <ja...@ultratendency.com>
Committed: Sun Dec 24 12:58:50 2017 +0100

----------------------------------------------------------------------
 .../resources/hbase/checkstyle-suppressions.xml |  1 +
 hbase-rsgroup/pom.xml                           | 16 +++++++
 .../hadoop/hbase/rsgroup/RSGroupAdmin.java      |  4 +-
 .../hbase/rsgroup/RSGroupAdminClient.java       |  5 +-
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     | 17 ++++---
 .../hbase/rsgroup/RSGroupAdminServer.java       | 48 ++++++++++++--------
 .../hbase/rsgroup/RSGroupBasedLoadBalancer.java | 17 ++++---
 .../hbase/rsgroup/RSGroupInfoManager.java       |  3 +-
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java   |  7 +--
 .../hbase/rsgroup/RSGroupProtobufUtil.java      |  9 ++--
 .../hbase/rsgroup/RSGroupableBalancer.java      |  2 +-
 .../apache/hadoop/hbase/rsgroup/Utility.java    | 14 ++++--
 .../balancer/TestRSGroupBasedLoadBalancer.java  | 29 ++----------
 .../hadoop/hbase/rsgroup/TestRSGroups.java      |  5 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java  | 32 +++++++++----
 .../rsgroup/VerifyingRSGroupAdminClient.java    | 25 +++++-----
 16 files changed, 137 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
index 1ecae86..b9c9b21 100644
--- a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
+++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
@@ -35,4 +35,5 @@
   <suppress checks="." files=".*/generated-jamon/.*\.java"/>
   <suppress checks="MagicNumberCheck" files=".*/src/test/.*\.java"/>
   <suppress checks="VisibilityModifier" files=".*/src/test/.*\.java"/>
+  <suppress checks="InterfaceIsTypeCheck" files="RSGroupableBalancer.java"/>
 </suppressions>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml
index dba5810..16c198b 100644
--- a/hbase-rsgroup/pom.xml
+++ b/hbase-rsgroup/pom.xml
@@ -68,6 +68,22 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>checkstyle</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+            <configuration>
+              <failOnViolation>true</failOnViolation>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     <pluginManagement>
       <plugins>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java
index 453ef54..9ea996b 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java
@@ -22,8 +22,8 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.net.Address;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Group user API interface used between client and server.
@@ -84,7 +84,7 @@ public interface RSGroupAdmin {
    * @param servers set of servers to move
    * @param tables set of tables to move
    * @param targetGroup the target group name
-   * @throws IOException
+   * @throws IOException if moving the server and tables fail
    */
   void moveServersAndTables(Set<Address> servers, Set<TableName> tables,
                             String targetGroup) throws IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java
index be83a7b..e39fe76 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java
@@ -17,13 +17,14 @@
  */
 package org.apache.hadoop.hbase.rsgroup;
 
+import com.google.protobuf.ServiceException;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -44,9 +45,9 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RSGroupAdmi
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
-import com.google.protobuf.ServiceException;
 
 /**
  * Client used for managing region server group information.

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 7558840..4ee6a17 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.hbase.rsgroup;
 
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
@@ -26,10 +30,6 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -74,11 +74,12 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGro
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersResponse;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+
 // TODO: Encapsulate MasterObserver functions into separate subclass.
 @CoreCoprocessor
 @InterfaceAudience.Private
@@ -231,7 +232,8 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
     public void balanceRSGroup(RpcController controller,
         BalanceRSGroupRequest request, RpcCallback<BalanceRSGroupResponse> done) {
       BalanceRSGroupResponse.Builder builder = BalanceRSGroupResponse.newBuilder();
-      LOG.info(master.getClientIdAuditPrefix() + " balance rsgroup, group=" + request.getRSGroupName());
+      LOG.info(master.getClientIdAuditPrefix() + " balance rsgroup, group=" +
+              request.getRSGroupName());
       try {
         builder.setBalanceRan(groupAdminServer.balanceRSGroup(request.getRSGroupName()));
       } catch (IOException e) {
@@ -263,7 +265,8 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       try {
         Address hp = Address.fromParts(request.getServer().getHostName(),
             request.getServer().getPort());
-        LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server=" + hp);
+        LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server=" +
+                hp);
         RSGroupInfo RSGroupInfo = groupAdminServer.getRSGroupOfServer(hp);
         if (RSGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
index b4d35e3..df30709 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
@@ -43,12 +43,13 @@ import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
 import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+
 /**
  * Service to support Region Server Grouping (HBase-6721).
  */
@@ -116,7 +117,10 @@ public class RSGroupAdminServer implements RSGroupAdmin {
     LinkedList<RegionInfo> regions = new LinkedList<>();
     for (Map.Entry<RegionInfo, ServerName> el :
         master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
-      if (el.getValue() == null) continue;
+      if (el.getValue() == null) {
+        continue;
+      }
+
       if (el.getValue().getAddress().equals(server)) {
         addRegion(regions, el.getKey());
       }
@@ -133,17 +137,20 @@ public class RSGroupAdminServer implements RSGroupAdmin {
     // If meta, move it last otherwise other unassigns fail because meta is not
     // online for them to update state in. This is dodgy. Needs to be made more
     // robust. See TODO below.
-    if (hri.isMetaRegion()) regions.addLast(hri);
-    else regions.addFirst(hri);
+    if (hri.isMetaRegion()) {
+      regions.addLast(hri);
+    } else {
+      regions.addFirst(hri);
+    }
   }
 
   /**
    * Check servers and tables.
-   * Fail if nulls or if servers and tables not belong to the same group
+   *
    * @param servers servers to move
    * @param tables tables to move
    * @param targetGroupName target group name
-   * @throws IOException
+   * @throws IOException if nulls or if servers and tables not belong to the same group
    */
   private void checkServersAndTables(Set<Address> servers, Set<TableName> tables,
                                      String targetGroupName) throws IOException {
@@ -157,7 +164,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
     }
     RSGroupInfo srcGrp = new RSGroupInfo(tmpSrcGrp);
     if (srcGrp.getName().equals(targetGroupName)) {
-      throw new ConstraintException( "Target RSGroup " + targetGroupName +
+      throw new ConstraintException("Target RSGroup " + targetGroupName +
               " is same as source " + srcGrp.getName() + " RSGroup.");
     }
     // Only move online servers
@@ -181,8 +188,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
       }
     }
 
-    if (srcGrp.getServers().size() <= servers.size()
-            && srcGrp.getTables().size() > tables.size() ) {
+    if (srcGrp.getServers().size() <= servers.size() && srcGrp.getTables().size() > tables.size()) {
       throw new ConstraintException("Cannot leave a RSGroup " + srcGrp.getName() +
               " that contains tables without servers to host them.");
     }
@@ -194,7 +200,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
    * @param servers the servers that will move to new group
    * @param tables these tables will be kept on the servers, others will be moved
    * @param targetGroupName the target group name
-   * @throws IOException
+   * @throws IOException if moving the server and tables fail
    */
   private void moveRegionsFromServers(Set<Address> servers, Set<TableName> tables,
       String targetGroupName) throws IOException {
@@ -244,18 +250,19 @@ public class RSGroupAdminServer implements RSGroupAdmin {
   /**
    * Moves every region of tables which should be kept on the servers,
    * but currently they are located on other servers.
-   * @param servers the regions of these servers will be kept on the servers,
-   * others will be moved
+   * @param servers the regions of these servers will be kept on the servers, others will be moved
    * @param tables the tables that will move to new group
    * @param targetGroupName the target group name
-   * @throws IOException
+   * @throws IOException if moving the region fails
    */
   private void moveRegionsToServers(Set<Address> servers, Set<TableName> tables,
       String targetGroupName) throws IOException {
     for (TableName table: tables) {
       LOG.info("Moving region(s) from " + table + " for table move to " + targetGroupName);
-      for (RegionInfo region : master.getAssignmentManager().getRegionStates().getRegionsOfTable(table)) {
-        ServerName sn = master.getAssignmentManager().getRegionStates().getRegionServerOfRegion(region);
+      for (RegionInfo region : master.getAssignmentManager().getRegionStates()
+              .getRegionsOfTable(table)) {
+        ServerName sn = master.getAssignmentManager().getRegionStates()
+                .getRegionServerOfRegion(region);
         if (!servers.contains(sn.getAddress())) {
           master.getAssignmentManager().move(region);
         }
@@ -294,7 +301,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
             + " does not exist.");
       }
       if (srcGrp.getName().equals(targetGroupName)) {
-        throw new ConstraintException( "Target RSGroup " + targetGroupName +
+        throw new ConstraintException("Target RSGroup " + targetGroupName +
             " is same as source " + srcGrp + " RSGroup.");
       }
       // Only move online servers (when moving from 'default') or servers from other
@@ -482,7 +489,10 @@ public class RSGroupAdminServer implements RSGroupAdmin {
 
     synchronized (balancer) {
       // If balance not true, don't run balancer.
-      if (!((HMaster) master).isBalancerOn()) return false;
+      if (!((HMaster) master).isBalancerOn()) {
+        return false;
+      }
+
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preBalanceRSGroup(groupName);
       }
@@ -547,7 +557,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
   @Override
   public void moveServersAndTables(Set<Address> servers, Set<TableName> tables, String targetGroup)
       throws IOException {
-    if (servers == null || servers.isEmpty() ) {
+    if (servers == null || servers.isEmpty()) {
       throw new ConstraintException("The list of servers to move cannot be null or empty.");
     }
     if (tables == null || tables.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
index a228a76..9579257 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
@@ -43,16 +43,17 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
 import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.util.ReflectionUtils;
+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.collect.ArrayListMultimap;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.LinkedListMultimap;
 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.collect.Maps;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * GroupBasedLoadBalancer, used when Region Server Grouping is configured (HBase-6721)
@@ -324,8 +325,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
   }
 
   private ServerName findServerForRegion(
-      Map<ServerName, List<RegionInfo>> existingAssignments, RegionInfo region)
-  {
+      Map<ServerName, List<RegionInfo>> existingAssignments, RegionInfo region) {
     for (Map.Entry<ServerName, List<RegionInfo>> entry : existingAssignments.entrySet()) {
       if (entry.getValue().contains(region)) {
         return entry.getKey();
@@ -392,7 +392,10 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
   }
 
   public boolean isOnline() {
-    if (this.rsGroupInfoManager == null) return false;
+    if (this.rsGroupInfoManager == null) {
+      return false;
+    }
+
     return this.rsGroupInfoManager.isOnline();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
index 5eee236..77aefbd 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
@@ -18,16 +18,15 @@
 
 package org.apache.hadoop.hbase.rsgroup;
 
-
 import java.io.IOException;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.net.Address;
 
 /**
  * Interface used to manage RSGroupInfo storage. An implementation

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 67dfde7..56867bf 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hbase.rsgroup;
 
+import com.google.protobuf.ServiceException;
+
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -81,14 +83,13 @@ 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.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * This is an implementation of {@link RSGroupInfoManager} which makes
  * use of an HBase table as the persistence store for the group information.
@@ -114,7 +115,7 @@ import com.google.protobuf.ServiceException;
  * no other has access concurrently. Reads must be able to continue concurrently.
  */
 @InterfaceAudience.Private
-class RSGroupInfoManagerImpl implements RSGroupInfoManager {
+final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
   private static final Logger LOG = LoggerFactory.getLogger(RSGroupInfoManagerImpl.class);
 
   /** Table descriptor for <code>hbase:rsgroup</code> catalog table */

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupProtobufUtil.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupProtobufUtil.java
index 30f6427..56e35e7 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupProtobufUtil.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupProtobufUtil.java
@@ -21,15 +21,18 @@ package org.apache.hadoop.hbase.rsgroup;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
+import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
-class RSGroupProtobufUtil {
+final class RSGroupProtobufUtil {
+  private RSGroupProtobufUtil() {
+  }
+
   static RSGroupInfo toGroupInfo(RSGroupProtos.RSGroupInfo proto) {
     RSGroupInfo RSGroupInfo = new RSGroupInfo(proto.getName());
     for(HBaseProtos.ServerName el: proto.getServersList()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupableBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupableBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupableBalancer.java
index 90f45ac..d091b3c 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupableBalancer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupableBalancer.java
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.hbase.rsgroup;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Marker Interface. RSGroups feature will check for a LoadBalancer

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java
index f0c6998..4d7bf43 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/Utility.java
@@ -23,22 +23,28 @@ import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.net.Address;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Utility for this RSGroup package in hbase-rsgroup.
  */
 @InterfaceAudience.Private
-class Utility {
+final class Utility {
+  private Utility() {
+  }
+
   /**
-   * @param master
+   * @param master the master to get online servers for
    * @return Set of online Servers named for their hostname and port (not ServerName).
    */
   static Set<Address> getOnlineServers(final MasterServices master) {
     Set<Address> onlineServers = new HashSet<Address>();
-    if (master == null) return onlineServers;
+    if (master == null) {
+      return onlineServers;
+    }
+
     for(ServerName server: master.getServerManager().getOnlineServers().keySet()) {
       onlineServers.add(server.getAddress());
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
index 797022c..da74e7d 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
@@ -74,8 +74,7 @@ public class TestRSGroupBasedLoadBalancer {
   private static RSGroupBasedLoadBalancer loadBalancer;
   private static SecureRandom rand;
 
-  static String[]  groups = new String[] { RSGroupInfo.DEFAULT_GROUP, "dg2", "dg3",
-      "dg4" };
+  static String[]  groups = new String[] { RSGroupInfo.DEFAULT_GROUP, "dg2", "dg3", "dg4" };
   static TableName[] tables =
       new TableName[] { TableName.valueOf("dt1"),
           TableName.valueOf("dt2"),
@@ -110,8 +109,6 @@ public class TestRSGroupBasedLoadBalancer {
    *
    * Invariant is that all servers of the group should be hosting either floor(average) or
    * ceiling(average)
-   *
-   * @throws Exception
    */
   @Test
   public void testBalanceCluster() throws Exception {
@@ -163,12 +160,6 @@ public class TestRSGroupBasedLoadBalancer {
 
   /**
    * All regions have an assignment.
-   *
-   * @param regions
-   * @param servers
-   * @param assignments
-   * @throws java.io.IOException
-   * @throws java.io.FileNotFoundException
    */
   private void assertImmediateAssignment(List<RegionInfo> regions,
                                          List<ServerName> servers,
@@ -193,8 +184,6 @@ public class TestRSGroupBasedLoadBalancer {
    * Round-robin. Should yield a balanced cluster so same invariant as the
    * load balancer holds, all servers holding either floor(avg) or
    * ceiling(avg).
-   *
-   * @throws Exception
    */
   @Test
   public void testBulkAssignment() throws Exception {
@@ -226,10 +215,7 @@ public class TestRSGroupBasedLoadBalancer {
   }
 
   /**
-   * Test the cluster startup bulk assignment which attempts to retain
-   * assignment info.
-   *
-   * @throws Exception
+   * Test the cluster startup bulk assignment which attempts to retain assignment info.
    */
   @Test
   public void testRetainAssignment() throws Exception {
@@ -249,8 +235,7 @@ public class TestRSGroupBasedLoadBalancer {
   }
 
   /**
-   * Test BOGUS_SERVER_NAME among groups do not overwrite each other
-   * @throws Exception
+   * Test BOGUS_SERVER_NAME among groups do not overwrite each other.
    */
   @Test
   public void testRoundRobinAssignment() throws Exception {
@@ -288,11 +273,6 @@ public class TestRSGroupBasedLoadBalancer {
    * <li>If a region had an existing assignment to a server with the same
    * address a a currently online server, it will be assigned to it
    * </ul>
-   *
-   * @param existing
-   * @param assignment
-   * @throws java.io.IOException
-   * @throws java.io.FileNotFoundException
    */
   private void assertRetainedAssignment(
       Map<RegionInfo, ServerName> existing, List<ServerName> servers,
@@ -305,8 +285,9 @@ public class TestRSGroupBasedLoadBalancer {
       assertTrue(
           "Region assigned to server that was not listed as online",
           onlineServerSet.contains(a.getKey()));
-      for (RegionInfo r : a.getValue())
+      for (RegionInfo r : a.getValue()) {
         assignedRegions.add(r);
+      }
     }
     assertEquals(existing.size(), assignedRegions.size());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
index 378c1ab..cbd862d 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
@@ -40,8 +40,6 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
@@ -53,7 +51,10 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 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.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 
 @Category({MediumTests.class})
 public class TestRSGroups extends TestRSGroupsBase {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index e4386bc..7fa264b 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -343,8 +343,10 @@ public abstract class TestRSGroupsBase {
       @Override
       public boolean evaluate() throws Exception {
         List<String> regions = getTableRegionMap().get(tableName);
-        if (regions == null)
+        if (regions == null) {
           return false;
+        }
+
         return getTableRegionMap().get(tableName).size() >= 5;
       }
     });
@@ -476,8 +478,10 @@ public abstract class TestRSGroupsBase {
       @Override
       public boolean evaluate() throws Exception {
         List<String> regions = getTableRegionMap().get(tableName);
-        if (regions == null)
+        if (regions == null) {
           return false;
+        }
+
         return getTableRegionMap().get(tableName).size() >= 6;
       }
     });
@@ -516,8 +520,8 @@ public abstract class TestRSGroupsBase {
     });
 
     // Lets move this region to the new group.
-    TEST_UTIL.getAdmin().move(Bytes.toBytes(RegionInfo.encodeRegionName(Bytes.toBytes(targetRegion))),
-        Bytes.toBytes(targetServer.getServerName()));
+    TEST_UTIL.getAdmin().move(Bytes.toBytes(RegionInfo.encodeRegionName(
+            Bytes.toBytes(targetRegion))), Bytes.toBytes(targetServer.getServerName()));
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
@@ -676,11 +680,14 @@ public abstract class TestRSGroupsBase {
       @Override
       public boolean evaluate() throws Exception {
         List<String> regionsA = getTableRegionMap().get(tableNameA);
-        if (regionsA == null)
+        if (regionsA == null) {
           return false;
+        }
+
         List<String> regionsB = getTableRegionMap().get(tableNameB);
-        if (regionsB == null)
+        if (regionsB == null) {
           return false;
+        }
 
         return getTableRegionMap().get(tableNameA).size() >= 1
                 && getTableRegionMap().get(tableNameB).size() >= 1;
@@ -704,7 +711,8 @@ public abstract class TestRSGroupsBase {
             rsGroupAdmin.getRSGroupInfoOfTable(tableNameB).getName());
 
     //verify tables' not exist in old group
-    Set<TableName> DefaultTables = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables();
+    Set<TableName> DefaultTables = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP)
+            .getTables();
     assertFalse(DefaultTables.contains(tableNameA));
     assertFalse(DefaultTables.contains(tableNameB));
 
@@ -758,8 +766,10 @@ public abstract class TestRSGroupsBase {
       @Override
       public boolean evaluate() throws Exception {
         List<String> regions = getTableRegionMap().get(tableName);
-        if (regions == null)
+        if (regions == null) {
           return false;
+        }
+
         return getTableRegionMap().get(tableName).size() >= 5;
       }
     });
@@ -847,7 +857,8 @@ public abstract class TestRSGroupsBase {
             rsGroupAdmin.getRSGroupInfoOfTable(tableName).getName());
 
     //verify servers' not exist in old group
-    Set<Address> defaultServers = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers();
+    Set<Address> defaultServers = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP)
+            .getServers();
     assertFalse(defaultServers.contains(targetServer.getAddress()));
 
     //verify servers' exist in new group
@@ -855,7 +866,8 @@ public abstract class TestRSGroupsBase {
     assertTrue(newGroupServers.contains(targetServer.getAddress()));
 
     //verify tables' not exist in old group
-    Set<TableName> defaultTables = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables();
+    Set<TableName> defaultTables = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP)
+            .getTables();
     assertFalse(defaultTables.contains(tableName));
 
     //verify tables' exist in new group

http://git-wip-us.apache.org/repos/asf/hbase/blob/c24cf2d5/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
index 0a26a35..5a466a7 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
@@ -17,12 +17,14 @@
  */
 package org.apache.hadoop.hbase.rsgroup;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+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.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
@@ -32,15 +34,14 @@ import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Assert;
 
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 
 @InterfaceAudience.Private
 public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
@@ -51,7 +52,8 @@ public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
   public VerifyingRSGroupAdminClient(RSGroupAdmin RSGroupAdmin, Configuration conf)
       throws IOException {
     wrapped = RSGroupAdmin;
-    table = ConnectionFactory.createConnection(conf).getTable(RSGroupInfoManager.RSGROUP_TABLE_NAME);
+    table = ConnectionFactory.createConnection(conf)
+            .getTable(RSGroupInfoManager.RSGROUP_TABLE_NAME);
     zkw = new ZKWatcher(conf, this.getClass().getSimpleName(), null);
   }
 
@@ -105,7 +107,8 @@ public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
   }
 
   @Override
-  public void moveServersAndTables(Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException {
+  public void moveServersAndTables(Set<Address> servers, Set<TableName> tables, String targetGroup)
+          throws IOException {
     wrapped.moveServersAndTables(servers, tables, targetGroup);
     verify();
   }


[06/22] hbase git commit: HBASE-19602 Cleanup the usage of ReplicationAdmin from document

Posted by zh...@apache.org.
HBASE-19602 Cleanup the usage of ReplicationAdmin from document


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

Branch: refs/heads/HBASE-19397
Commit: 2f255894223abb51b6f9d5c680e226b25c6d788a
Parents: 0dadb45
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Dec 23 10:25:08 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Sat Dec 23 18:55:54 2017 +0800

----------------------------------------------------------------------
 hbase-common/src/main/resources/hbase-default.xml                | 2 +-
 .../org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java    | 2 +-
 src/main/asciidoc/_chapters/architecture.adoc                    | 4 ++--
 src/main/asciidoc/_chapters/hbase-default.adoc                   | 2 +-
 4 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2f255894/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 8758741..8a34bad 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1604,7 +1604,7 @@ possible configurations would overwhelm and obscure the important.
       If this is enabled, a replication peer named "region_replica_replication" will be created
       which will tail the logs and replicate the mutations to region replicas for tables that
       have region replication > 1. If this is enabled once, disabling this replication also
-      requires disabling the replication peer using shell or ReplicationAdmin java class.
+      requires disabling the replication peer using shell or Admin java class.
       Replication to secondary region replicas works over standard inter-cluster replication.
     </description>
   </property>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f255894/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
index fe514d8..8a43d17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
@@ -50,7 +50,7 @@ public class ServerRegionReplicaUtil extends RegionReplicaUtil {
    * If this is enabled, a replication peer named "region_replica_replication" will be created
    * which will tail the logs and replicate the mutatations to region replicas for tables that
    * have region replication &gt; 1. If this is enabled once, disabling this replication also
-   * requires disabling the replication peer using shell or ReplicationAdmin java class.
+   * requires disabling the replication peer using shell or {@link Admin} java class.
    * Replication to secondary region replicas works over standard inter-cluster replication.·
    */
   public static final String REGION_REPLICA_REPLICATION_CONF_KEY

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f255894/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 9a3cbd9..9091d5e 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -2505,7 +2505,7 @@ Committing writes in the same order as in primary ensures that the secondaries w
 Async WAL Replication is *disabled* by default. You can enable this feature by setting `hbase.region.replica.replication.enabled` to `true`.
 Asyn WAL Replication feature will add a new replication peer named `region_replica_replication` as a replication peer when you create a table with region replication > 1 for the first time. Once enabled, if you want to disable this feature, you need to do two actions:
 * Set configuration property `hbase.region.replica.replication.enabled` to false in `hbase-site.xml` (see Configuration section below)
-* Disable the replication peer named `region_replica_replication` in the cluster using hbase shell or `ReplicationAdmin` class:
+* Disable the replication peer named `region_replica_replication` in the cluster using hbase shell or `Admin` class:
 [source,bourne]
 ----
 	hbase> disable_peer 'region_replica_replication'
@@ -2558,7 +2558,7 @@ Instead you can change the number of region replicas per table to increase or de
     <name>hbase.region.replica.replication.enabled</name>
     <value>true</value>
     <description>
-      Whether asynchronous WAL replication to the secondary region replicas is enabled or not. If this is enabled, a replication peer named "region_replica_replication" will be created which will tail the logs and replicate the mutations to region replicas for tables that have region replication > 1. If this is enabled once, disabling this replication also      requires disabling the replication peer using shell or ReplicationAdmin java class. Replication to secondary region replicas works over standard inter-cluster replication.
+      Whether asynchronous WAL replication to the secondary region replicas is enabled or not. If this is enabled, a replication peer named "region_replica_replication" will be created which will tail the logs and replicate the mutations to region replicas for tables that have region replication > 1. If this is enabled once, disabling this replication also requires disabling the replication peer using shell or Admin java class. Replication to secondary region replicas works over standard inter-cluster replication.
     </description>
 </property>
 <property>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f255894/src/main/asciidoc/_chapters/hbase-default.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/hbase-default.adoc b/src/main/asciidoc/_chapters/hbase-default.adoc
index 9b3cfb7..7798657 100644
--- a/src/main/asciidoc/_chapters/hbase-default.adoc
+++ b/src/main/asciidoc/_chapters/hbase-default.adoc
@@ -2033,7 +2033,7 @@ A comma-separated list of
       If this is enabled, a replication peer named "region_replica_replication" will be created
       which will tail the logs and replicate the mutations to region replicas for tables that
       have region replication > 1. If this is enabled once, disabling this replication also
-      requires disabling the replication peer using shell or ReplicationAdmin java class.
+      requires disabling the replication peer using shell or Admin java class.
       Replication to secondary region replicas works over standard inter-cluster replication.
 
 


[04/22] hbase git commit: Add Yi Liang to pom.xml

Posted by zh...@apache.org.
Add Yi Liang to pom.xml


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

Branch: refs/heads/HBASE-19397
Commit: 4a9c298a5038cc71a34226722be4114b1ccbad15
Parents: 6007725
Author: Yi Liang <ea...@gmail.com>
Authored: Fri Dec 22 21:08:35 2017 -0800
Committer: Yi Liang <ea...@gmail.com>
Committed: Fri Dec 22 21:08:35 2017 -0800

----------------------------------------------------------------------
 pom.xml | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4a9c298a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e029b47..53b17d3 100755
--- a/pom.xml
+++ b/pom.xml
@@ -566,12 +566,18 @@
       <email>zjushch@apache.org</email>
       <timezone>+8</timezone>
     </developer>
-     <developer>
+    <developer>
       <id>churro</id>
       <name>Rahul Gidwani</name>
       <email>churro@apache.org</email>
       <timezone>-8</timezone>
     </developer>
+    <developer>
+      <id>yiliang</id>
+      <name>Yi Liang</name>
+      <email>yiliang@apache.org</email>
+      <timezone>-8</timezone>
+    </developer>
   </developers>
   <build>
     <extensions>


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

Branch: refs/heads/HBASE-19397
Commit: 8a902289c705259467582d6ebe76d3be5b1ec66a
Parents: d56366c
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 20 20:57:37 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:28:54 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/8a902289/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/8a902289/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() {


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

Branch: refs/heads/HBASE-19397
Commit: 0e53efd00e18000832326bdf6f21072d391be060
Parents: beab0c0
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 25 18:49:56 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:49:56 2017 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |  20 +-
 .../replication/ReplicationPeersZKImpl.java     |  24 +-
 .../replication/ReplicationQueueStorage.java    |  26 +-
 .../replication/ReplicationQueuesClient.java    |  93 ----
 .../ReplicationQueuesClientArguments.java       |  40 --
 .../ReplicationQueuesClientZKImpl.java          | 176 -------
 .../TableBasedReplicationQueuesClientImpl.java  | 113 -----
 .../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     |  22 +-
 .../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 --------------
 .../TestReplicationStateHBaseImpl.java          | 495 -------------------
 .../replication/TestReplicationStateZKImpl.java | 227 ---------
 .../replication/TestReplicationTableBase.java   | 109 ----
 .../TestReplicationSourceManagerZkImpl.java     |  84 +---
 ...tTableBasedReplicationSourceManagerImpl.java |  62 ---
 29 files changed, 913 insertions(+), 2088 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0e53efd0/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..07111c6 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,7 +31,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static final Class defaultReplicationQueueClass = ReplicationQueuesZKImpl.class;
+  public static final Class<?> defaultReplicationQueueClass = ReplicationQueuesZKImpl.class;
 
   public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args)
       throws Exception {
@@ -40,22 +40,14 @@ public class ReplicationFactory {
     return (ReplicationQueues) ConstructorUtils.invokeConstructor(classToBuild, 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 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/0e53efd0/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 02cf698..a7e50af 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -33,10 +33,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.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -50,6 +50,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
@@ -81,17 +83,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
@@ -514,14 +516,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)) {
@@ -532,7 +536,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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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..291980e 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
@@ -19,10 +19,12 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.regex.Pattern;
@@ -36,11 +38,13 @@ 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;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -168,7 +172,7 @@ public final class ReplicationPeerManager {
       // 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 +209,17 @@ 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());
-
+    Map<String, String> conf = new HashMap<>();
+    conf.putAll(oldPeerConfig.getConfiguration());
+    conf.putAll(peerConfig.getConfiguration());
+    Map<byte[], byte[]> peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    peerData.putAll(oldPeerConfig.getPeerData());
+    peerData.putAll(peerConfig.getPeerData());
+    ReplicationPeerConfig newPeerConfig =
+      newPeerConfigBuilder.setConfiguration(conf).setPeerData(peerData).build();
     peerStorage.updatePeerConfig(peerId, newPeerConfig);
     peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e53efd0/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;
-    }
-  }
 }


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

Branch: refs/heads/HBASE-19397
Commit: 71e9d950d92a1b027460f4dbe7ec3bf65b817dd2
Parents: c218494
Author: huzheng <op...@gmail.com>
Authored: Wed Dec 20 10:47:18 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:28:54 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 ++-
 ...tTableBasedReplicationSourceManagerImpl.java |   7 +-
 23 files changed, 536 insertions(+), 476 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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 011142e..02cf698 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
@@ -503,21 +503,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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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..a9844f3 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
@@ -763,4 +763,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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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/71e9d950/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 {

http://git-wip-us.apache.org/repos/asf/hbase/blob/71e9d950/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
index 19457e2..9ff1865 100644
--- 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
@@ -53,10 +53,9 @@ public class TestTableBasedReplicationSourceManagerImpl extends TestReplicationS
       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"));
+    Waiter.waitFor(conf, 3 * 1000, () -> utility.getMiniHBaseCluster().getMaster().isInitialized());
+    utility.waitUntilAllRegionsAssigned(
+      TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication"));
     setupZkAndReplication();
   }
 


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

Branch: refs/heads/HBASE-19397
Commit: aab7747f13f839a23195cb8e7ac66aa3e85f215b
Parents: df22d88
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 18 15:22:36 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:28:54 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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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..011142e 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
@@ -534,7 +534,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);
           }
         }
@@ -542,7 +542,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/aab7747f/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 52b7b35..024a6ff 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;
@@ -170,7 +180,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;
@@ -331,15 +340,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;
@@ -1198,7 +1207,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));
@@ -2318,11 +2326,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;
           }
@@ -2748,7 +2753,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return initialized;
   }
 
@@ -2767,7 +2772,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureExecutor.getEnvironment().setEventReady(serverCrashProcessingEnabled, b);
   }
 
-  public ProcedureEvent getServerCrashProcessingEnabledEvent() {
+  public ProcedureEvent<?> getServerCrashProcessingEnabledEvent() {
     return serverCrashProcessingEnabled;
   }
 
@@ -3318,54 +3323,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
@@ -3384,17 +3371,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
@@ -3547,10 +3528,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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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/aab7747f/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) {
   }
 
 }


[07/22] hbase git commit: HBASE-19605 Fixed Checkstyle errors in hbase-metrics-api and enabled Checkstyle to fail on violations

Posted by zh...@apache.org.
HBASE-19605 Fixed Checkstyle errors in hbase-metrics-api and enabled Checkstyle to fail on violations


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

Branch: refs/heads/HBASE-19397
Commit: ab6571ee7ae8250b2e4d37e62d47b8255c49e258
Parents: 2f25589
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Sat Dec 23 17:51:22 2017 +0100
Committer: Jan Hentschel <ja...@ultratendency.com>
Committed: Sun Dec 24 00:43:47 2017 +0100

----------------------------------------------------------------------
 hbase-metrics-api/pom.xml                           | 16 ++++++++++++++++
 .../hbase/metrics/MetricRegistriesLoader.java       |  9 +++++----
 .../org/apache/hadoop/hbase/metrics/MetricSet.java  | 12 ++++++------
 3 files changed, 27 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ab6571ee/hbase-metrics-api/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/pom.xml b/hbase-metrics-api/pom.xml
index 5e13849..f2a89af 100644
--- a/hbase-metrics-api/pom.xml
+++ b/hbase-metrics-api/pom.xml
@@ -68,6 +68,22 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>checkstyle</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+            <configuration>
+              <failOnViolation>true</failOnViolation>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab6571ee/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
index d398c25..f084d99 100644
--- a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
@@ -17,28 +17,29 @@
  * limitations under the License.
  */
 
-
 package org.apache.hadoop.hbase.metrics;
 
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.ServiceLoader;
 
+import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 @InterfaceAudience.Private
-public class MetricRegistriesLoader {
+public final class MetricRegistriesLoader {
   private static final Logger LOG = LoggerFactory.getLogger(MetricRegistries.class);
 
   private static final String defaultClass
       = "org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl";
 
+  private MetricRegistriesLoader() {
+  }
+
   /**
    * Creates a {@link MetricRegistries} instance using the corresponding {@link MetricRegistries}
    * available to {@link ServiceLoader} on the classpath. If no instance is found, then default

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab6571ee/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
index 41ddcee..5e1c873 100644
--- a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
@@ -32,10 +32,10 @@ import org.apache.yetus.audience.InterfaceStability;
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 @InterfaceStability.Evolving
 public interface MetricSet extends Metric {
-    /**
-     * A map of metric names to metrics.
-     *
-     * @return the metrics
-     */
-    Map<String, Metric> getMetrics();
+  /**
+   * A map of metric names to metrics.
+   *
+   * @return the metrics
+   */
+  Map<String, Metric> getMetrics();
 }


[02/22] hbase git commit: HBASE-19514 TestJMXListener now uses random port

Posted by zh...@apache.org.
HBASE-19514 TestJMXListener now uses random port

Signed-off-by: Jan Hentschel <ja...@ultratendency.com>


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

Branch: refs/heads/HBASE-19397
Commit: c0c802f62b66a707c704bc945f951debbc57b691
Parents: 8ed3d4b
Author: tedyu <yu...@gmail.com>
Authored: Fri Dec 22 18:40:03 2017 +0100
Committer: Jan Hentschel <ja...@ultratendency.com>
Committed: Fri Dec 22 18:40:58 2017 +0100

----------------------------------------------------------------------
 .../src/test/java/org/apache/hadoop/hbase/TestJMXListener.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c0c802f6/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java
index 520294a..4c7f129 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXListener.java
@@ -46,7 +46,7 @@ import org.slf4j.LoggerFactory;
 public class TestJMXListener {
   private static final Logger LOG = LoggerFactory.getLogger(TestJMXListener.class);
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static int connectorPort = 61120;
+  private static int connectorPort = UTIL.randomFreePort();
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {


[05/22] hbase git commit: HBASE-19576 Introduce builder for ReplicationPeerConfig and make it immutable

Posted by zh...@apache.org.
HBASE-19576 Introduce builder for ReplicationPeerConfig and make it immutable


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

Branch: refs/heads/HBASE-19397
Commit: 0dadb45ad7db849d4cd56f64a2a054250909c68b
Parents: 4a9c298
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Dec 22 11:42:40 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Sat Dec 23 18:55:29 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |   4 +-
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |   4 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   5 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  19 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  17 +-
 .../client/replication/ReplicationAdmin.java    |  15 +-
 .../replication/ReplicationPeerConfigUtil.java  |  93 ++++++----
 .../replication/ReplicationPeerConfig.java      | 181 ++++++++++++++++++-
 .../ReplicationPeerConfigBuilder.java           |  64 +++++++
 .../replication/ReplicationPeersZKImpl.java     |  21 ++-
 .../client/TestAsyncReplicationAdminApi.java    |  17 +-
 .../replication/TestReplicationAdmin.java       |  19 +-
 12 files changed, 383 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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 de022d3..ff2722e 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
@@ -2527,7 +2527,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   void appendReplicationPeerTableCFs(String id,
-      Map<TableName, ? extends Collection<String>> tableCfs)
+      Map<TableName, List<String>> tableCfs)
       throws ReplicationException, IOException;
 
   /**
@@ -2538,7 +2538,7 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   void removeReplicationPeerTableCFs(String id,
-      Map<TableName, ? extends Collection<String>> tableCfs)
+      Map<TableName, List<String>> tableCfs)
       throws ReplicationException, IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index c01d5fa..e976033 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -597,7 +597,7 @@ public interface AsyncAdmin {
    * @param tableCfs A map from tableName to column family names
    */
   CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
-      Map<TableName, ? extends Collection<String>> tableCfs);
+      Map<TableName, List<String>> tableCfs);
 
   /**
    * Remove some table-cfs from config of the specified peer
@@ -605,7 +605,7 @@ public interface AsyncAdmin {
    * @param tableCfs A map from tableName to column family names
    */
   CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId,
-      Map<TableName, ? extends Collection<String>> tableCfs);
+      Map<TableName, List<String>> tableCfs);
 
   /**
    * Return a list of replication peers.

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 09fdeff..e60e422 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.RpcChannel;
-import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -409,13 +408,13 @@ class AsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
-      Map<TableName, ? extends Collection<String>> tableCfs) {
+      Map<TableName, List<String>> tableCfs) {
     return wrap(rawAdmin.appendReplicationPeerTableCFs(peerId, tableCfs));
   }
 
   @Override
   public CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId,
-      Map<TableName, ? extends Collection<String>> tableCfs) {
+      Map<TableName, List<String>> tableCfs) {
     return wrap(rawAdmin.removeReplicationPeerTableCFs(peerId, tableCfs));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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 bbcc825..af3916d 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
@@ -26,7 +26,6 @@ import java.io.InterruptedIOException;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -3926,26 +3925,28 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void appendReplicationPeerTableCFs(String id,
-      Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException,
-      IOException {
+      Map<TableName, List<String>> tableCfs)
+      throws ReplicationException, IOException {
     if (tableCfs == null) {
       throw new ReplicationException("tableCfs is null");
     }
     ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
-    ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
-    updateReplicationPeerConfig(id, peerConfig);
+    ReplicationPeerConfig newPeerConfig =
+        ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
+    updateReplicationPeerConfig(id, newPeerConfig);
   }
 
   @Override
   public void removeReplicationPeerTableCFs(String id,
-      Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException,
-      IOException {
+      Map<TableName, List<String>> tableCfs)
+      throws ReplicationException, IOException {
     if (tableCfs == null) {
       throw new ReplicationException("tableCfs is null");
     }
     ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
-    ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
-    updateReplicationPeerConfig(id, peerConfig);
+    ReplicationPeerConfig newPeerConfig =
+        ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
+    updateReplicationPeerConfig(id, newPeerConfig);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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 5e1c654..dac83f3 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
@@ -26,7 +26,6 @@ import com.google.protobuf.RpcChannel;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -1593,7 +1592,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<Void> appendReplicationPeerTableCFs(String id,
-      Map<TableName, ? extends Collection<String>> tableCfs) {
+      Map<TableName, List<String>> tableCfs) {
     if (tableCfs == null) {
       return failedFuture(new ReplicationException("tableCfs is null"));
     }
@@ -1601,8 +1600,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     CompletableFuture<Void> future = new CompletableFuture<Void>();
     getReplicationPeerConfig(id).whenComplete((peerConfig, error) -> {
       if (!completeExceptionally(future, error)) {
-        ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
-        updateReplicationPeerConfig(id, peerConfig).whenComplete((result, err) -> {
+        ReplicationPeerConfig newPeerConfig =
+            ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
+        updateReplicationPeerConfig(id, newPeerConfig).whenComplete((result, err) -> {
           if (!completeExceptionally(future, error)) {
             future.complete(result);
           }
@@ -1614,7 +1614,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<Void> removeReplicationPeerTableCFs(String id,
-      Map<TableName, ? extends Collection<String>> tableCfs) {
+      Map<TableName, List<String>> tableCfs) {
     if (tableCfs == null) {
       return failedFuture(new ReplicationException("tableCfs is null"));
     }
@@ -1623,14 +1623,15 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     getReplicationPeerConfig(id).whenComplete(
       (peerConfig, error) -> {
         if (!completeExceptionally(future, error)) {
+          ReplicationPeerConfig newPeerConfig = null;
           try {
-            ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig,
-              id);
+            newPeerConfig = ReplicationPeerConfigUtil
+                .removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
           } catch (ReplicationException e) {
             future.completeExceptionally(e);
             return;
           }
-          updateReplicationPeerConfig(id, peerConfig).whenComplete((result, err) -> {
+          updateReplicationPeerConfig(id, newPeerConfig).whenComplete((result, err) -> {
             if (!completeExceptionally(future, error)) {
               future.complete(result);
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index 78d4fbb..ccfc444 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.TreeMap;
 import java.util.regex.Pattern;
 
+import org.apache.curator.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
@@ -251,7 +252,7 @@ public class ReplicationAdmin implements Closeable {
   @Deprecated
   public void appendPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
       throws ReplicationException, IOException {
-    this.admin.appendReplicationPeerTableCFs(id, tableCfs);
+    this.admin.appendReplicationPeerTableCFs(id, copyTableCFs(tableCfs));
   }
 
   /**
@@ -279,7 +280,17 @@ public class ReplicationAdmin implements Closeable {
   @Deprecated
   public void removePeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
       throws ReplicationException, IOException {
-    this.admin.removeReplicationPeerTableCFs(id, tableCfs);
+    this.admin.removeReplicationPeerTableCFs(id, copyTableCFs(tableCfs));
+  }
+
+  private Map<TableName, List<String>>
+      copyTableCFs(Map<TableName, ? extends Collection<String>> tableCfs) {
+    Map<TableName, List<String>> newTableCfs = new HashMap<>();
+    if (tableCfs != null) {
+      tableCfs.forEach(
+        (table, cfs) -> newTableCfs.put(table, cfs != null ? Lists.newArrayList(cfs) : null));
+    }
+    return newTableCfs;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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 ec80eca..a50d48f 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,6 +26,7 @@ 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;
@@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
@@ -260,63 +262,66 @@ public final class ReplicationPeerConfigUtil {
       return convert(peer);
     } else {
       if (bytes.length > 0) {
-        return new ReplicationPeerConfig().setClusterKey(Bytes.toString(bytes));
+        return ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build();
       }
-      return new ReplicationPeerConfig().setClusterKey("");
+      return ReplicationPeerConfig.newBuilder().setClusterKey("").build();
     }
   }
 
   public static ReplicationPeerConfig convert(ReplicationProtos.ReplicationPeer peer) {
-    ReplicationPeerConfig peerConfig = new ReplicationPeerConfig();
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
     if (peer.hasClusterkey()) {
-      peerConfig.setClusterKey(peer.getClusterkey());
+      builder.setClusterKey(peer.getClusterkey());
     }
     if (peer.hasReplicationEndpointImpl()) {
-      peerConfig.setReplicationEndpointImpl(peer.getReplicationEndpointImpl());
+      builder.setReplicationEndpointImpl(peer.getReplicationEndpointImpl());
     }
 
+    Map<byte[], byte[]> peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     for (HBaseProtos.BytesBytesPair pair : peer.getDataList()) {
-      peerConfig.getPeerData().put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
+      peerData.put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
     }
+    builder.setPeerData(peerData);
 
+    Map<String, String> configuration = new HashMap<>();
     for (HBaseProtos.NameStringPair pair : peer.getConfigurationList()) {
-      peerConfig.getConfiguration().put(pair.getName(), pair.getValue());
+      configuration.put(pair.getName(), pair.getValue());
     }
+    builder.setConfiguration(configuration);
 
-    Map<TableName, ? extends Collection<String>> tableCFsMap = convert2Map(
+    Map<TableName, List<String>> tableCFsMap = convert2Map(
       peer.getTableCfsList().toArray(new ReplicationProtos.TableCF[peer.getTableCfsCount()]));
     if (tableCFsMap != null) {
-      peerConfig.setTableCFsMap(tableCFsMap);
+      builder.setTableCFsMap(tableCFsMap);
     }
 
     List<ByteString> namespacesList = peer.getNamespacesList();
     if (namespacesList != null && namespacesList.size() != 0) {
-      peerConfig.setNamespaces(
+      builder.setNamespaces(
         namespacesList.stream().map(ByteString::toStringUtf8).collect(Collectors.toSet()));
     }
 
     if (peer.hasBandwidth()) {
-      peerConfig.setBandwidth(peer.getBandwidth());
+      builder.setBandwidth(peer.getBandwidth());
     }
 
     if (peer.hasReplicateAll()) {
-      peerConfig.setReplicateAllUserTables(peer.getReplicateAll());
+      builder.setReplicateAllUserTables(peer.getReplicateAll());
     }
 
-    Map<TableName, ? extends Collection<String>> excludeTableCFsMap =
-        convert2Map(peer.getExcludeTableCfsList()
-            .toArray(new ReplicationProtos.TableCF[peer.getExcludeTableCfsCount()]));
+    Map<TableName, List<String>> excludeTableCFsMap = convert2Map(peer.getExcludeTableCfsList()
+        .toArray(new ReplicationProtos.TableCF[peer.getExcludeTableCfsCount()]));
     if (excludeTableCFsMap != null) {
-      peerConfig.setExcludeTableCFsMap(excludeTableCFsMap);
+      builder.setExcludeTableCFsMap(excludeTableCFsMap);
     }
 
     List<ByteString> excludeNamespacesList = peer.getExcludeNamespacesList();
     if (excludeNamespacesList != null && excludeNamespacesList.size() != 0) {
-      peerConfig.setExcludeNamespaces(
+      builder.setExcludeNamespaces(
         excludeNamespacesList.stream().map(ByteString::toStringUtf8).collect(Collectors.toSet()));
     }
 
-    return peerConfig;
+    return builder.build();
   }
 
   public static ReplicationProtos.ReplicationPeer convert(ReplicationPeerConfig peerConfig) {
@@ -408,56 +413,69 @@ public final class ReplicationPeerConfigUtil {
     return builder.build();
   }
 
-  public static void appendTableCFsToReplicationPeerConfig(
-      Map<TableName, ? extends Collection<String>> tableCfs, ReplicationPeerConfig peerConfig) {
+  public static ReplicationPeerConfig appendTableCFsToReplicationPeerConfig(
+      Map<TableName, List<String>> tableCfs, ReplicationPeerConfig peerConfig) {
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(peerConfig);
     Map<TableName, List<String>> preTableCfs = peerConfig.getTableCFsMap();
     if (preTableCfs == null) {
-      peerConfig.setTableCFsMap(tableCfs);
+      builder.setTableCFsMap(tableCfs);
     } else {
+      Map<TableName, List<String>> newTableCfs = copyTableCFsMap(preTableCfs);
       for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
         TableName table = entry.getKey();
         Collection<String> appendCfs = entry.getValue();
-        if (preTableCfs.containsKey(table)) {
-          List<String> cfs = preTableCfs.get(table);
+        if (newTableCfs.containsKey(table)) {
+          List<String> cfs = newTableCfs.get(table);
           if (cfs == null || appendCfs == null || appendCfs.isEmpty()) {
-            preTableCfs.put(table, null);
+            newTableCfs.put(table, null);
           } else {
             Set<String> cfSet = new HashSet<String>(cfs);
             cfSet.addAll(appendCfs);
-            preTableCfs.put(table, Lists.newArrayList(cfSet));
+            newTableCfs.put(table, Lists.newArrayList(cfSet));
           }
         } else {
           if (appendCfs == null || appendCfs.isEmpty()) {
-            preTableCfs.put(table, null);
+            newTableCfs.put(table, null);
           } else {
-            preTableCfs.put(table, Lists.newArrayList(appendCfs));
+            newTableCfs.put(table, Lists.newArrayList(appendCfs));
           }
         }
       }
+      builder.setTableCFsMap(newTableCfs);
     }
+    return builder.build();
   }
 
-  public static void removeTableCFsFromReplicationPeerConfig(
-      Map<TableName, ? extends Collection<String>> tableCfs, ReplicationPeerConfig peerConfig,
+  private static Map<TableName, List<String>>
+      copyTableCFsMap(Map<TableName, List<String>> preTableCfs) {
+    Map<TableName, List<String>> newTableCfs = new HashMap<>();
+    preTableCfs.forEach(
+      (table, cfs) -> newTableCfs.put(table, cfs != null ? Lists.newArrayList(cfs) : null));
+    return newTableCfs;
+  }
+
+  public static ReplicationPeerConfig removeTableCFsFromReplicationPeerConfig(
+      Map<TableName, List<String>> tableCfs, ReplicationPeerConfig peerConfig,
       String id) throws ReplicationException {
     Map<TableName, List<String>> preTableCfs = peerConfig.getTableCFsMap();
     if (preTableCfs == null) {
       throw new ReplicationException("Table-Cfs for peer: " + id + " is null");
     }
+    Map<TableName, List<String>> newTableCfs = copyTableCFsMap(preTableCfs);
     for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
       TableName table = entry.getKey();
       Collection<String> removeCfs = entry.getValue();
-      if (preTableCfs.containsKey(table)) {
-        List<String> cfs = preTableCfs.get(table);
+      if (newTableCfs.containsKey(table)) {
+        List<String> cfs = newTableCfs.get(table);
         if (cfs == null && (removeCfs == null || removeCfs.isEmpty())) {
-          preTableCfs.remove(table);
+          newTableCfs.remove(table);
         } else if (cfs != null && (removeCfs != null && !removeCfs.isEmpty())) {
           Set<String> cfSet = new HashSet<String>(cfs);
           cfSet.removeAll(removeCfs);
           if (cfSet.isEmpty()) {
-            preTableCfs.remove(table);
+            newTableCfs.remove(table);
           } else {
-            preTableCfs.put(table, Lists.newArrayList(cfSet));
+            newTableCfs.put(table, Lists.newArrayList(cfSet));
           }
         } else if (cfs == null && (removeCfs != null && !removeCfs.isEmpty())) {
           throw new ReplicationException("Cannot remove cf of table: " + table
@@ -467,10 +485,13 @@ public final class ReplicationPeerConfigUtil {
               + " which has specified cfs from table-cfs config in peer: " + id);
         }
       } else {
-        throw new ReplicationException("No table: "
-            + table + " in table-cfs config of peer: " + id);
+        throw new ReplicationException(
+            "No table: " + table + " in table-cfs config of peer: " + id);
       }
     }
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(peerConfig);
+    builder.setTableCFsMap(newTableCfs);
+    return builder.build();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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 52a5fe9..8f6b938 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -41,12 +42,44 @@ public class ReplicationPeerConfig {
   private final Map<String, String> configuration;
   private Map<TableName, ? extends Collection<String>> tableCFsMap = null;
   private Set<String> namespaces = null;
-  private long bandwidth = 0;
   // Default value is true, means replicate all user tables to peer cluster.
   private boolean replicateAllUserTables = true;
   private Map<TableName, ? extends Collection<String>> excludeTableCFsMap = null;
   private Set<String> excludeNamespaces = null;
+  private long bandwidth = 0;
+
+  private ReplicationPeerConfig(ReplicationPeerConfigBuilderImpl builder) {
+    this.clusterKey = builder.clusterKey;
+    this.replicationEndpointImpl = builder.replicationEndpointImpl;
+    this.peerData = Collections.unmodifiableMap(builder.peerData);
+    this.configuration = Collections.unmodifiableMap(builder.configuration);
+    this.tableCFsMap =
+        builder.tableCFsMap != null ? unmodifiableTableCFsMap(builder.tableCFsMap) : null;
+    this.namespaces =
+        builder.namespaces != null ? Collections.unmodifiableSet(builder.namespaces) : null;
+    this.replicateAllUserTables = builder.replicateAllUserTables;
+    this.excludeTableCFsMap =
+        builder.excludeTableCFsMap != null ? unmodifiableTableCFsMap(builder.excludeTableCFsMap)
+            : null;
+    this.excludeNamespaces =
+        builder.excludeNamespaces != null ? Collections.unmodifiableSet(builder.excludeNamespaces)
+            : null;
+    this.bandwidth = builder.bandwidth;
+  }
 
+  private Map<TableName, List<String>>
+      unmodifiableTableCFsMap(Map<TableName, List<String>> tableCFsMap) {
+    Map<TableName, List<String>> newTableCFsMap = new HashMap<>();
+    tableCFsMap.forEach((table, cfs) -> newTableCFsMap.put(table,
+      cfs != null ? Collections.unmodifiableList(cfs) : null));
+    return Collections.unmodifiableMap(newTableCFsMap);
+  }
+
+  /**
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder} to create new ReplicationPeerConfig.
+   */
+  @Deprecated
   public ReplicationPeerConfig() {
     this.peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     this.configuration = new HashMap<>(0);
@@ -54,8 +87,11 @@ public class ReplicationPeerConfig {
 
   /**
    * Set the clusterKey which is the concatenation of the slave cluster's:
-   *          hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+   * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder#setClusterKey(String)} instead.
    */
+  @Deprecated
   public ReplicationPeerConfig setClusterKey(String clusterKey) {
     this.clusterKey = clusterKey;
     return this;
@@ -64,7 +100,10 @@ public class ReplicationPeerConfig {
   /**
    * Sets the ReplicationEndpoint plugin class for this peer.
    * @param replicationEndpointImpl a class implementing ReplicationEndpoint
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder#setReplicationEndpointImpl(String)} instead.
    */
+  @Deprecated
   public ReplicationPeerConfig setReplicationEndpointImpl(String replicationEndpointImpl) {
     this.replicationEndpointImpl = replicationEndpointImpl;
     return this;
@@ -90,6 +129,11 @@ public class ReplicationPeerConfig {
     return (Map<TableName, List<String>>) tableCFsMap;
   }
 
+  /**
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder#setTableCFsMap(Map)} instead.
+   */
+  @Deprecated
   public ReplicationPeerConfig setTableCFsMap(Map<TableName,
                                               ? extends Collection<String>> tableCFsMap) {
     this.tableCFsMap = tableCFsMap;
@@ -100,6 +144,11 @@ public class ReplicationPeerConfig {
     return this.namespaces;
   }
 
+  /**
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder#setNamespaces(Set)} instead.
+   */
+  @Deprecated
   public ReplicationPeerConfig setNamespaces(Set<String> namespaces) {
     this.namespaces = namespaces;
     return this;
@@ -109,6 +158,11 @@ public class ReplicationPeerConfig {
     return this.bandwidth;
   }
 
+  /**
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder#setBandwidth(long)} instead.
+   */
+  @Deprecated
   public ReplicationPeerConfig setBandwidth(long bandwidth) {
     this.bandwidth = bandwidth;
     return this;
@@ -118,6 +172,11 @@ public class ReplicationPeerConfig {
     return this.replicateAllUserTables;
   }
 
+  /**
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder#setReplicateAllUserTables(boolean)} instead.
+   */
+  @Deprecated
   public ReplicationPeerConfig setReplicateAllUserTables(boolean replicateAllUserTables) {
     this.replicateAllUserTables = replicateAllUserTables;
     return this;
@@ -127,6 +186,11 @@ public class ReplicationPeerConfig {
     return (Map<TableName, List<String>>) excludeTableCFsMap;
   }
 
+  /**
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder#setExcludeTableCFsMap(Map)} instead.
+   */
+  @Deprecated
   public ReplicationPeerConfig setExcludeTableCFsMap(Map<TableName,
                                               ? extends Collection<String>> tableCFsMap) {
     this.excludeTableCFsMap = tableCFsMap;
@@ -137,11 +201,124 @@ public class ReplicationPeerConfig {
     return this.excludeNamespaces;
   }
 
+  /**
+   * @deprecated as release of 2.0.0, and it will be removed in 3.0.0. Use
+   *             {@link ReplicationPeerConfigBuilder#setExcludeNamespaces(Set)} instead.
+   */
+  @Deprecated
   public ReplicationPeerConfig setExcludeNamespaces(Set<String> namespaces) {
     this.excludeNamespaces = namespaces;
     return this;
   }
 
+  public static ReplicationPeerConfigBuilder newBuilder() {
+    return new ReplicationPeerConfigBuilderImpl();
+  }
+
+  public static ReplicationPeerConfigBuilder newBuilder(ReplicationPeerConfig peerConfig) {
+    ReplicationPeerConfigBuilderImpl builder = new ReplicationPeerConfigBuilderImpl();
+    builder.setClusterKey(peerConfig.getClusterKey())
+        .setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl())
+        .setPeerData(peerConfig.getPeerData()).setConfiguration(peerConfig.getConfiguration())
+        .setTableCFsMap(peerConfig.getTableCFsMap()).setNamespaces(peerConfig.getNamespaces())
+        .setReplicateAllUserTables(peerConfig.replicateAllUserTables())
+        .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap())
+        .setExcludeNamespaces(peerConfig.getExcludeNamespaces())
+        .setBandwidth(peerConfig.getBandwidth());
+    return builder;
+  }
+
+  static class ReplicationPeerConfigBuilderImpl implements ReplicationPeerConfigBuilder {
+
+    private String clusterKey;
+
+    private String replicationEndpointImpl;
+
+    private Map<byte[], byte[]> peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+
+    private Map<String, String> configuration = new HashMap<>();
+
+    private Map<TableName, List<String>> tableCFsMap = null;
+
+    private Set<String> namespaces = null;
+
+    // Default value is true, means replicate all user tables to peer cluster.
+    private boolean replicateAllUserTables = true;
+
+    private Map<TableName, List<String>> excludeTableCFsMap = null;
+
+    private Set<String> excludeNamespaces = null;
+
+    private long bandwidth = 0;
+
+    @Override
+    public ReplicationPeerConfigBuilder setClusterKey(String clusterKey) {
+      this.clusterKey = clusterKey;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder setReplicationEndpointImpl(String replicationEndpointImpl) {
+      this.replicationEndpointImpl = replicationEndpointImpl;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder setPeerData(Map<byte[], byte[]> peerData) {
+      this.peerData = peerData;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder setConfiguration(Map<String, String> configuration) {
+      this.configuration = configuration;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder
+        setTableCFsMap(Map<TableName, List<String>> tableCFsMap) {
+      this.tableCFsMap = tableCFsMap;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder setNamespaces(Set<String> namespaces) {
+      this.namespaces = namespaces;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder setReplicateAllUserTables(boolean replicateAllUserTables) {
+      this.replicateAllUserTables = replicateAllUserTables;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder
+        setExcludeTableCFsMap(Map<TableName, List<String>> excludeTableCFsMap) {
+      this.excludeTableCFsMap = excludeTableCFsMap;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder setExcludeNamespaces(Set<String> excludeNamespaces) {
+      this.excludeNamespaces = excludeNamespaces;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfigBuilder setBandwidth(long bandwidth) {
+      this.bandwidth = bandwidth;
+      return this;
+    }
+
+    @Override
+    public ReplicationPeerConfig build() {
+      return new ReplicationPeerConfig(this);
+    }
+  }
+
   @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("clusterKey=").append(clusterKey).append(",");

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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
new file mode 100644
index 0000000..b642acf
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
@@ -0,0 +1,64 @@
+/**
+ * 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.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * For creating {@link ReplicationPeerConfig}.
+ */
+@InterfaceAudience.Public
+public interface ReplicationPeerConfigBuilder {
+
+  /**
+   * Set the clusterKey which is the concatenation of the slave cluster's:
+   * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+   */
+  ReplicationPeerConfigBuilder setClusterKey(String clusterKey);
+
+  /**
+   * Sets the ReplicationEndpoint plugin class for this peer.
+   * @param replicationEndpointImpl a class implementing ReplicationEndpoint
+   */
+  ReplicationPeerConfigBuilder setReplicationEndpointImpl(String replicationEndpointImpl);
+
+  ReplicationPeerConfigBuilder setPeerData(Map<byte[], byte[]> peerData);
+
+  ReplicationPeerConfigBuilder setConfiguration(Map<String, String> configuration);
+
+  ReplicationPeerConfigBuilder
+      setTableCFsMap(Map<TableName, List<String>> tableCFsMap);
+
+  ReplicationPeerConfigBuilder setNamespaces(Set<String> namespaces);
+
+  ReplicationPeerConfigBuilder setBandwidth(long bandwidth);
+
+  ReplicationPeerConfigBuilder setReplicateAllUserTables(boolean replicateAllUserTables);
+
+  ReplicationPeerConfigBuilder setExcludeTableCFsMap(Map<TableName, List<String>> tableCFsMap);
+
+  ReplicationPeerConfigBuilder setExcludeNamespaces(Set<String> namespaces);
+
+  ReplicationPeerConfig build();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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 8f5e8d5..ff6c07b 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,6 +21,7 @@ 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;
@@ -37,6 +38,7 @@ 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,18 +365,19 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     }
     //Update existingConfig's peer config and peer data with the new values, but don't touch config
     // or data that weren't explicitly changed
-    existingConfig.getConfiguration().putAll(newConfig.getConfiguration());
-    existingConfig.getPeerData().putAll(newConfig.getPeerData());
-    existingConfig.setTableCFsMap(newConfig.getTableCFsMap());
-    existingConfig.setNamespaces(newConfig.getNamespaces());
-    existingConfig.setBandwidth(newConfig.getBandwidth());
-    existingConfig.setReplicateAllUserTables(newConfig.replicateAllUserTables());
-    existingConfig.setExcludeNamespaces(newConfig.getExcludeNamespaces());
-    existingConfig.setExcludeTableCFsMap(newConfig.getExcludeTableCFsMap());
+    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);
 
     try {
       ZKUtil.setData(this.zookeeper, getPeerNode(id),
-          ReplicationPeerConfigUtil.toByteArray(existingConfig));
+          ReplicationPeerConfigUtil.toByteArray(builder.build()));
     }
     catch(KeeperException ke){
       throw new ReplicationException("There was a problem trying to save changes to the " +

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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 6591826..28a7562 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -68,6 +69,20 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
   }
 
+  @After
+  public void cleanupPeer() {
+    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();
+    } catch (Exception e) {
+      LOG.debug("Replication peer " + ID_SECOND + " may already be removed");
+    }
+  }
+
   @Test
   public void testAddRemovePeer() throws Exception {
     ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
@@ -350,7 +365,7 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
     // update peer config only contains ns1
     rpc = admin.getReplicationPeerConfig(ID_ONE).get();
-    namespaces.clear();
+    namespaces = new HashSet<>();
     namespaces.add(ns1);
     rpc.setNamespaces(namespaces);
     admin.updateReplicationPeerConfig(ID_ONE, rpc).join();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0dadb45a/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 8770299..dd33564 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -99,6 +100,20 @@ public class TestReplicationAdmin {
     TEST_UTIL.shutdownMiniCluster();
   }
 
+  @After
+  public void cleanupPeer() {
+    try {
+      hbaseAdmin.removeReplicationPeer(ID_ONE);
+    } catch (Exception e) {
+      LOG.debug("Replication peer " + ID_ONE + " may already be removed");
+    }
+    try {
+      hbaseAdmin.removeReplicationPeer(ID_SECOND);
+    } catch (Exception e) {
+      LOG.debug("Replication peer " + ID_SECOND + " may already be removed");
+    }
+  }
+
   /**
    * Simple testing of adding and removing peers, basically shows that
    * all interactions with ZK work
@@ -449,7 +464,7 @@ public class TestReplicationAdmin {
     assertTrue(namespaces.contains(ns2));
 
     rpc = hbaseAdmin.getReplicationPeerConfig(ID_ONE);
-    namespaces.clear();
+    namespaces = new HashSet<>();
     namespaces.add(ns1);
     rpc.setNamespaces(namespaces);
     hbaseAdmin.updateReplicationPeerConfig(ID_ONE, rpc);
@@ -505,7 +520,7 @@ public class TestReplicationAdmin {
     assertTrue(namespaces.contains(ns2));
 
     rpc = hbaseAdmin.getReplicationPeerConfig(ID_ONE);
-    namespaces.clear();
+    namespaces = new HashSet<String>();
     namespaces.add(ns1);
     rpc.setExcludeNamespaces(namespaces);
     hbaseAdmin.updateReplicationPeerConfig(ID_ONE, rpc);


[20/22] 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/0e53efd0/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 9ff1865..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java
+++ /dev/null
@@ -1,62 +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();
-  }
-
-}


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

Branch: refs/heads/HBASE-19397
Commit: beab0c05c5a561758ce0f7feb852e7fb1367b965
Parents: 71e9d95
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 22 14:37:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:30:42 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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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 024a6ff..b72255a 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;
@@ -329,7 +330,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
@@ -700,8 +701,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);
@@ -719,7 +720,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();
@@ -764,9 +765,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());
@@ -3356,18 +3356,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
@@ -3386,7 +3387,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);
     }
@@ -3536,7 +3538,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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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


[18/22] 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/beab0c05/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/beab0c05/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/beab0c05/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/beab0c05/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 a9844f3..fd62c81 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,17 +37,16 @@ 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.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;
@@ -49,15 +54,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
@@ -65,8 +61,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();
 
@@ -101,16 +95,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);
     }
   }
 
@@ -200,32 +195,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();
   }
 
   /**
@@ -421,7 +413,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/beab0c05/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/beab0c05/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/beab0c05/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");


[09/22] hbase git commit: HBASE-19608 Race in MasterRpcServices.getProcedureResult

Posted by zh...@apache.org.
HBASE-19608 Race in MasterRpcServices.getProcedureResult


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

Branch: refs/heads/HBASE-19397
Commit: 27c56860da637fec0954c0fc4456f42e577da6af
Parents: c24cf2d
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 25 10:45:35 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 11:52:34 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/MasterRpcServices.java  | 44 +++++++++-----------
 1 file changed, 19 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/27c56860/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 f0f2e10..2e4b5e1 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
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
 import org.apache.hadoop.hbase.procedure2.LockType;
 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.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
@@ -1153,35 +1154,28 @@ public class MasterRpcServices extends RSRpcServices
     try {
       master.checkInitialized();
       GetProcedureResultResponse.Builder builder = GetProcedureResultResponse.newBuilder();
-
-      Procedure<?> result = master.getMasterProcedureExecutor()
-          .getResultOrProcedure(request.getProcId());
-      if (result == null) {
-        builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
-      } else {
-        boolean remove = false;
-
-        if (result.isFinished() || result.isFailed()) {
-          builder.setState(GetProcedureResultResponse.State.FINISHED);
-          remove = true;
-        } else {
-          builder.setState(GetProcedureResultResponse.State.RUNNING);
-        }
-
+      long procId = request.getProcId();
+      ProcedureExecutor<?> executor = master.getMasterProcedureExecutor();
+      Procedure<?> result = executor.getResultOrProcedure(procId);
+      if (result != null) {
         builder.setSubmittedTime(result.getSubmittedTime());
         builder.setLastUpdate(result.getLastUpdate());
-        if (result.isFailed()) {
-          IOException exception = result.getException().unwrapRemoteIOException();
-          builder.setException(ForeignExceptionUtil.toProtoForeignException(exception));
-        }
-        byte[] resultData = result.getResult();
-        if (resultData != null) {
-          builder.setResult(UnsafeByteOperations.unsafeWrap(resultData));
-        }
-
-        if (remove) {
+        if (executor.isFinished(procId)) {
+          builder.setState(GetProcedureResultResponse.State.FINISHED);
+          if (result.isFailed()) {
+            IOException exception = result.getException().unwrapRemoteIOException();
+            builder.setException(ForeignExceptionUtil.toProtoForeignException(exception));
+          }
+          byte[] resultData = result.getResult();
+          if (resultData != null) {
+            builder.setResult(UnsafeByteOperations.unsafeWrap(resultData));
+          }
           master.getMasterProcedureExecutor().removeResult(request.getProcId());
+        } else {
+          builder.setState(GetProcedureResultResponse.State.RUNNING);
         }
+      } else {
+        builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
       }
       return builder.build();
     } catch (IOException e) {


[03/22] hbase git commit: HBASE-19597 Fixed Checkstyle errors in hbase-spark and enabled Checkstyle to fail on violations

Posted by zh...@apache.org.
HBASE-19597 Fixed Checkstyle errors in hbase-spark and enabled Checkstyle to fail on violations


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

Branch: refs/heads/HBASE-19397
Commit: 60077255219b2a235846a348a0469880b804f773
Parents: c0c802f
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Fri Dec 22 13:03:48 2017 +0100
Committer: Jan Hentschel <ja...@ultratendency.com>
Committed: Fri Dec 22 19:11:07 2017 +0100

----------------------------------------------------------------------
 hbase-spark/pom.xml                             | 16 +++++++
 .../hbase/spark/SparkSQLPushDownFilter.java     | 25 +++++-----
 .../JavaHBaseBulkDeleteExample.java             |  6 +--
 .../hbasecontext/JavaHBaseBulkLoadExample.java  | 32 ++++++++-----
 .../hbasecontext/JavaHBaseMapGetPutExample.java | 30 ++++++------
 .../hbase/spark/TestJavaHBaseContext.java       | 50 ++++++++++++--------
 6 files changed, 96 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/60077255/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index a1ac466..5397d99 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -713,6 +713,22 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>checkstyle</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+            <configuration>
+              <failOnViolation>true</failOnViolation>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
   <profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/60077255/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
index 6643169..e9bb511 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
@@ -17,10 +17,15 @@
 
 package org.apache.hadoop.hbase.spark;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.hbase.Cell;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.spark.datasources.BytesEncoder;
@@ -29,15 +34,11 @@ import org.apache.hadoop.hbase.spark.protobuf.generated.SparkFilterProtos;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.spark.sql.datasources.hbase.Field;
-import scala.collection.mutable.MutableList;
-
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.ByteString;
+import scala.collection.mutable.MutableList;
 
 /**
  * This filter will push down all qualifier logic given to us
@@ -179,7 +180,7 @@ public class SparkSQLPushDownFilter extends FilterBase{
   /**
    * @param pbBytes A pb serialized instance
    * @return An instance of SparkSQLPushDownFilter
-   * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
+   * @throws DeserializationException if the filter cannot be parsed from the given bytes
    */
   @SuppressWarnings("unused")
   public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes)

http://git-wip-us.apache.org/repos/asf/hbase/blob/60077255/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java
index 97cf140..d7c424e 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java
@@ -16,6 +16,9 @@
  */
 package org.apache.hadoop.hbase.spark.example.hbasecontext;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
@@ -27,9 +30,6 @@ import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.api.java.function.Function;
 
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * This is a simple example of deleting records in HBase
  * with the bulkDelete function.

http://git-wip-us.apache.org/repos/asf/hbase/blob/60077255/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java
index 54ff658..f0f3e79 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java
@@ -16,6 +16,10 @@
  */
 package org.apache.hadoop.hbase.spark.example.hbasecontext;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
@@ -25,23 +29,21 @@ import org.apache.hadoop.hbase.spark.JavaHBaseContext;
 import org.apache.hadoop.hbase.spark.KeyFamilyQualifier;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.function.Function;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
 /**
  * Run this example using command below:
  *
- *  SPARK_HOME/bin/spark-submit --master local[2] --class org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample
+ *  SPARK_HOME/bin/spark-submit --master local[2]
+ *  --class org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample
  *  path/to/hbase-spark.jar {path/to/output/HFiles}
  *
  * This example will output put hfiles in {path/to/output/HFiles}, and user can run
- * 'hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles' to load the HFiles into table to verify this example.
+ * 'hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles' to load the HFiles into table to
+ * verify this example.
  */
 final public class JavaHBaseBulkLoadExample {
   private JavaHBaseBulkLoadExample() {}
@@ -85,17 +87,21 @@ final public class JavaHBaseBulkLoadExample {
     }
   }
 
-  public static class BulkLoadFunction implements Function<String, Pair<KeyFamilyQualifier, byte[]>> {
-
+  public static class BulkLoadFunction
+          implements Function<String, Pair<KeyFamilyQualifier, byte[]>> {
     @Override
     public Pair<KeyFamilyQualifier, byte[]> call(String v1) throws Exception {
-      if (v1 == null)
+      if (v1 == null) {
         return null;
+      }
+
       String[] strs = v1.split(",");
-      if(strs.length != 4)
+      if(strs.length != 4) {
         return null;
-      KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]),
-          Bytes.toBytes(strs[2]));
+      }
+
+      KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]),
+              Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2]));
       return new Pair(kfq, Bytes.toBytes(strs[3]));
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/60077255/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java
index 316f8a1..9a1259e 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java
@@ -73,24 +73,24 @@ final public class JavaHBaseMapGetPutExample {
 
       hbaseContext.foreachPartition(rdd,
               new VoidFunction<Tuple2<Iterator<byte[]>, Connection>>() {
-        public void call(Tuple2<Iterator<byte[]>, Connection> t)
-                throws Exception {
-          Table table = t._2().getTable(TableName.valueOf(tableName));
-          BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName));
+          public void call(Tuple2<Iterator<byte[]>, Connection> t)
+                  throws Exception {
+            Table table = t._2().getTable(TableName.valueOf(tableName));
+            BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName));
 
-          while (t._1().hasNext()) {
-            byte[] b = t._1().next();
-            Result r = table.get(new Get(b));
-            if (r.getExists()) {
-              mutator.mutate(new Put(b));
+            while (t._1().hasNext()) {
+              byte[] b = t._1().next();
+              Result r = table.get(new Get(b));
+              if (r.getExists()) {
+                mutator.mutate(new Put(b));
+              }
             }
-          }
 
-          mutator.flush();
-          mutator.close();
-          table.close();
-        }
-      });
+            mutator.flush();
+            mutator.close();
+            table.close();
+          }
+        });
     } finally {
       jsc.stop();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/60077255/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java b/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java
index f9a813d..c2ecbc8 100644
--- a/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java
+++ b/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java
@@ -20,16 +20,16 @@ import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HConstants;
 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.Connection;
@@ -41,12 +41,11 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkDeleteExample;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
-
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
@@ -61,6 +60,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import scala.Tuple2;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.io.Files;
 
 @Category({MiscTests.class, MediumTests.class})
@@ -317,10 +317,12 @@ public class TestJavaHBaseContext implements Serializable {
 
 
 
-    hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName), new BulkLoadFunction(), output.toUri().getPath(),
-        new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
+    hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName), new BulkLoadFunction(),
+            output.toUri().getPath(), new HashMap<byte[], FamilyHFileWriteOptions>(), false,
+            HConstants.DEFAULT_MAX_FILE_SIZE);
 
-    try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) {
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+         Admin admin = conn.getAdmin()) {
       Table table = conn.getTable(TableName.valueOf(tableName));
       // Do bulk load
       LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf);
@@ -387,11 +389,13 @@ public class TestJavaHBaseContext implements Serializable {
     Configuration conf = htu.getConfiguration();
     JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
 
-    hbaseContext.bulkLoadThinRows(rdd, TableName.valueOf(tableName), new BulkLoadThinRowsFunction(), output.toString(),
-        new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
+    hbaseContext.bulkLoadThinRows(rdd, TableName.valueOf(tableName), new BulkLoadThinRowsFunction(),
+            output.toString(), new HashMap<byte[], FamilyHFileWriteOptions>(), false,
+            HConstants.DEFAULT_MAX_FILE_SIZE);
 
 
-    try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) {
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+         Admin admin = conn.getAdmin()) {
       Table table = conn.getTable(TableName.valueOf(tableName));
       // Do bulk load
       LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf);
@@ -429,25 +433,31 @@ public class TestJavaHBaseContext implements Serializable {
     }
 
   }
-  public static class BulkLoadFunction implements Function<String, Pair<KeyFamilyQualifier, byte[]>> {
-
+  public static class BulkLoadFunction
+          implements Function<String, Pair<KeyFamilyQualifier, byte[]>> {
     @Override public Pair<KeyFamilyQualifier, byte[]> call(String v1) throws Exception {
-      if (v1 == null)
+      if (v1 == null) {
         return null;
+      }
+
       String[] strs = v1.split(",");
-      if(strs.length != 4)
+      if(strs.length != 4) {
         return null;
-      KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]),
-          Bytes.toBytes(strs[2]));
+      }
+
+      KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]),
+              Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2]));
       return new Pair(kfq, Bytes.toBytes(strs[3]));
     }
   }
 
-  public static class BulkLoadThinRowsFunction implements Function<List<String>, Pair<ByteArrayWrapper, FamiliesQualifiersValues>> {
-
-    @Override public Pair<ByteArrayWrapper, FamiliesQualifiersValues> call(List<String> list) throws Exception {
-      if (list == null)
+  public static class BulkLoadThinRowsFunction
+          implements Function<List<String>, Pair<ByteArrayWrapper, FamiliesQualifiersValues>> {
+    @Override public Pair<ByteArrayWrapper, FamiliesQualifiersValues> call(List<String> list) {
+      if (list == null) {
         return null;
+      }
+
       ByteArrayWrapper rowKey = null;
       FamiliesQualifiersValues fqv = new FamiliesQualifiersValues();
       for (String cell : list) {


[10/22] hbase git commit: HBASE-19487 Remove IterablesUtil Class

Posted by zh...@apache.org.
HBASE-19487 Remove IterablesUtil Class

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-19397
Commit: 38472e1c07bc48079f4b9766d45e6b64163bd0d4
Parents: 27c5686
Author: BELUGA BEHR <da...@gmail.com>
Authored: Mon Dec 25 16:11:37 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Dec 25 16:12:22 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/KeyValueTestUtil.java   |  5 +--
 .../org/apache/hadoop/hbase/KeyValueUtil.java   |  4 +-
 .../org/apache/hadoop/hbase/util/Bytes.java     | 22 ++++++----
 .../apache/hadoop/hbase/util/IterableUtils.java | 44 --------------------
 4 files changed, 17 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/38472e1c/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java
index 3d9b4f9..efea740 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java
@@ -22,11 +22,10 @@ import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.commons.collections4.IterableUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.IterableUtils;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.yetus.audience.InterfaceAudience;
-
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 @InterfaceAudience.Private
@@ -64,7 +63,7 @@ public class KeyValueTestUtil {
       boolean includeMemstoreTS) {
     int totalBytes = KeyValueUtil.totalLengthWithMvccVersion(kvs, includeMemstoreTS);
     ByteBuffer bb = ByteBuffer.allocate(totalBytes);
-    for (KeyValue kv : IterableUtils.nullSafe(kvs)) {
+    for (KeyValue kv : IterableUtils.emptyIfNull(kvs)) {
       KeyValueUtil.appendToByteBuffer(bb, kv, includeMemstoreTS);
     }
     bb.rewind();

http://git-wip-us.apache.org/repos/asf/hbase/blob/38472e1c/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
index 60e4049..9d8a608 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
@@ -28,11 +28,11 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.collections4.IterableUtils;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.IterableUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -90,7 +90,7 @@ public class KeyValueUtil {
   public static int totalLengthWithMvccVersion(final Iterable<? extends KeyValue> kvs,
       final boolean includeMvccVersion) {
     int length = 0;
-    for (KeyValue kv : IterableUtils.nullSafe(kvs)) {
+    for (KeyValue kv : IterableUtils.emptyIfNull(kvs)) {
       length += lengthWithMvccVersion(kv, includeMvccVersion);
     }
     return length;

http://git-wip-us.apache.org/repos/asf/hbase/blob/38472e1c/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
index 159a790..12e4788 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
@@ -30,8 +30,10 @@ import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.security.SecureRandom;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
@@ -47,7 +49,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.collect.Lists;
 
 import com.google.protobuf.ByteString;
 
@@ -2339,21 +2340,24 @@ public class Bytes implements Comparable<Bytes> {
   }
 
   public static boolean isSorted(Collection<byte[]> arrays) {
-    byte[] previous = new byte[0];
-    for (byte[] array : IterableUtils.nullSafe(arrays)) {
-      if (Bytes.compareTo(previous, array) > 0) {
-        return false;
+    if (!CollectionUtils.isEmpty(arrays)) {
+      byte[] previous = new byte[0];
+      for (byte[] array : arrays) {
+        if (Bytes.compareTo(previous, array) > 0) {
+          return false;
+        }
+        previous = array;
       }
-      previous = array;
     }
     return true;
   }
 
   public static List<byte[]> getUtf8ByteArrays(List<String> strings) {
-    List<byte[]> byteArrays = Lists.newArrayListWithCapacity(CollectionUtils.nullSafeSize(strings));
-    for (String s : IterableUtils.nullSafe(strings)) {
-      byteArrays.add(Bytes.toBytes(s));
+    if (CollectionUtils.isEmpty(strings)) {
+      return Collections.emptyList();
     }
+    List<byte[]> byteArrays = new ArrayList<>(strings.size());
+    strings.forEach(s -> byteArrays.add(Bytes.toBytes(s)));
     return byteArrays;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/38472e1c/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IterableUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IterableUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IterableUtils.java
deleted file mode 100644
index e0bca84..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IterableUtils.java
+++ /dev/null
@@ -1,44 +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.util;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Utility methods for Iterable including null-safe handlers.
- */
-@InterfaceAudience.Private
-public class IterableUtils {
-
-  private static final List<Object> EMPTY_LIST = Collections
-      .unmodifiableList(new ArrayList<>(0));
-
-  @SuppressWarnings("unchecked")
-  public static <T> Iterable<T> nullSafe(Iterable<T> in) {
-    if (in == null) {
-      return (List<T>) EMPTY_LIST;
-    }
-    return in;
-  }
-
-}


[21/22] 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/0e53efd0/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;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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 fd62c81..328f6d0 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
@@ -47,6 +47,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;
@@ -81,6 +82,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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/0e53efd0/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/22] 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/df22d88b
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/df22d88b
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/df22d88b

Branch: refs/heads/HBASE-19397
Commit: df22d88bde2896f9dfa2da6da3e31c8dbccdfc71
Parents: 38472e1
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 15 21:06:44 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:28:54 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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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..52b7b35 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;
@@ -330,8 +330,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,
@@ -3530,4 +3529,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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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/df22d88b/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.


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

Branch: refs/heads/HBASE-19397
Commit: d56366c308d8db2cce60313f15d680ffad5756fb
Parents: aab7747
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 19 15:50:57 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 25 18:28:54 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/d56366c3/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/d56366c3/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/d56366c3/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) -> {