You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/02/04 12:47:52 UTC

[01/46] hbase git commit: HBASE-19919 Tidying up logging; ADDENDUM Fix tests w/ mocked Servers [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19397-branch-2 754e4f7fb -> 88c4aa7ad (forced update)


HBASE-19919 Tidying up logging; ADDENDUM Fix tests w/ mocked Servers


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 00653a4d18b4e3a97c93d3fd9d44ed9b6149db53
Parents: 66a11f2
Author: Michael Stack <st...@apache.org>
Authored: Sat Feb 3 09:25:16 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 09:25:45 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/SplitLogManager.java   | 6 +++++-
 .../hadoop/hbase/master/assignment/AssignmentManager.java      | 6 +++++-
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/00653a4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
index 63af398..2e2f8bf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
@@ -121,8 +121,12 @@ public class SplitLogManager {
       throws IOException {
     this.server = master;
     this.conf = conf;
+    // Get Server Thread name. Sometimes the Server is mocked so may not implement HasThread.
+    // For example, in tests.
+    String name = master instanceof HasThread? ((HasThread)master).getName():
+        master.getServerName().toShortString();
     this.choreService =
-        new ChoreService(((HasThread)master).getName() + ".splitLogManager.");
+        new ChoreService(name + ".splitLogManager.");
     if (server.getCoordinatedStateManager() != null) {
       SplitLogManagerCoordination coordination = getSplitLogManagerCoordination();
       Set<String> failedDeletions = Collections.synchronizedSet(new HashSet<String>());

http://git-wip-us.apache.org/repos/asf/hbase/blob/00653a4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 1f65230..e09b29b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -1624,7 +1624,11 @@ public class AssignmentManager implements ServerListener {
   }
 
   private void startAssignmentThread() {
-    assignThread = new Thread(((HasThread)this.master).getName()) {
+    // Get Server Thread name. Sometimes the Server is mocked so may not implement HasThread.
+    // For example, in tests.
+    String name = master instanceof HasThread? ((HasThread)master).getName():
+        master.getServerName().toShortString();
+    assignThread = new Thread(name) {
       @Override
       public void run() {
         while (isRunning()) {


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 762770b090820fd92085466eea20033cbe5763cd
Parents: e5dc52a
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 19 15:50:57 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/762770b0/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 40dac2f..b8546fa 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
@@ -2473,7 +2473,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)
@@ -2484,7 +2484,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
    */
@@ -2492,6 +2492,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
@@ -2499,6 +2530,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
@@ -2506,6 +2549,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
@@ -2513,6 +2568,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
@@ -2523,13 +2590,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/762770b0/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 c137383..8685984 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -200,7 +201,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 
 /**
@@ -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/762770b0/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 a826f8c..050bfe2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -36,6 +36,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -1558,47 +1559,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
@@ -1617,13 +1605,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
@@ -2582,6 +2568,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) -> {


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 3be13975aeda434cc4816f7c10ca30213a58994b
Parents: 7bb1768
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 2 13:25:58 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:42:08 2018 +0800

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


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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3be13975/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 cc57dfb..b5a50c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -75,7 +74,6 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 120000);
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
     TEST_UTIL.getConfiguration().setInt(START_LOG_ERRORS_AFTER_COUNT_KEY, 0);
-    TEST_UTIL.getConfiguration().setInt(ReadOnlyZKClient.RECOVERY_RETRY, 1);
     TEST_UTIL.startMiniCluster();
     ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
   }

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3be13975/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
index ed181dd..b98ca7f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
@@ -162,15 +162,14 @@ public class TestReplicationSource {
       }
     };
     replicationEndpoint.start();
-    ReplicationPeers mockPeers = Mockito.mock(ReplicationPeers.class);
     ReplicationPeer mockPeer = Mockito.mock(ReplicationPeer.class);
     Mockito.when(mockPeer.getPeerBandwidth()).thenReturn(0L);
     Configuration testConf = HBaseConfiguration.create();
     testConf.setInt("replication.source.maxretriesmultiplier", 1);
     ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class);
     Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong());
-    source.init(testConf, null, manager, null, mockPeers, null, "testPeer", null,
-      replicationEndpoint, p -> OptionalLong.empty(), null);
+    source.init(testConf, null, manager, null, mockPeer, null, "testPeer", null,
+      p -> OptionalLong.empty(), null);
     ExecutorService executor = Executors.newSingleThreadExecutor();
     Future<?> future = executor.submit(new Runnable() {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3be13975/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 77b2fb2..ffa889a 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
@@ -64,8 +64,8 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
+import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@@ -736,9 +736,9 @@ public abstract class TestReplicationSourceManager {
 
     @Override
     public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-        ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
-        UUID clusterId, ReplicationEndpoint replicationEndpoint,
-        WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
+        ReplicationQueueStorage rq, ReplicationPeer rp, Server server, String peerClusterId,
+        UUID clusterId, WALFileLengthProvider walFileLengthProvider, MetricsSource metrics)
+        throws IOException {
       throw new IOException("Failing deliberately");
     }
   }


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

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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 068dcb478e2fa38d41e5bd5ba2d73b2134b94a8b
Parents: 7671652
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 26 21:10:00 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

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


[32/46] hbase git commit: HBASE-19697 Remove TestReplicationAdminUsingProcedure

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 4112552692f997ba7d1be0da5181395799e00375
Parents: f4c9da4
Author: zhangduo <zh...@apache.org>
Authored: Wed Jan 3 21:13:57 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:42:08 2018 +0800

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


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


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

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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 51c850971683f48f24cfd494870c108ece8a260d
Parents: 9e8400f
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Dec 23 21:04:27 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

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


[07/46] hbase git commit: Revert "HBASE-19928 TestVisibilityLabelsOnNewVersionBehaviorTable fails; ADDENDUM Fix failing TestMetaWithReplicas#testShutdownHandling; it was reading meta TableState" Wrong log message!

Posted by zh...@apache.org.
Revert "HBASE-19928 TestVisibilityLabelsOnNewVersionBehaviorTable fails; ADDENDUM Fix failing TestMetaWithReplicas#testShutdownHandling; it was reading meta TableState"
Wrong log message!

This reverts commit fbcb453ce2bbe4ffe0fa723b2ae05a7f91a6fc5d.


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: cd610607e76ebc8ae01a0efe30134bfa887c4679
Parents: fbcb453
Author: Michael Stack <st...@apache.org>
Authored: Sat Feb 3 21:48:39 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 21:48:39 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java  | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cd610607/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 5dc0565..f80bbc0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1109,9 +1109,6 @@ public class MetaTableAccessor {
   @Nullable
   public static TableState getTableState(Connection conn, TableName tableName)
       throws IOException {
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
-      return new TableState(tableName, TableState.State.ENABLED);
-    }
     Table metaHTable = getMetaHTable(conn);
     Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getTableStateColumn());
     long time = EnvironmentEdgeManager.currentTime();


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

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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 8bb3cf6688799beb587df4bff8fa40be191af374
Parents: d5ddc7e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 12 22:04:38 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:43:05 2018 +0800

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


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


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 767165288b71773e3f77202542c0e9d1ba00a96e
Parents: 51c8509
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 26 11:39:34 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/76716528/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
new file mode 100644
index 0000000..2c7ea9b
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
@@ -0,0 +1,169 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+@InterfaceAudience.Private
+public class ReplicationPeerImpl implements ReplicationPeer {
+  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerImpl.class);
+
+  private final ReplicationPeerStorage peerStorage;
+
+  private final Configuration conf;
+
+  private final String id;
+
+  private volatile ReplicationPeerConfig peerConfig;
+
+  private volatile PeerState peerState;
+
+  private final List<ReplicationPeerConfigListener> peerConfigListeners;
+
+  /**
+   * Constructor that takes all the objects required to communicate with the specified peer, except
+   * for the region server addresses.
+   * @param conf configuration object to this peer
+   * @param id string representation of this peer's identifier
+   * @param peerConfig configuration for the replication peer
+   */
+  public ReplicationPeerImpl(ZKWatcher zkWatcher, Configuration conf, String id,
+      ReplicationPeerConfig peerConfig) {
+    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkWatcher, conf);
+    this.conf = conf;
+    this.peerConfig = peerConfig;
+    this.id = id;
+    this.peerConfigListeners = new ArrayList<>();
+  }
+
+  public void refreshPeerState() throws ReplicationException {
+    this.peerState = peerStorage.isPeerEnabled(id) ? PeerState.ENABLED : PeerState.DISABLED;
+  }
+
+  public void refreshPeerConfig() throws ReplicationException {
+    this.peerConfig = peerStorage.getPeerConfig(id).orElse(peerConfig);
+    peerConfigListeners.forEach(listener -> listener.peerConfigUpdated(peerConfig));
+  }
+
+  /**
+   * Get the identifier of this peer
+   * @return string representation of the id (short)
+   */
+  @Override
+  public String getId() {
+    return id;
+  }
+
+  @Override
+  public PeerState getPeerState() {
+    return peerState;
+  }
+
+  /**
+   * Get the peer config object
+   * @return the ReplicationPeerConfig for this peer
+   */
+  @Override
+  public ReplicationPeerConfig getPeerConfig() {
+    return peerConfig;
+  }
+
+  /**
+   * Get the configuration object required to communicate with this peer
+   * @return configuration object
+   */
+  @Override
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  /**
+   * Get replicable (table, cf-list) map of this peer
+   * @return the replicable (table, cf-list) map
+   */
+  @Override
+  public Map<TableName, List<String>> getTableCFs() {
+    return this.peerConfig.getTableCFsMap();
+  }
+
+  /**
+   * Get replicable namespace set of this peer
+   * @return the replicable namespaces set
+   */
+  @Override
+  public Set<String> getNamespaces() {
+    return this.peerConfig.getNamespaces();
+  }
+
+  @Override
+  public long getPeerBandwidth() {
+    return this.peerConfig.getBandwidth();
+  }
+
+  @Override
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
+    this.peerConfigListeners.add(listener);
+  }
+
+  /**
+   * Parse the raw data from ZK to get a peer's state
+   * @param bytes raw ZK data
+   * @return True if the passed in <code>bytes</code> are those of a pb serialized ENABLED state.
+   * @throws DeserializationException
+   */
+  public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
+    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
+    return ReplicationProtos.ReplicationState.State.ENABLED == state;
+  }
+
+  /**
+   * @param bytes Content of a state znode.
+   * @return State parsed from the passed bytes.
+   * @throws DeserializationException
+   */
+  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
+      throws DeserializationException {
+    ProtobufUtil.expectPBMagicPrefix(bytes);
+    int pbLen = ProtobufUtil.lengthOfPBMagic();
+    ReplicationProtos.ReplicationState.Builder builder =
+        ReplicationProtos.ReplicationState.newBuilder();
+    ReplicationProtos.ReplicationState state;
+    try {
+      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
+      state = builder.build();
+      return state.getState();
+    } catch (IOException e) {
+      throw new DeserializationException(e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/76716528/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
deleted file mode 100644
index 49b9460..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.log.HBaseMarkers;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-
-@InterfaceAudience.Private
-public class ReplicationPeerZKImpl extends ReplicationStateZKBase
-    implements ReplicationPeer, Abortable, Closeable {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerZKImpl.class);
-
-  private volatile ReplicationPeerConfig peerConfig;
-  private final String id;
-  private volatile PeerState peerState;
-  private volatile Map<TableName, List<String>> tableCFs = new HashMap<>();
-  private final Configuration conf;
-
-  private final List<ReplicationPeerConfigListener> peerConfigListeners;
-
-  /**
-   * Constructor that takes all the objects required to communicate with the specified peer, except
-   * for the region server addresses.
-   * @param conf configuration object to this peer
-   * @param id string representation of this peer's identifier
-   * @param peerConfig configuration for the replication peer
-   */
-  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf, String id,
-      ReplicationPeerConfig peerConfig, Abortable abortable) throws ReplicationException {
-    super(zkWatcher, conf, abortable);
-    this.conf = conf;
-    this.peerConfig = peerConfig;
-    this.id = id;
-    this.peerConfigListeners = new ArrayList<>();
-  }
-
-  private PeerState readPeerState() throws ReplicationException {
-    try {
-      byte[] data = ZKUtil.getData(zookeeper, this.getPeerStateNode(id));
-      this.peerState = isStateEnabled(data) ? PeerState.ENABLED : PeerState.DISABLED;
-    } catch (DeserializationException | KeeperException | InterruptedException e) {
-      throw new ReplicationException("Get and deserialize peer state data from zookeeper failed: ",
-          e);
-    }
-    return this.peerState;
-  }
-
-  private ReplicationPeerConfig readPeerConfig() throws ReplicationException {
-    try {
-      byte[] data = ZKUtil.getData(zookeeper, this.getPeerNode(id));
-      if (data != null) {
-        this.peerConfig = ReplicationPeerConfigUtil.parsePeerFrom(data);
-      }
-    } catch (DeserializationException | KeeperException | InterruptedException e) {
-      throw new ReplicationException("Get and deserialize peer config date from zookeeper failed: ",
-          e);
-    }
-    return this.peerConfig;
-  }
-
-  @Override
-  public PeerState getPeerState() {
-    return peerState;
-  }
-
-  @Override
-  public PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException {
-    if (loadFromBackingStore) {
-      return readPeerState();
-    } else {
-      return peerState;
-    }
-  }
-
-  /**
-   * Get the identifier of this peer
-   * @return string representation of the id (short)
-   */
-  @Override
-  public String getId() {
-    return id;
-  }
-
-  /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
-  @Override
-  public ReplicationPeerConfig getPeerConfig() {
-    return peerConfig;
-  }
-
-  @Override
-  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
-      throws ReplicationException {
-    if (loadFromBackingStore) {
-      return readPeerConfig();
-    } else {
-      return peerConfig;
-    }
-  }
-
-  /**
-   * Get the configuration object required to communicate with this peer
-   * @return configuration object
-   */
-  @Override
-  public Configuration getConfiguration() {
-    return conf;
-  }
-
-  /**
-   * Get replicable (table, cf-list) map of this peer
-   * @return the replicable (table, cf-list) map
-   */
-  @Override
-  public Map<TableName, List<String>> getTableCFs() {
-    this.tableCFs = peerConfig.getTableCFsMap();
-    return this.tableCFs;
-  }
-
-  /**
-   * Get replicable namespace set of this peer
-   * @return the replicable namespaces set
-   */
-  @Override
-  public Set<String> getNamespaces() {
-    return this.peerConfig.getNamespaces();
-  }
-
-  @Override
-  public long getPeerBandwidth() {
-    return this.peerConfig.getBandwidth();
-  }
-
-  @Override
-  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
-    this.peerConfigListeners.add(listener);
-  }
-
-  @Override
-  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig) {
-    for (ReplicationPeerConfigListener listener : this.peerConfigListeners) {
-      listener.peerConfigUpdated(newPeerConfig);
-    }
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    LOG.error(HBaseMarkers.FATAL, "The ReplicationPeer corresponding to peer " +
-        peerConfig + " was aborted for the following reason(s):" + why, e);
-  }
-
-  @Override
-  public boolean isAborted() {
-    // Currently the replication peer is never "Aborted", we just log when the
-    // abort method is called.
-    return false;
-  }
-
-  @Override
-  public void close() throws IOException {
-    // TODO: stop zkw?
-  }
-
-  /**
-   * Parse the raw data from ZK to get a peer's state
-   * @param bytes raw ZK data
-   * @return True if the passed in <code>bytes</code> are those of a pb serialized ENABLED state.
-   * @throws DeserializationException
-   */
-  public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
-    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
-    return ReplicationProtos.ReplicationState.State.ENABLED == state;
-  }
-
-  /**
-   * @param bytes Content of a state znode.
-   * @return State parsed from the passed bytes.
-   * @throws DeserializationException
-   */
-  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
-      throws DeserializationException {
-    ProtobufUtil.expectPBMagicPrefix(bytes);
-    int pbLen = ProtobufUtil.lengthOfPBMagic();
-    ReplicationProtos.ReplicationState.Builder builder =
-        ReplicationProtos.ReplicationState.newBuilder();
-    ReplicationProtos.ReplicationState state;
-    try {
-      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
-      state = builder.build();
-      return state.getState();
-    } catch (IOException e) {
-      throw new DeserializationException(e);
-    }
-  }
-}

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

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

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

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


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

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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 6c947a3baade25c5875bab0cd27639c1aa4859bc
Parents: 0381e83
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Jan 4 16:58:01 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:43:05 2018 +0800

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


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c947a3b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
index 3e17025..604e0bb 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -28,6 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 public class ReplicationPeerImpl implements ReplicationPeer {
+
   private final Configuration conf;
 
   private final String id;

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c947a3b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index 1830103..08178f4 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
@@ -38,8 +37,6 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @Category({ ReplicationTests.class, MediumTests.class })
 public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
@@ -48,8 +45,6 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestReplicationStateZKImpl.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateZKImpl.class);
-
   private static Configuration conf;
   private static HBaseZKTestingUtility utility;
   private static ZKWatcher zkw;
@@ -97,20 +92,4 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
   public static void tearDownAfterClass() throws Exception {
     utility.shutdownMiniZKCluster();
   }
-
-  private static class WarnOnlyAbortable implements Abortable {
-
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.warn("TestReplicationStateZKImpl received abort, ignoring.  Reason: " + why);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(e.toString(), e);
-      }
-    }
-
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c947a3b/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 977dd15..23ba773 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c947a3b/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 58dfb4f..b6388fe 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
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
-import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@@ -187,11 +186,7 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
    */
   @Override
   public void startReplicationService() throws IOException {
-    try {
-      this.replicationManager.init();
-    } catch (ReplicationException e) {
-      throw new IOException(e);
-    }
+    this.replicationManager.init();
     this.replicationSink = new ReplicationSink(this.conf, this.server);
     this.scheduleThreadPool.scheduleAtFixedRate(
       new ReplicationStatisticsTask(this.replicationSink, this.replicationManager),
@@ -210,9 +205,9 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
       throws IOException {
     try {
       this.replicationManager.addHFileRefs(tableName, family, pairs);
-    } catch (ReplicationException e) {
+    } catch (IOException e) {
       LOG.error("Failed to add hfile references in the replication queue.", e);
-      throw new IOException(e);
+      throw e;
     }
   }
 

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

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

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


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: f4c9da403e500969f22b86357fac94c6d24ebe35
Parents: 9631c6a
Author: huzheng <op...@gmail.com>
Authored: Fri Dec 29 15:55:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:42:08 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/f4c9da40/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 6c66aff..2a970ba 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -33,9 +33,8 @@ public class ReplicationFactory {
     return new ReplicationPeers(zk, conf);
   }
 
-  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,
-      final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
+  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, Abortable abortable,
       Stoppable stopper) {
-    return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, abortable, stopper);
+    return new ReplicationTrackerZKImpl(zookeeper, abortable, stopper);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f4c9da40/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
deleted file mode 100644
index f49537c..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * This is a base class for maintaining replication state in zookeeper.
- */
-@InterfaceAudience.Private
-public abstract class ReplicationStateZKBase {
-
-  /**
-   * The name of the znode that contains the replication status of a remote slave (i.e. peer)
-   * cluster.
-   */
-  protected final String peerStateNodeName;
-  /** The name of the base znode that contains all replication state. */
-  protected final String replicationZNode;
-  /** The name of the znode that contains a list of all remote slave (i.e. peer) clusters. */
-  protected final String peersZNode;
-  /** The name of the znode that contains all replication queues */
-  protected final String queuesZNode;
-  /** The name of the znode that contains queues of hfile references to be replicated */
-  protected final String hfileRefsZNode;
-  /** The cluster key of the local cluster */
-  protected final String ourClusterKey;
-  /** The name of the znode that contains tableCFs */
-  protected final String tableCFsNodeName;
-
-  protected final ZKWatcher zookeeper;
-  protected final Configuration conf;
-  protected final Abortable abortable;
-
-  public static final byte[] ENABLED_ZNODE_BYTES =
-      toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
-  public static final byte[] DISABLED_ZNODE_BYTES =
-      toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
-  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
-      "zookeeper.znode.replication.hfile.refs";
-  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
-
-  public ReplicationStateZKBase(ZKWatcher zookeeper, Configuration conf,
-                                Abortable abortable) {
-    this.zookeeper = zookeeper;
-    this.conf = conf;
-    this.abortable = abortable;
-
-    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
-    String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
-    String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
-    String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
-      ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
-    this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
-    this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
-    this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
-    this.replicationZNode = ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode,
-      replicationZNodeName);
-    this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);
-    this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName);
-    this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName);
-  }
-
-  public List<String> getListOfReplicators() {
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of replicators", e);
-    }
-    return result;
-  }
-
-  /**
-   * @param state
-   * @return Serialized protobuf of <code>state</code> with pb magic prefix prepended suitable for
-   *         use as content of a peer-state znode under a peer cluster id as in
-   *         /hbase/replication/peers/PEER_ID/peer-state.
-   */
-  protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) {
-    ReplicationProtos.ReplicationState msg =
-        ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
-    // There is no toByteArray on this pb Message?
-    // 32 bytes is default which seems fair enough here.
-    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
-      CodedOutputStream cos = CodedOutputStream.newInstance(baos, 16);
-      msg.writeTo(cos);
-      cos.flush();
-      baos.flush();
-      return ProtobufUtil.prependPBMagic(baos.toByteArray());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  protected boolean peerExists(String id) throws KeeperException {
-    return ZKUtil.checkExists(this.zookeeper, ZNodePaths.joinZNode(this.peersZNode, id)) >= 0;
-  }
-
-  /**
-   * Determine if a ZK path points to a peer node.
-   * @param path path to be checked
-   * @return true if the path points to a peer node, otherwise false
-   */
-  protected boolean isPeerPath(String path) {
-    return path.split("/").length == peersZNode.split("/").length + 1;
-  }
-
-  @VisibleForTesting
-  protected String getTableCFsNode(String id) {
-    return ZNodePaths.joinZNode(this.peersZNode, ZNodePaths.joinZNode(id, this.tableCFsNodeName));
-  }
-
-  @VisibleForTesting
-  protected String getPeerStateNode(String id) {
-    return ZNodePaths.joinZNode(this.peersZNode, ZNodePaths.joinZNode(id, this.peerStateNodeName));
-  }
-  @VisibleForTesting
-  protected String getPeerNode(String id) {
-    return ZNodePaths.joinZNode(this.peersZNode, id);
-  }
-}

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/f4c9da40/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 9244774..58dfb4f 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
@@ -110,8 +110,7 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
           ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf);
       this.replicationPeers.init();
       this.replicationTracker =
-          ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.replicationPeers,
-            this.conf, this.server, this.server);
+          ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.server, this.server);
     } catch (Exception e) {
       throw new IOException("Failed replication handler create", e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f4c9da40/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index d128a41..9b61aa0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -328,8 +328,7 @@ public class HBaseFsck extends Configured implements Closeable {
    * @throws MasterNotRunningException if the master is not running
    * @throws ZooKeeperConnectionException if unable to connect to ZooKeeper
    */
-  public HBaseFsck(Configuration conf) throws MasterNotRunningException,
-      ZooKeeperConnectionException, IOException, ClassNotFoundException {
+  public HBaseFsck(Configuration conf) throws IOException, ClassNotFoundException {
     this(conf, createThreadPool(conf));
   }
 

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/f4c9da40/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
index b1fe2e9..dcb6bb1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java
@@ -29,14 +29,13 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -59,12 +58,19 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
 
   private static ZKWatcher zkw = null;
   private static Abortable abortable = null;
+  private static ZKStorageUtil zkStorageUtil = null;
+
+  private static class ZKStorageUtil extends ZKReplicationPeerStorage {
+    public ZKStorageUtil(ZKWatcher zookeeper, Configuration conf) {
+      super(zookeeper, conf);
+    }
+  }
 
   @Rule
   public TestName name = new TestName();
 
   public TestTableCFsUpdater() {
-    super(zkw, TEST_UTIL.getConfiguration(), abortable);
+    super(zkw, TEST_UTIL.getConfiguration());
   }
 
   @BeforeClass
@@ -83,6 +89,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
       }
     };
     zkw = new ZKWatcher(conf, "TableCFs", abortable, true);
+    zkStorageUtil = new ZKStorageUtil(zkw, conf);
   }
 
   @AfterClass
@@ -91,8 +98,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
   }
 
   @Test
-  public void testUpgrade() throws KeeperException, InterruptedException,
-      DeserializationException {
+  public void testUpgrade() throws Exception {
     String peerId = "1";
     final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1");
     final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2");
@@ -100,13 +106,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
 
     ReplicationPeerConfig rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(zkw.getQuorum());
-    String peerNode = getPeerNode(peerId);
+    String peerNode = zkStorageUtil.getPeerNode(peerId);
     ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc));
 
     String tableCFs = tableName1 + ":cf1,cf2;" + tableName2 + ":cf3;" + tableName3;
     String tableCFsNode = getTableCFsNode(peerId);
     LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
-    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+    ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs));
 
     ReplicationPeerConfig actualRpc =
         ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
@@ -119,13 +125,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     peerId = "2";
     rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(zkw.getQuorum());
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc));
 
     tableCFs = tableName1 + ":cf1,cf3;" + tableName2 + ":cf2";
     tableCFsNode = getTableCFsNode(peerId);
     LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
-    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+    ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs));
 
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
@@ -137,13 +143,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     peerId = "3";
     rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(zkw.getQuorum());
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc));
 
     tableCFs = "";
     tableCFsNode = getTableCFsNode(peerId);
     LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
-    ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
+    ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs));
 
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
@@ -155,7 +161,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     peerId = "4";
     rpc = new ReplicationPeerConfig();
     rpc.setClusterKey(zkw.getQuorum());
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc));
 
     tableCFsNode = getTableCFsNode(peerId);
@@ -169,7 +175,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     copyTableCFs();
 
     peerId = "1";
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
     Map<TableName, List<String>> tableNameListMap = actualRpc.getTableCFsMap();
@@ -184,9 +190,8 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     assertEquals("cf3", tableNameListMap.get(tableName2).get(0));
     assertNull(tableNameListMap.get(tableName3));
 
-
     peerId = "2";
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
     tableNameListMap = actualRpc.getTableCFsMap();
@@ -200,19 +205,17 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader {
     assertEquals("cf2", tableNameListMap.get(tableName2).get(0));
 
     peerId = "3";
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
     tableNameListMap = actualRpc.getTableCFsMap();
     assertNull(tableNameListMap);
 
     peerId = "4";
-    peerNode = getPeerNode(peerId);
+    peerNode = zkStorageUtil.getPeerNode(peerId);
     actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
     assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
     tableNameListMap = actualRpc.getTableCFsMap();
     assertNull(tableNameListMap);
   }
-
-
 }

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


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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c947a3b/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 79621fc..a53cba3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -322,7 +321,7 @@ public abstract class TestReplicationSourceManager {
     wal.rollWriter();
 
     manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
-        "1", 0, false, false);
+        "1", 0, false);
 
     wal.append(hri,
         new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),


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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index a5cd9b2..01a230d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,22 +36,19 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
- * In a scenario of Replication based Disaster/Recovery, when hbase
- * Master-Cluster crashes, this tool is used to sync-up the delta from Master to
- * Slave using the info from ZooKeeper. The tool will run on Master-Cluser, and
- * assume ZK, Filesystem and NetWork still available after hbase crashes
+ * In a scenario of Replication based Disaster/Recovery, when hbase Master-Cluster crashes, this
+ * tool is used to sync-up the delta from Master to Slave using the info from ZooKeeper. The tool
+ * will run on Master-Cluser, and assume ZK, Filesystem and NetWork still available after hbase
+ * crashes
  *
+ * <pre>
  * hbase org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp
+ * </pre>
  */
-
 public class ReplicationSyncUp extends Configured implements Tool {
 
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationSyncUp.class.getName());
-
   private static Configuration conf;
 
   private static final long SLEEP_TIME = 10000;
@@ -106,13 +102,14 @@ public class ReplicationSyncUp extends Configured implements Tool {
     replication = new Replication();
     replication.initialize(new DummyServer(zkw), fs, logDir, oldLogDir, null);
     manager = replication.getReplicationManager();
-    manager.init();
+    manager.init().get();
 
     try {
-      int numberOfOldSource = 1; // default wait once
-      while (numberOfOldSource > 0) {
+      while (manager.activeFailoverTaskCount() > 0) {
+        Thread.sleep(SLEEP_TIME);
+      }
+      while (manager.getOldSources().size() > 0) {
         Thread.sleep(SLEEP_TIME);
-        numberOfOldSource = manager.getOldSources().size();
       }
     } catch (InterruptedException e) {
       System.err.println("didn't wait long enough:" + e);
@@ -122,7 +119,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
     manager.join();
     zkw.close();
 
-    return (0);
+    return 0;
   }
 
   static class DummyServer implements Server {

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/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 102c8d9..e1eb822 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
@@ -48,8 +48,8 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -81,16 +81,13 @@ public class TestReplicationHFileCleaner {
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicationHFileCleaner.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Server server;
-  private static ReplicationQueues rq;
+  private static ReplicationQueueStorage rq;
   private static ReplicationPeers rp;
   private static final String peerId = "TestReplicationHFileCleaner";
   private static Configuration conf = TEST_UTIL.getConfiguration();
   static FileSystem fs = null;
   Path root;
 
-  /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniZKCluster();
@@ -99,20 +96,10 @@ public class TestReplicationHFileCleaner {
     HMaster.decorateMasterConfiguration(conf);
     rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf, server);
     rp.init();
-    rq = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, server, server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
-    try {
-      fs = FileSystem.get(conf);
-    } finally {
-      if (fs != null) {
-        fs.close();
-      }
-    }
+    rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
+    fs = FileSystem.get(conf);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniZKCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/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 22ed1a3..4bcde0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
@@ -26,10 +26,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -55,14 +53,12 @@ public class TestReplicationZKNodeCleaner {
 
   private final Configuration conf;
   private final ZKWatcher zkw;
-  private final ReplicationQueues repQueues;
+  private final ReplicationQueueStorage repQueues;
 
   public TestReplicationZKNodeCleaner() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     zkw = new ZKWatcher(conf, "TestReplicationZKNodeCleaner", null);
-    repQueues = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null,
-        zkw));
-    assertTrue(repQueues instanceof ReplicationQueuesZKImpl);
+    repQueues = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
   }
 
   @BeforeClass
@@ -78,9 +74,8 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleaner() throws Exception {
-    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
-    repQueues.addLog(ID_ONE, "file1");
+    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
 
     ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
     Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
@@ -90,7 +85,7 @@ public class TestReplicationZKNodeCleaner {
     assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE));
 
     // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addLog(ID_TWO + "-" + SERVER_TWO, "file2");
+    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
 
     undeletedQueues = cleaner.getUnDeletedQueues();
     assertEquals(1, undeletedQueues.size());
@@ -106,11 +101,10 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleanerChore() throws Exception {
-    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
-    repQueues.addLog(ID_ONE, "file1");
+    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
     // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addLog(ID_TWO + "-" + SERVER_TWO, "file2");
+    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
 
     // Wait the cleaner chore to run
     Thread.sleep(20000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index 7ea79f9..14c5e56 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -46,9 +45,10 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   MetricsSource metrics;
   WALFileLengthProvider walFileLengthProvider;
   AtomicBoolean startup = new AtomicBoolean(false);
+
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues rq, ReplicationPeers rp, Server server, String peerClusterId,
+      ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
       UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     this.manager = manager;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
index 1672390..19aeac1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -69,7 +69,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
 
   @Before
   public void setUp() throws Exception {
-
     HColumnDescriptor fam;
 
     t1_syncupSource = new HTableDescriptor(t1_su);
@@ -105,7 +104,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
    * check's gone Also check the puts and deletes are not replicated back to
    * the originating cluster.
    */
-  @Test(timeout = 300000)
+  @Test
   public void testSyncUpTool() throws Exception {
 
     /**
@@ -181,7 +180,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
      * verify correctly replicated to Slave
      */
     mimicSyncUpAfterPut();
-
   }
 
   protected void setupReplication() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/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 d1a9f83..1d3b6ea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -68,10 +68,10 @@ import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -338,18 +338,14 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testClaimQueues() throws Exception {
-    final Server server = new DummyServer("hostname0.example.org");
-
-
-    ReplicationQueues rq =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-          server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname0.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     files.add("log1");
     files.add("log2");
     for (String file : files) {
-      rq.addLog("1", file);
+      rq.addWAL(server.getServerName(), "1", file);
     }
     // create 3 DummyServers
     Server s1 = new DummyServer("dummyserver1.example.org");
@@ -357,12 +353,9 @@ public abstract class TestReplicationSourceManager {
     Server s3 = new DummyServer("dummyserver3.example.org");
 
     // create 3 DummyNodeFailoverWorkers
-    DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s1);
-    DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s2);
-    DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s3);
+    DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(server.getServerName(), s1);
+    DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(server.getServerName(), s2);
+    DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(server.getServerName(), s3);
 
     latch = new CountDownLatch(3);
     // start the threads
@@ -381,11 +374,9 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testCleanupFailoverQueues() throws Exception {
-    final Server server = new DummyServer("hostname1.example.org");
-    ReplicationQueues rq =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-          server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname1.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     SortedSet<String> files = new TreeSet<>();
     String group = "testgroup";
@@ -394,19 +385,14 @@ public abstract class TestReplicationSourceManager {
     files.add(file1);
     files.add(file2);
     for (String file : files) {
-      rq.addLog("1", file);
+      rq.addWAL(server.getServerName(), "1", file);
     }
     Server s1 = new DummyServer("dummyserver1.example.org");
-    ReplicationQueues rq1 =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1,
-            s1.getZooKeeper()));
-    rq1.init(s1.getServerName().toString());
     ReplicationPeers rp1 =
         ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration(), s1);
     rp1.init();
     NodeFailoverWorker w1 =
-        manager.new NodeFailoverWorker(server.getServerName().getServerName(), rq1, rp1, new UUID(
-            new Long(1), new Long(2)));
+        manager.new NodeFailoverWorker(server.getServerName());
     w1.run();
     assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
     String id = "1-" + server.getServerName().getServerName();
@@ -418,17 +404,16 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testCleanupUnknownPeerZNode() throws Exception {
-    final Server server = new DummyServer("hostname2.example.org");
-    ReplicationQueues rq = ReplicationFactory.getReplicationQueues(
-      new ReplicationQueuesArguments(server.getConfiguration(), server, server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname2.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     // add log to an unknown peer
     String group = "testgroup";
-    rq.addLog("2", group + ".log1");
-    rq.addLog("2", group + ".log2");
+    rq.addWAL(server.getServerName(), "2", group + ".log1");
+    rq.addWAL(server.getServerName(), "2", group + ".log2");
 
-    NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName().getServerName());
+    NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName());
     w1.run();
 
     // The log of the unknown peer should be removed from zk
@@ -506,10 +491,8 @@ public abstract class TestReplicationSourceManager {
         .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase");
     try {
       DummyServer server = new DummyServer();
-      final ReplicationQueues rq =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(
-              server.getConfiguration(), server, server.getZooKeeper()));
-      rq.init(server.getServerName().toString());
+      ReplicationQueueStorage rq = ReplicationStorageFactory
+          .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
       // Purposely fail ReplicationSourceManager.addSource() by causing ReplicationSourceInterface
       // initialization to throw an exception.
       conf.set("replication.replicationsource.implementation",
@@ -523,11 +506,11 @@ public abstract class TestReplicationSourceManager {
       assertNull(manager.getSource(peerId));
 
       // Create a replication queue for the fake peer
-      rq.addLog(peerId, "FakeFile");
+      rq.addWAL(server.getServerName(), peerId, "FakeFile");
       // Unregister peer, this should remove the peer and clear all queues associated with it
       // Need to wait for the ReplicationTracker to pick up the changes and notify listeners.
       removePeerAndWait(peerId);
-      assertFalse(rq.getAllQueues().contains(peerId));
+      assertFalse(rq.getAllQueues(server.getServerName()).contains(peerId));
     } finally {
       conf.set("replication.replicationsource.implementation", replicationSourceImplName);
       removePeerAndWait(peerId);
@@ -650,11 +633,12 @@ public abstract class TestReplicationSourceManager {
       }
     }
     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
-      @Override public boolean evaluate() throws Exception {
+      @Override
+      public boolean evaluate() throws Exception {
         List<String> peers = rp.getAllPeerIds();
-        return (!manager.getAllQueues().contains(peerId)) && (rp.getConnectedPeer(peerId) == null)
-            && (!peers.contains(peerId))
-            && manager.getSource(peerId) == null;
+        return (!manager.getAllQueues().contains(peerId)) &&
+          (rp.getConnectedPeer(peerId) == null) && (!peers.contains(peerId)) &&
+          manager.getSource(peerId) == null;
       }
     });
   }
@@ -697,25 +681,24 @@ public abstract class TestReplicationSourceManager {
   static class DummyNodeFailoverWorker extends Thread {
     private Map<String, Set<String>> logZnodesMap;
     Server server;
-    private String deadRsZnode;
-    ReplicationQueues rq;
+    private ServerName deadRS;
+    ReplicationQueueStorage rq;
 
-    public DummyNodeFailoverWorker(String znode, Server s) throws Exception {
-      this.deadRsZnode = znode;
+    public DummyNodeFailoverWorker(ServerName deadRS, Server s) throws Exception {
+      this.deadRS = deadRS;
       this.server = s;
-      this.rq =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-            server.getZooKeeper()));
-      this.rq.init(this.server.getServerName().toString());
+      this.rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(),
+        server.getConfiguration());
     }
 
     @Override
     public void run() {
       try {
         logZnodesMap = new HashMap<>();
-        List<String> queues = rq.getUnClaimedQueueIds(deadRsZnode);
-        for(String queue:queues){
-          Pair<String, SortedSet<String>> pair = rq.claimQueue(deadRsZnode, queue);
+        List<String> queues = rq.getAllQueues(deadRS);
+        for (String queue : queues) {
+          Pair<String, SortedSet<String>> pair =
+              rq.claimQueue(deadRS, queue, server.getServerName());
           if (pair != null) {
             logZnodesMap.put(pair.getFirst(), pair.getSecond());
           }
@@ -754,7 +737,7 @@ public abstract class TestReplicationSourceManager {
 
     @Override
     public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-        ReplicationQueues rq, ReplicationPeers rp, Server server, String peerClusterId,
+        ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
         UUID clusterId, ReplicationEndpoint replicationEndpoint,
         WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
       throw new IOException("Failing deliberately");

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


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: c85716fc3cc240abdd77a993ed66d7a6f7a26cac
Parents: a4d0440
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 27 22:03:51 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:41:30 2018 +0800

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


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
deleted file mode 100644
index 7f440b1..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import java.util.List;
-import java.util.SortedSet;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * This provides an interface for maintaining a region server's replication queues. These queues
- * keep track of the WALs and HFile references (if hbase.replication.bulkload.enabled is enabled)
- * that still need to be replicated to remote clusters.
- */
-@InterfaceAudience.Private
-public interface ReplicationQueues {
-
-  /**
-   * Initialize the region server replication queue interface.
-   * @param serverName The server name of the region server that owns the replication queues this
-   *          interface manages.
-   */
-  void init(String serverName) throws ReplicationException;
-
-  /**
-   * Remove a replication queue.
-   * @param queueId a String that identifies the queue.
-   */
-  void removeQueue(String queueId);
-
-  /**
-   * Add a new WAL file to the given queue. If the queue does not exist it is created.
-   * @param queueId a String that identifies the queue.
-   * @param filename name of the WAL
-   */
-  void addLog(String queueId, String filename) throws ReplicationException;
-
-  /**
-   * Remove an WAL file from the given queue.
-   * @param queueId a String that identifies the queue.
-   * @param filename name of the WAL
-   */
-  void removeLog(String queueId, String filename);
-
-  /**
-   * Set the current position for a specific WAL in a given queue.
-   * @param queueId a String that identifies the queue
-   * @param filename name of the WAL
-   * @param position the current position in the file
-   */
-  void setLogPosition(String queueId, String filename, long position);
-
-  /**
-   * Get the current position for a specific WAL in a given queue.
-   * @param queueId a String that identifies the queue
-   * @param filename name of the WAL
-   * @return the current position in the file
-   */
-  long getLogPosition(String queueId, String filename) throws ReplicationException;
-
-  /**
-   * Remove all replication queues for this region server.
-   */
-  void removeAllQueues();
-
-  /**
-   * Get a list of all WALs in the given queue.
-   * @param queueId a String that identifies the queue
-   * @return a list of WALs, null if no such queue exists for this server
-   */
-  List<String> getLogsInQueue(String queueId);
-
-  /**
-   * Get a list of all queues for this region server.
-   * @return a list of queueIds, an empty list if this region server is dead and has no outstanding queues
-   */
-  List<String> getAllQueues();
-
-  /**
-   * Get queueIds from a dead region server, whose queues has not been claimed by other region
-   * servers.
-   * @return empty if the queue exists but no children, null if the queue does not exist.
-  */
-  List<String> getUnClaimedQueueIds(String regionserver);
-
-  /**
-   * Take ownership for the queue identified by queueId and belongs to a dead region server.
-   * @param regionserver the id of the dead region server
-   * @param queueId the id of the queue
-   * @return the new PeerId and A SortedSet of WALs in its queue, and null if no unclaimed queue.
-   */
-  Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId);
-
-  /**
-   * Remove the znode of region server if the queue is empty.
-   * @param regionserver
-   */
-  void removeReplicatorIfQueueIsEmpty(String regionserver);
-
-  /**
-   * Get a list of all region servers that have outstanding replication queues. These servers could
-   * be alive, dead or from a previous run of the cluster.
-   * @return a list of server names
-   */
-  List<String> getListOfReplicators();
-
-  /**
-   * Checks if the provided znode is the same as this region server's
-   * @param regionserver the id of the region server
-   * @return if this is this rs's znode
-   */
-  boolean isThisOurRegionServer(String regionserver);
-
-  /**
-   * Add a peer to hfile reference queue if peer does not exist.
-   * @param peerId peer cluster id to be added
-   * @throws ReplicationException if fails to add a peer id to hfile reference queue
-   */
-  void addPeerToHFileRefs(String peerId) throws ReplicationException;
-
-  /**
-   * Remove a peer from hfile reference queue.
-   * @param peerId peer cluster id to be removed
-   */
-  void removePeerFromHFileRefs(String peerId);
-
-  /**
-   * Add new hfile references to the queue.
-   * @param peerId peer cluster id to which the hfiles need to be replicated
-   * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which
-   *          will be added in the queue }
-   * @throws ReplicationException if fails to add a hfile reference
-   */
-  void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs) throws ReplicationException;
-
-  /**
-   * Remove hfile references from the queue.
-   * @param peerId peer cluster id from which this hfile references needs to be removed
-   * @param files list of hfile references to be removed
-   */
-  void removeHFileRefs(String peerId, List<String> files);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
deleted file mode 100644
index c2a5df3..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Wrapper around common arguments used to construct ReplicationQueues. Used to construct various
- * ReplicationQueues Implementations with different constructor arguments by reflection.
- */
-@InterfaceAudience.Private
-public class ReplicationQueuesArguments {
-
-  private ZKWatcher zk;
-  private Configuration conf;
-  private Abortable abort;
-
-  public ReplicationQueuesArguments(Configuration conf, Abortable abort) {
-    this.conf = conf;
-    this.abort = abort;
-  }
-
-  public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZKWatcher zk) {
-    this(conf, abort);
-    setZk(zk);
-  }
-
-  public ZKWatcher getZk() {
-    return zk;
-  }
-
-  public void setZk(ZKWatcher zk) {
-    this.zk = zk;
-  }
-
-  public Configuration getConf() {
-    return conf;
-  }
-
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  public Abortable getAbortable() {
-    return abort;
-  }
-
-  public void setAbortable(Abortable abort) {
-    this.abort = abort;
-  }
-}

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
index 2c522f6..5659e4b 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -54,6 +53,8 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
     super(zookeeper, conf, abortable);
     this.stopper = stopper;
     this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper));
+    // watch the changes
+    refreshOtherRegionServersList(true);
   }
 
   @Override
@@ -71,7 +72,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    */
   @Override
   public List<String> getListOfRegionServers() {
-    refreshOtherRegionServersList();
+    refreshOtherRegionServersList(false);
 
     List<String> list = null;
     synchronized (otherRegionServers) {
@@ -137,7 +138,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
       if (!path.startsWith(this.watcher.znodePaths.rsZNode)) {
         return false;
       }
-      return refreshOtherRegionServersList();
+      return refreshOtherRegionServersList(true);
     }
   }
 
@@ -157,8 +158,8 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    * @return true if the local list of the other region servers was updated with the ZK data (even
    *         if it was empty), false if the data was missing in ZK
    */
-  private boolean refreshOtherRegionServersList() {
-    List<String> newRsList = getRegisteredRegionServers();
+  private boolean refreshOtherRegionServersList(boolean watch) {
+    List<String> newRsList = getRegisteredRegionServers(watch);
     if (newRsList == null) {
       return false;
     } else {
@@ -174,10 +175,14 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    * Get a list of all the other region servers in this cluster and set a watch
    * @return a list of server nanes
    */
-  private List<String> getRegisteredRegionServers() {
+  private List<String> getRegisteredRegionServers(boolean watch) {
     List<String> result = null;
     try {
-      result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      if (watch) {
+        result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      } else {
+        result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      }
     } catch (KeeperException e) {
       this.abortable.abort("Get list of registered region servers", e);
     }

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/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 6571ba0..26fa6b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -38,8 +37,8 @@ import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -53,7 +52,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
-
 /**
  * Gateway to Replication. Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  */
@@ -63,7 +61,7 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
       LoggerFactory.getLogger(Replication.class);
   private boolean isReplicationForBulkLoadDataEnabled;
   private ReplicationSourceManager replicationManager;
-  private ReplicationQueues replicationQueues;
+  private ReplicationQueueStorage queueStorage;
   private ReplicationPeers replicationPeers;
   private ReplicationTracker replicationTracker;
   private Configuration conf;
@@ -106,10 +104,8 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
     }
 
     try {
-      this.replicationQueues =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, this.server,
-            server.getZooKeeper()));
-      this.replicationQueues.init(this.server.getServerName().toString());
+      this.queueStorage =
+          ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
       this.replicationPeers =
           ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server);
       this.replicationPeers.init();
@@ -125,8 +121,8 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
     } catch (KeeperException ke) {
       throw new IOException("Could not read cluster id", ke);
     }
-    this.replicationManager = new ReplicationSourceManager(replicationQueues, replicationPeers,
-        replicationTracker, conf, this.server, fs, logDir, oldLogDir, clusterId,
+    this.replicationManager = new ReplicationSourceManager(queueStorage, replicationPeers, replicationTracker, conf,
+      this.server, fs, logDir, oldLogDir, clusterId,
         walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty());
     if (walProvider != null) {
       walProvider

http://git-wip-us.apache.org/repos/asf/hbase/blob/c85716fc/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 a2eb202..58ea6ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -31,7 +30,6 @@ import java.util.concurrent.PriorityBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,7 +47,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -83,7 +81,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   private Map<String, PriorityBlockingQueue<Path>> queues = new HashMap<>();
   // per group queue size, keep no more than this number of logs in each wal group
   protected int queueSizePerGroup;
-  protected ReplicationQueues replicationQueues;
+  protected ReplicationQueueStorage queueStorage;
   private ReplicationPeers replicationPeers;
 
   protected Configuration conf;
@@ -148,7 +146,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
    */
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server,
+      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
       String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     this.server = server;
@@ -161,7 +159,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     this.maxRetriesMultiplier =
         this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per
     this.queueSizePerGroup = this.conf.getInt("hbase.regionserver.maxlogs", 32);
-    this.replicationQueues = replicationQueues;
+    this.queueStorage = queueStorage;
     this.replicationPeers = replicationPeers;
     this.manager = manager;
     this.fs = fs;
@@ -230,7 +228,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       List<String> tableCfs = tableCFMap.get(tableName);
       if (tableCFMap.containsKey(tableName)
           && (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) {
-        this.replicationQueues.addHFileRefs(peerId, pairs);
+        this.queueStorage.addHFileRefs(peerId, pairs);
         metrics.incrSizeOfHFileRefsQueue(pairs.size());
       } else {
         LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
@@ -239,7 +237,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     } else {
       // user has explicitly not defined any table cfs for replication, means replicate all the
       // data
-      this.replicationQueues.addHFileRefs(peerId, pairs);
+      this.queueStorage.addHFileRefs(peerId, pairs);
       metrics.incrSizeOfHFileRefsQueue(pairs.size());
     }
   }

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


[06/46] hbase git commit: HBASE-19928 TestVisibilityLabelsOnNewVersionBehaviorTable fails; ADDENDUM Fix failing TestMetaWithReplicas#testShutdownHandling; it was reading meta TableState

Posted by zh...@apache.org.
HBASE-19928 TestVisibilityLabelsOnNewVersionBehaviorTable fails; ADDENDUM Fix failing TestMetaWithReplicas#testShutdownHandling; it was reading meta TableState


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: fbcb453ce2bbe4ffe0fa723b2ae05a7f91a6fc5d
Parents: cb138c2
Author: Michael Stack <st...@apache.org>
Authored: Sat Feb 3 21:47:59 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 21:48:18 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fbcb453c/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index f80bbc0..5dc0565 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1109,6 +1109,9 @@ public class MetaTableAccessor {
   @Nullable
   public static TableState getTableState(Connection conn, TableName tableName)
       throws IOException {
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      return new TableState(tableName, TableState.State.ENABLED);
+    }
     Table metaHTable = getMetaHTable(conn);
     Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getTableStateColumn());
     long time = EnvironmentEdgeManager.currentTime();


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: a4d04406e372e9b35f2bdf9f9ed0e48e5fc7b9b8
Parents: 3b27510
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 27 18:27:13 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: d97e83af9737eb7fafa4acf23adbd07af7bc19f2
Parents: 12678c5
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 21 21:59:46 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 0381e83b5cfb6efef0f1ac89dcf7f8e757ab85c6
Parents: 4112552
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 4 16:18:21 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:42:08 2018 +0800

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


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/0381e83b/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 dbfcdc6..44934a6 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
@@ -3459,36 +3459,40 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   }
 
   @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
   public ExecuteProceduresResponse executeProcedures(RpcController controller,
       ExecuteProceduresRequest request) throws ServiceException {
-    if (request.getOpenRegionCount() > 0) {
-      for (OpenRegionRequest req : request.getOpenRegionList()) {
-        openRegion(controller, req);
+    try {
+      checkOpen();
+      regionServer.getRegionServerCoprocessorHost().preExecuteProcedures();
+      if (request.getOpenRegionCount() > 0) {
+        for (OpenRegionRequest req : request.getOpenRegionList()) {
+          openRegion(controller, req);
+        }
       }
-    }
-    if (request.getCloseRegionCount() > 0) {
-      for (CloseRegionRequest req : request.getCloseRegionList()) {
-        closeRegion(controller, req);
+      if (request.getCloseRegionCount() > 0) {
+        for (CloseRegionRequest req : request.getCloseRegionList()) {
+          closeRegion(controller, req);
+        }
       }
-    }
-    if (request.getProcCount() > 0) {
-      for (RemoteProcedureRequest req : request.getProcList()) {
-        RSProcedureCallable callable;
-        try {
-          callable =
-            Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance();
-        } catch (Exception e) {
-          // here we just ignore the error as this should not happen and we do not provide a general
-          // way to report errors for all types of remote procedure. The procedure will hang at
-          // master side but after you solve the problem and restart master it will be executed
-          // again and pass.
-          LOG.warn("create procedure of type " + req.getProcClass() + " failed, give up", e);
-          continue;
+      if (request.getProcCount() > 0) {
+        for (RemoteProcedureRequest req : request.getProcList()) {
+          RSProcedureCallable callable;
+          try {
+            callable =
+              Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance();
+          } catch (Exception e) {
+            regionServer.remoteProcedureComplete(req.getProcId(), e);
+            continue;
+          }
+          callable.init(req.getProcData().toByteArray(), regionServer);
+          regionServer.executeProcedure(req.getProcId(), callable);
         }
-        callable.init(req.getProcData().toByteArray(), regionServer);
-        regionServer.executeProcedure(req.getProcId(), callable);
       }
+      regionServer.getRegionServerCoprocessorHost().postExecuteProcedures();
+      return ExecuteProceduresResponse.getDefaultInstance();
+    } catch (IOException e) {
+      throw new ServiceException(e);
     }
-    return ExecuteProceduresResponse.getDefaultInstance();
   }
 }

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/0381e83b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 3a4bf82..1fbf01d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -117,13 +117,6 @@ import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
-import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker;
-import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -136,6 +129,14 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
+import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker;
+import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+
 /**
  * Provides basic authorization checks for data access and administrative
  * operations.
@@ -2455,7 +2456,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       throws IOException {
     requirePermission(ctx, "replicateLogEntries", Action.WRITE);
   }
-
+  
   @Override
   public void  preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
           throws IOException {
@@ -2507,8 +2508,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
 
   @Override
   public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
-      TableName tableName, RegionInfo[] regionInfos, String description)
-  throws IOException {
+      TableName tableName, RegionInfo[] regionInfos, String description) throws IOException {
     // There are operations in the CREATE and ADMIN domain which may require lock, READ
     // or WRITE. So for any lock request, we check for these two perms irrespective of lock type.
     String reason = String.format("Description=%s", description);
@@ -2521,12 +2521,18 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
     checkLockPermissions(ctx, null, tableName, null, description);
   }
 
+  @Override
+  public void preExecuteProcedures(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
+      throws IOException {
+    checkSystemOrSuperUser(getActiveUser(ctx));
+  }
+
   /**
    * Returns the active user to which authorization checks should be applied.
    * If we are in the context of an RPC call, the remote user is used,
    * otherwise the currently logged in user is used.
    */
-  public User getActiveUser(ObserverContext<?> ctx) throws IOException {
+  private User getActiveUser(ObserverContext<?> ctx) throws IOException {
     // for non-rpc handling, fallback to system user
     Optional<User> optionalUser = ctx.getCaller();
     if (optionalUser.isPresent()) {

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/0381e83b/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 abb0ddd..cb9ccdd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -30,7 +30,6 @@ import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
@@ -38,7 +37,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -3074,4 +3072,18 @@ public class TestAccessController extends SecureTestUtil {
     verifyAllowed(
         action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
   }
+
+  @Test
+  public void testExecuteProcedures() throws Exception {
+    AccessTestAction action = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preExecuteProcedures(ObserverContextImpl.createAndPrepare(RSCP_ENV));
+        return null;
+      }
+    };
+
+    verifyAllowed(action, SUPERUSER);
+    verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER, USER_ADMIN);
+  }
 }


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 3fc2f85765ef145614fff15c85949fed8e584536
Parents: ab52775
Author: huzheng <op...@gmail.com>
Authored: Tue Dec 26 16:46:10 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:41:30 2018 +0800

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


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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fc2f857/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 5e70e57..6c66aff 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -29,14 +29,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
-      Abortable abortable) {
-    return getReplicationPeers(zk, conf, null, abortable);
-  }
-
-  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
-      ReplicationQueueStorage queueStorage, Abortable abortable) {
-    return new ReplicationPeersZKImpl(zk, conf, queueStorage, abortable);
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf) {
+    return new ReplicationPeers(zk, conf);
   }
 
   public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fc2f857/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index 73e600e..27bda2d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -310,7 +310,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
 
     queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
     replicationPeers =
-        ReplicationFactory.getReplicationPeers(zkw, getConf(), queueStorage, connection);
+        ReplicationFactory.getReplicationPeers(zkw, getConf());
     replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
       new WarnOnlyAbortable(), new WarnOnlyStoppable());
     Set<String> liveRegionServers = new HashSet<>(replicationTracker.getListOfRegionServers());

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fc2f857/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 26fa6b1..9244774 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
@@ -107,7 +107,7 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
       this.queueStorage =
           ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
       this.replicationPeers =
-          ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server);
+          ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf);
       this.replicationPeers.init();
       this.replicationTracker =
           ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.replicationPeers,

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fc2f857/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 58ea6ee..8250992 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
@@ -222,8 +222,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       // A peerId will not have "-" in its name, see HBASE-11394
       peerId = peerClusterZnode.split("-")[0];
     }
-    Map<TableName, List<String>> tableCFMap =
-        replicationPeers.getConnectedPeer(peerId).getTableCFs();
+    Map<TableName, List<String>> tableCFMap = replicationPeers.getPeer(peerId).getTableCFs();
     if (tableCFMap != null) {
       List<String> tableCfs = tableCFMap.get(tableName);
       if (tableCFMap.containsKey(tableName)
@@ -371,7 +370,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   }
 
   private long getCurrentBandwidth() {
-    ReplicationPeer replicationPeer = this.replicationPeers.getConnectedPeer(peerId);
+    ReplicationPeer replicationPeer = this.replicationPeers.getPeer(peerId);
     long peerBandwidth = replicationPeer != null ? replicationPeer.getPeerBandwidth() : 0;
     // user can set peer bandwidth to 0 to use default bandwidth
     return peerBandwidth != 0 ? peerBandwidth : defaultBandwidth;
@@ -416,7 +415,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
    */
   @Override
   public boolean isPeerEnabled() {
-    return this.replicationPeers.getStatusOfPeer(this.peerId);
+    return this.replicationPeers.isPeerEnabled(this.peerId);
   }
 
   @Override


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 12d321d4c26474a515bb11351e699b26a50381fe
Parents: a46f2a9
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 22 14:37:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/12d321d4/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/12d321d4/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index edce309..77517cc 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -104,6 +104,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/12d321d4/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/12d321d4/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/12d321d4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index 05bbc84..f49537c 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -63,7 +63,6 @@ public abstract class ReplicationStateZKBase {
   protected final Configuration conf;
   protected final Abortable abortable;
 
-  // Public for testing
   public static final byte[] ENABLED_ZNODE_BYTES =
       toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
   public static final byte[] DISABLED_ZNODE_BYTES =

http://git-wip-us.apache.org/repos/asf/hbase/blob/12d321d4/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/12d321d4/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/12d321d4/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/12d321d4/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/12d321d4/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/12d321d4/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/12d321d4/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 20ae7ee..caea3ee 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
@@ -40,6 +40,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -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;
@@ -330,7 +331,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
@@ -721,8 +722,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);
@@ -740,7 +741,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();
@@ -785,9 +786,8 @@ public class HMaster extends HRegionServer implements MasterServices {
    * <li>Handle either fresh cluster start or master failover</li>
    * </ol>
    */
-  private void finishActiveMasterInitialization(MonitoredTask status)
-      throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
-
+  private void finishActiveMasterInitialization(MonitoredTask status) throws IOException,
+      InterruptedException, KeeperException, CoordinatedStateException, ReplicationException {
     Thread zombieDetector = new Thread(new InitializationMonitor(this),
         "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
     zombieDetector.setDaemon(true);
@@ -3391,18 +3391,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
@@ -3421,7 +3422,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);
     }
@@ -3571,8 +3573,8 @@ 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/12d321d4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index e798455..9d371bd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -459,9 +459,9 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
-   * Returns the {@link ReplicationManager}.
+   * Returns the {@link ReplicationPeerManager}.
    */
-  ReplicationManager getReplicationManager();
+  ReplicationPeerManager getReplicationPeerManager();
 
   /**
    * Update the peerConfig for the specified peer

http://git-wip-us.apache.org/repos/asf/hbase/blob/12d321d4/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/12d321d4/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/12d321d4/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/12d321d4/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/12d321d4/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/12d321d4/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/46] 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/a46f2a9e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a46f2a9e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a46f2a9e

Branch: refs/heads/HBASE-19397-branch-2
Commit: a46f2a9eb64dfc878a01cc9851d145fdbb72eee0
Parents: d97e83a
Author: huzheng <op...@gmail.com>
Authored: Wed Dec 20 10:47:18 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a46f2a9e/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/a46f2a9e/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 15ee06d..977dd15 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;
 
 /**
@@ -27,4 +28,9 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public interface ReplicationSourceService extends ReplicationService {
+
+  /**
+   * Returns a Handler to handle peer procedures.
+   */
+  PeerProcedureHandler getPeerProcedureHandler();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a46f2a9e/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/a46f2a9e/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 42d0299..56576a6 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/a46f2a9e/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/a46f2a9e/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/a46f2a9e/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/a46f2a9e/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 ad12c66..6571ba0 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
@@ -76,6 +76,8 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
   // ReplicationLoad to access replication metrics
   private ReplicationLoad replicationLoad;
 
+  private PeerProcedureHandler peerProcedureHandler;
+
   /**
    * Empty constructor
    */
@@ -134,6 +136,13 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
     LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
     this.replicationLoad = new ReplicationLoad();
+
+    this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager);
+  }
+
+  @Override
+  public PeerProcedureHandler getPeerProcedureHandler() {
+    return peerProcedureHandler;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a46f2a9e/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 9db3cc2..a2eb202 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
@@ -446,12 +446,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/a46f2a9e/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 cbbfca0..0460280 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
@@ -568,6 +568,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).
@@ -615,6 +627,8 @@ public class ReplicationSourceManager implements ReplicationListener {
       }
       deleteSource(id, true);
     }
+    // Remove HFile Refs znode from zookeeper
+    this.replicationQueues.removePeerFromHFileRefs(id);
   }
 
   @Override
@@ -622,29 +636,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/a46f2a9e/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/a46f2a9e/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/a46f2a9e/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/a46f2a9e/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 69500a1..de7e768 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
@@ -18,7 +18,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;
@@ -160,41 +159,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;
@@ -226,21 +190,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/a46f2a9e/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 eec8e8a..d1a9f83 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
@@ -185,6 +185,12 @@ public abstract class TestReplicationSourceManager {
     replication = new Replication();
     replication.initialize(new DummyServer(), fs, logDir, oldLogDir, null);
     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) {
@@ -560,18 +566,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);
@@ -581,9 +585,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);
     }
@@ -600,8 +603,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);
     }
   }
@@ -634,6 +643,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 {


[39/46] hbase git commit: HBASE-19748 TestRegionReplicaFailover and TestRegionReplicaReplicationEndpoint UT hangs

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: d5ddc7eb1959917cb16d3b374c2dd77764dfcac2
Parents: 6fb48ee
Author: huzheng <op...@gmail.com>
Authored: Wed Jan 10 15:00:30 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:43:05 2018 +0800

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


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


[05/46] hbase git commit: HBASE-19914 Refactor TestVisibilityLabelsOnNewVersionBehaviorTable

Posted by zh...@apache.org.
HBASE-19914 Refactor TestVisibilityLabelsOnNewVersionBehaviorTable


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: cb138c2da84079fc639676e465b97b53faa40764
Parents: a323357
Author: zhangduo <zh...@apache.org>
Authored: Sun Feb 4 08:38:46 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 13:37:09 2018 +0800

----------------------------------------------------------------------
 .../client/ColumnFamilyDescriptorBuilder.java   |    5 +
 ...sibilityLabelsOnNewVersionBehaviorTable.java |   25 +-
 ...ibilityLabelsWithDefaultVisLabelService.java |    4 +-
 .../TestVisibilityLabelsWithDeletes.java        | 1478 +++++++-----------
 .../VisibilityLabelsWithDeletesTestBase.java    |  313 ++++
 5 files changed, 887 insertions(+), 938 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cb138c2d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
index aaa460b..d7feea6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
@@ -552,6 +552,11 @@ public class ColumnFamilyDescriptorBuilder {
     return this;
   }
 
+  public ColumnFamilyDescriptorBuilder setNewVersionBehavior(final boolean value) {
+    desc.setNewVersionBehavior(value);
+    return this;
+  }
+
   public ColumnFamilyDescriptorBuilder setValue(final Bytes key, final Bytes value) {
     desc.setValue(key, value);
     return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb138c2d/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
index d3177f9..4093ace 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
@@ -18,30 +18,33 @@
 package org.apache.hadoop.hbase.security.visibility;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
-@Category({SecurityTests.class, MediumTests.class})
-public class TestVisibilityLabelsOnNewVersionBehaviorTable extends TestVisibilityLabelsWithDeletes {
+
+@Category({ SecurityTests.class, MediumTests.class })
+public class TestVisibilityLabelsOnNewVersionBehaviorTable
+    extends VisibilityLabelsWithDeletesTestBase {
+
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestVisibilityLabelsOnNewVersionBehaviorTable.class);
 
   @Override
-  protected Table createTable(HColumnDescriptor fam) throws IOException {
-    fam.setNewVersionBehavior(true);
-    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    HTableDescriptor table = new HTableDescriptor(tableName);
-    table.addFamily(fam);
-    TEST_UTIL.getHBaseAdmin().createTable(table);
+  protected Table createTable(byte[] fam) throws IOException {
+    TableName tableName = TableName.valueOf(testName.getMethodName());
+    TEST_UTIL.getAdmin()
+        .createTable(TableDescriptorBuilder.newBuilder(tableName)
+            .addColumnFamily(
+              ColumnFamilyDescriptorBuilder.newBuilder(fam).setNewVersionBehavior(true).build())
+            .build());
     return TEST_UTIL.getConnection().getTable(tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb138c2d/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
index fdf665a..9f7df14 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
@@ -114,7 +114,7 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili
     SUPERUSER.runAs(action);
   }
 
-  @Test(timeout = 60 * 1000)
+  @Test
   public void testAddVisibilityLabelsOnRSRestart() throws Exception {
     List<RegionServerThread> regionServerThreads = TEST_UTIL.getHBaseCluster()
         .getRegionServerThreads();
@@ -227,7 +227,7 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili
     SUPERUSER.runAs(action);
   }
 
-  @Test(timeout = 60 * 1000)
+  @Test
   public void testVisibilityLabelsOnWALReplay() throws Exception {
     final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
     try (Table table = createTableAndWriteDataWithLabels(tableName,


[41/46] hbase git commit: HBASE-19719 Fix checkstyle issues

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 6fb48eeda3244317660395e186e8f91d9b2f3553
Parents: 088927d
Author: zhangduo <zh...@apache.org>
Authored: Sat Jan 6 08:30:55 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:43:05 2018 +0800

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


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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fb48eed/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
index d6260bd..ac1acbe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -24,7 +23,6 @@ import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLI
 import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLICATION_ZNODE_DEFAULT;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Admin;
@@ -43,9 +41,11 @@ import org.apache.yetus.audience.InterfaceStability;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
 /**
  * This class is used to upgrade TableCFs from HBase 1.0, 1.1, 1.2, 1.3 to HBase 1.4 or 2.x. It will
  * be removed in HBase 3.x. See HBASE-11393


[08/46] hbase git commit: HBASE-19726 Failed to start HMaster due to infinite retrying on meta assign; ADDENDUM Fix failing TestMetaWithReplicas#testShutdownHandling; it was reading meta TableState"" Retry applying this addendum. Previous commit added it

Posted by zh...@apache.org.
HBASE-19726 Failed to start HMaster due to infinite retrying on meta assign; ADDENDUM Fix failing TestMetaWithReplicas#testShutdownHandling; it was reading meta TableState""
Retry applying this addendum. Previous commit added it w/ wrong log
message so had to revert and then reapply here.


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 0ca7a2e91629c5829144dff92cb1aaf9ebd6de4d
Parents: cd61060
Author: Michael Stack <st...@apache.org>
Authored: Sat Feb 3 21:49:42 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 21:49:42 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0ca7a2e9/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index f80bbc0..5dc0565 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1109,6 +1109,9 @@ public class MetaTableAccessor {
   @Nullable
   public static TableState getTableState(Connection conn, TableName tableName)
       throws IOException {
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      return new TableState(tableName, TableState.State.ENABLED);
+    }
     Table metaHTable = getMetaHTable(conn);
     Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getTableStateColumn());
     long time = EnvironmentEdgeManager.currentTime();


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e8400fd/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/9e8400fd/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 9f3740f..cc57dfb 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.HBaseClassTestRule;
 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.ClassRule;
@@ -61,8 +66,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 {
@@ -70,21 +75,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);
+      }
     }
   }
 
@@ -93,7 +104,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)
@@ -106,19 +117,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/9e8400fd/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 29a577b..6596c6c 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
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterR
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -89,6 +90,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/9e8400fd/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 79d9720..768bd83 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,10 +24,8 @@ 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.conf.Configuration;
@@ -51,7 +49,6 @@ import org.apache.hadoop.hbase.master.HMaster;
 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.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -65,7 +62,6 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -201,24 +197,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/9e8400fd/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 0fc3806..102c8d9 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
@@ -25,7 +25,6 @@ import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -51,7 +50,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.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -68,7 +66,6 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -200,32 +197,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/9e8400fd/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 b0910e2..22ed1a3 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.HBaseClassTestRule;
 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;
@@ -49,9 +49,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;
@@ -78,12 +78,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());
@@ -106,7 +106,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/9e8400fd/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/9e8400fd/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 d51d3c3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ /dev/null
@@ -1,232 +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.HBaseClassTestRule;
-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.ClassRule;
-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 {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestReplicationStateZKImpl.class);
-
-  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/9e8400fd/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 64db7a8..4d81d1d 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
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
@@ -31,8 +29,6 @@ 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;
@@ -119,41 +115,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("");
-  }
 }


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 7bb1768d7955f44ac7eff8d772532dd3789ece3b
Parents: 3fc2f85
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 2 09:57:23 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:41:30 2018 +0800

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


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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb1768d/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
index d09a56b..2321e4f 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
@@ -19,13 +19,13 @@ package org.apache.hadoop.hbase.replication;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 

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

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

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

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

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

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

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

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

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

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

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

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

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


[42/46] hbase git commit: HBASE-19873 addendum add missing rule for new tests

Posted by zh...@apache.org.
HBASE-19873 addendum add missing rule for new tests


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 88c4aa7ada3125ac8704fe24a33e89598be2668f
Parents: 8bb3cf6
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 30 09:40:23 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:43:05 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/TestZKReplicationPeerStorage.java      | 7 ++++++-
 .../hbase/replication/TestZKReplicationQueueStorage.java     | 8 +++++++-
 .../hbase/replication/TestReplicationProcedureRetry.java     | 7 ++++++-
 .../apache/hadoop/hbase/util/TestHBaseFsckReplication.java   | 6 ++++++
 4 files changed, 25 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/88c4aa7a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
index 3eb11da..3290fb0 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
@@ -33,19 +33,24 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.stream.Stream;
-
+import org.apache.hadoop.hbase.HBaseClassTestRule;
 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.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ ReplicationTests.class, MediumTests.class })
 public class TestZKReplicationPeerStorage {
 
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestZKReplicationPeerStorage.class);
+
   private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
 
   private static ZKReplicationPeerStorage STORAGE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/88c4aa7a/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 786730f..2c01a26 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
@@ -27,8 +27,8 @@ 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.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -38,11 +38,17 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ ReplicationTests.class, MediumTests.class })
 public class TestZKReplicationQueueStorage {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestZKReplicationQueueStorage.class);
+
   private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
 
   private static ZKReplicationQueueStorage STORAGE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/88c4aa7a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
index ab35b46..a2ae0b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
@@ -27,8 +27,8 @@ import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Admin;
@@ -40,6 +40,7 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.invocation.InvocationOnMock;
@@ -50,6 +51,10 @@ import org.mockito.invocation.InvocationOnMock;
 @Category({ ReplicationTests.class, MediumTests.class })
 public class TestReplicationProcedureRetry {
 
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestReplicationProcedureRetry.class);
+
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/88c4aa7a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
index e64255c..8911982 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.List;
 import java.util.stream.Stream;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -33,12 +34,17 @@ import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ ReplicationTests.class, MediumTests.class })
 public class TestHBaseFsckReplication {
 
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestHBaseFsckReplication.class);
+
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
   @BeforeClass


[09/46] hbase git commit: HBASE-19926 Use a separated class to implement the WALActionListener for Replication

Posted by zh...@apache.org.
HBASE-19926 Use a separated class to implement the WALActionListener for Replication


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 3b603d2c08c1f1905a589597737412b43970a304
Parents: 0ca7a2e
Author: zhangduo <zh...@apache.org>
Authored: Sun Feb 4 10:42:33 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:32:14 2018 +0800

----------------------------------------------------------------------
 .../replication/regionserver/Replication.java   | 22 +----
 .../regionserver/ReplicationSourceManager.java  | 47 +---------
 .../ReplicationSourceWALActionListener.java     | 98 ++++++++++++++++++++
 .../TestReplicationSourceManager.java           | 30 ++----
 4 files changed, 108 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3b603d2c/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 0274b0a..ad12c66 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
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
@@ -44,8 +43,6 @@ import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -130,23 +127,8 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
         replicationTracker, conf, this.server, fs, logDir, oldLogDir, clusterId,
         walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty());
     if (walProvider != null) {
-      walProvider.addWALActionsListener(new WALActionsListener() {
-
-        @Override
-        public void preLogRoll(Path oldPath, Path newPath) throws IOException {
-          replicationManager.preLogRoll(newPath);
-        }
-
-        @Override
-        public void postLogRoll(Path oldPath, Path newPath) throws IOException {
-          replicationManager.postLogRoll(newPath);
-        }
-
-        @Override
-        public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
-          replicationManager.scopeWALEdits(logKey, logEdit);
-        }
-      });
+      walProvider
+        .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager));
     }
     this.statsThreadPeriod =
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b603d2c/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 8543896..cbbfca0 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
@@ -40,12 +40,9 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
@@ -64,21 +61,16 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
-import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-
 /**
  * This class is responsible to manage all the replication
  * sources. There are two classes of sources:
@@ -471,43 +463,6 @@ public class ReplicationSourceManager implements ReplicationListener {
     return totalBufferUsed;
   }
 
-  void scopeWALEdits(WALKey logKey, WALEdit logEdit) throws IOException {
-    scopeWALEdits(logKey, logEdit, this.conf);
-  }
-
-  /**
-   * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
-   * compaction WAL edits and if the scope is local.
-   * @param logKey Key that may get scoped according to its edits
-   * @param logEdit Edits used to lookup the scopes
-   * @throws IOException If failed to parse the WALEdit
-   */
-  @VisibleForTesting
-  static void scopeWALEdits(WALKey logKey, WALEdit logEdit, Configuration conf) throws IOException {
-    boolean replicationForBulkLoadEnabled =
-      ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf);
-    boolean foundOtherEdits = false;
-    for (Cell cell : logEdit.getCells()) {
-      if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
-        foundOtherEdits = true;
-        break;
-      }
-    }
-
-    if (!foundOtherEdits && logEdit.getCells().size() > 0) {
-      WALProtos.RegionEventDescriptor maybeEvent =
-        WALEdit.getRegionEventDescriptor(logEdit.getCells().get(0));
-      if (maybeEvent != null &&
-        (maybeEvent.getEventType() == WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE)) {
-        // In serially replication, we use scopes when reading close marker.
-        foundOtherEdits = true;
-      }
-    }
-    if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
-      ((WALKeyImpl) logKey).serializeReplicationScope(false);
-    }
-  }
-
   /**
    * Factory method to create a replication source
    * @param conf the configuration to use

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b603d2c/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
new file mode 100644
index 0000000..eb12614
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed 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.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.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
+/**
+ * Used to receive new wals.
+ */
+@InterfaceAudience.Private
+class ReplicationSourceWALActionListener implements WALActionsListener {
+
+  private final Configuration conf;
+
+  private final ReplicationSourceManager manager;
+
+  public ReplicationSourceWALActionListener(Configuration conf, ReplicationSourceManager manager) {
+    this.conf = conf;
+    this.manager = manager;
+  }
+
+  @Override
+  public void preLogRoll(Path oldPath, Path newPath) throws IOException {
+    manager.preLogRoll(newPath);
+  }
+
+  @Override
+  public void postLogRoll(Path oldPath, Path newPath) throws IOException {
+    manager.postLogRoll(newPath);
+  }
+
+  @Override
+  public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
+    scopeWALEdits(logKey, logEdit, conf);
+  }
+
+  /**
+   * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
+   * compaction WAL edits and if the scope is local.
+   * @param logKey Key that may get scoped according to its edits
+   * @param logEdit Edits used to lookup the scopes
+   * @throws IOException If failed to parse the WALEdit
+   */
+  @VisibleForTesting
+  static void scopeWALEdits(WALKey logKey, WALEdit logEdit, Configuration conf) throws IOException {
+    boolean replicationForBulkLoadEnabled =
+        ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf);
+    boolean foundOtherEdits = false;
+    for (Cell cell : logEdit.getCells()) {
+      if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+        foundOtherEdits = true;
+        break;
+      }
+    }
+
+    if (!foundOtherEdits && logEdit.getCells().size() > 0) {
+      WALProtos.RegionEventDescriptor maybeEvent =
+          WALEdit.getRegionEventDescriptor(logEdit.getCells().get(0));
+      if (maybeEvent != null &&
+        (maybeEvent.getEventType() == WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE)) {
+        // In serially replication, we use scopes when reading close marker.
+        foundOtherEdits = true;
+      }
+    }
+    if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
+      ((WALKeyImpl) logKey).serializeReplicationScope(false);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b603d2c/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 647e12d..eec8e8a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -40,7 +40,6 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -65,7 +64,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -84,7 +82,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
@@ -100,8 +97,10 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
@@ -270,23 +269,8 @@ public abstract class TestReplicationSourceManager {
     WALFactory wals =
       new WALFactory(utility.getConfiguration(), URLEncoder.encode("regionserver:60020", "UTF8"));
     ReplicationSourceManager replicationManager = replication.getReplicationManager();
-    wals.getWALProvider().addWALActionsListener(new WALActionsListener() {
-
-      @Override
-      public void preLogRoll(Path oldPath, Path newPath) throws IOException {
-        replicationManager.preLogRoll(newPath);
-      }
-
-      @Override
-      public void postLogRoll(Path oldPath, Path newPath) throws IOException {
-        replicationManager.postLogRoll(newPath);
-      }
-
-      @Override
-      public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
-        replicationManager.scopeWALEdits(logKey, logEdit);
-      }
-    });
+    wals.getWALProvider()
+      .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager));
     final WAL wal = wals.getWAL(hri);
     manager.init();
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame"));
@@ -459,7 +443,7 @@ public abstract class TestReplicationSourceManager {
     RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).setStartKey(HConstants.EMPTY_START_ROW)
         .setEndKey(HConstants.EMPTY_END_ROW).build();
     WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
-    ReplicationSourceManager.scopeWALEdits(new WALKeyImpl(), edit, conf);
+    ReplicationSourceWALActionListener.scopeWALEdits(new WALKeyImpl(), edit, conf);
   }
 
   @Test
@@ -471,7 +455,7 @@ public abstract class TestReplicationSourceManager {
     WALKeyImpl logKey = new WALKeyImpl(scope);
 
     // 3. Get the scopes for the key
-    ReplicationSourceManager.scopeWALEdits(logKey, logEdit, conf);
+    ReplicationSourceWALActionListener.scopeWALEdits(logKey, logEdit, conf);
 
     // 4. Assert that no bulk load entry scopes are added if bulk load hfile replication is disabled
     assertNull("No bulk load entries scope should be added if bulk load replication is disabled.",
@@ -490,7 +474,7 @@ public abstract class TestReplicationSourceManager {
     bulkLoadConf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
 
     // 4. Get the scopes for the key
-    ReplicationSourceManager.scopeWALEdits(logKey, logEdit, bulkLoadConf);
+    ReplicationSourceWALActionListener.scopeWALEdits(logKey, logEdit, bulkLoadConf);
 
     NavigableMap<byte[], Integer> scopes = logKey.getReplicationScopes();
     // Assert family with replication scope global is present in the key scopes


[04/46] hbase git commit: HBASE-19914 Refactor TestVisibilityLabelsOnNewVersionBehaviorTable

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/cb138c2d/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java
index 0e69a3a..7a5232a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.security.visibility;
 
-import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -27,17 +26,13 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -48,251 +43,64 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
-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;
 
-/**
- * Tests visibility labels with deletes
- */
-@Category({SecurityTests.class, MediumTests.class})
-public class TestVisibilityLabelsWithDeletes {
+@Category({ SecurityTests.class, LargeTests.class })
+public class TestVisibilityLabelsWithDeletes extends VisibilityLabelsWithDeletesTestBase {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestVisibilityLabelsWithDeletes.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestVisibilityLabelsWithDeletes.class);
-  private static final String TOPSECRET = "TOPSECRET";
-  private static final String PUBLIC = "PUBLIC";
-  private static final String PRIVATE = "PRIVATE";
-  private static final String CONFIDENTIAL = "CONFIDENTIAL";
-  private static final String SECRET = "SECRET";
-  public static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final byte[] row1 = Bytes.toBytes("row1");
-  private static final byte[] row2 = Bytes.toBytes("row2");
-  protected final static byte[] fam = Bytes.toBytes("info");
-  protected final static byte[] qual = Bytes.toBytes("qual");
-  private final static byte[] qual1 = Bytes.toBytes("qual1");
-  private final static byte[] qual2 = Bytes.toBytes("qual2");
-  protected final static byte[] value = Bytes.toBytes("value");
-  private final static byte[] value1 = Bytes.toBytes("value1");
-  public static Configuration conf;
-
-  @Rule
-  public final TestName TEST_NAME = new TestName();
-  public static User SUPERUSER;
-
-  @BeforeClass
-  public static void setupBeforeClass() throws Exception {
-    // setup configuration
-    conf = TEST_UTIL.getConfiguration();
-    VisibilityTestUtil.enableVisiblityLabels(conf);
-    conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
-        ScanLabelGenerator.class);
-    conf.set("hbase.superuser", "admin");
-    TEST_UTIL.startMiniCluster(2);
-    SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
-
-    // Wait for the labels table to become available
-    TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000);
-    addLabels();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
+    HBaseClassTestRule.forClass(TestVisibilityLabelsWithDeletes.class);
 
-  @After
-  public void tearDown() throws Exception {
-  }
-
-  protected Table createTable(HColumnDescriptor fam) throws IOException {
-    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    HTableDescriptor table = new HTableDescriptor(tableName);
-    table.addFamily(fam);
-    TEST_UTIL.getHBaseAdmin().createTable(table);
+  @Override
+  protected Table createTable(byte[] fam) throws IOException {
+    TableName tableName = TableName.valueOf(testName.getMethodName());
+    TEST_UTIL.getAdmin().createTable(TableDescriptorBuilder.newBuilder(tableName)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build());
     return TEST_UTIL.getConnection().getTable(tableName);
   }
 
-  @Test
-  public void testVisibilityLabelsWithDeleteColumns() throws Throwable {
-    setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-
-    try (Table table = createTableAndWriteDataWithLabels(SECRET + "&" + TOPSECRET, SECRET)) {
-      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
-            Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility(TOPSECRET + "&" + SECRET));
-            d.addColumns(fam, qual);
-            table.delete(d);
-          } catch (Throwable t) {
-            throw new IOException(t);
-          }
-          return null;
-        }
-      };
-      SUPERUSER.runAs(actiona);
-
-      TEST_UTIL.getAdmin().flush(tableName);
-      Scan s = new Scan();
-      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
-      ResultScanner scanner = table.getScanner(s);
-      Result[] next = scanner.next(3);
-      assertTrue(next.length == 1);
-      CellScanner cellScanner = next[0].cellScanner();
-      cellScanner.advance();
-      Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
-
-    }
+  private TableName createTable() throws IOException {
+    return createTable(-1);
   }
 
-  @Test
-  public void testVisibilityLabelsWithDeleteFamily() throws Exception {
-    setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    try (Table table = createTableAndWriteDataWithLabels(SECRET, CONFIDENTIAL + "|" + TOPSECRET);) {
-      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
-            Delete d = new Delete(row2);
-            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
-            d.addFamily(fam);
-            table.delete(d);
-          } catch (Throwable t) {
-            throw new IOException(t);
-          }
-          return null;
-        }
-      };
-      SUPERUSER.runAs(actiona);
-
-      TEST_UTIL.getAdmin().flush(tableName);
-      Scan s = new Scan();
-      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
-      ResultScanner scanner = table.getScanner(s);
-      Result[] next = scanner.next(3);
-      assertTrue(next.length == 1);
-      CellScanner cellScanner = next[0].cellScanner();
-      cellScanner.advance();
-      Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
-    }
+  private TableName createTable(int maxVersions) throws IOException {
+    TableName tableName = TableName.valueOf(testName.getMethodName());
+    createTable(tableName, maxVersions);
+    return tableName;
   }
 
-  @Test
-  public void testVisibilityLabelsWithDeleteFamilyVersion() throws Exception {
-    setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    long[] ts = new long[] { 123L, 125L };
-    try (Table table = createTableAndWriteDataWithLabels(ts,
-        CONFIDENTIAL + "|" + TOPSECRET, SECRET)) {
-      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
-            Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
-            d.addFamilyVersion(fam, 123L);
-            table.delete(d);
-          } catch (Throwable t) {
-            throw new IOException(t);
-          }
-          return null;
-        }
-      };
-      SUPERUSER.runAs(actiona);
-
-      TEST_UTIL.getAdmin().flush(tableName);
-      Scan s = new Scan();
-      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
-      ResultScanner scanner = table.getScanner(s);
-      Result[] next = scanner.next(3);
-      assertTrue(next.length == 1);
-      CellScanner cellScanner = next[0].cellScanner();
-      cellScanner.advance();
-      Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
-    }
-  }
-
-  @Test
-  public void testVisibilityLabelsWithDeleteColumnExactVersion() throws Exception {
-    setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    long[] ts = new long[] { 123L, 125L };
-    try (Table table = createTableAndWriteDataWithLabels(ts,
-        CONFIDENTIAL + "|" + TOPSECRET, SECRET);) {
-      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
-            Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
-            d.addColumn(fam, qual, 123L);
-            table.delete(d);
-          } catch (Throwable t) {
-            throw new IOException(t);
-          }
-          return null;
-        }
-      };
-      SUPERUSER.runAs(actiona);
-
-      TEST_UTIL.getAdmin().flush(tableName);
-      Scan s = new Scan();
-      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
-      ResultScanner scanner = table.getScanner(s);
-      Result[] next = scanner.next(3);
-      assertTrue(next.length == 1);
-      CellScanner cellScanner = next[0].cellScanner();
-      cellScanner.advance();
-      Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+  private void createTable(TableName tableName, int maxVersions) throws IOException {
+    ColumnFamilyDescriptorBuilder builder = ColumnFamilyDescriptorBuilder.newBuilder(fam);
+    if (maxVersions > 0) {
+      builder.setMaxVersions(maxVersions);
     }
+    TEST_UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(builder.build()).build());
   }
 
   @Test
   public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
-                SECRET + "&" + TOPSECRET+")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addColumns(fam, qual, 125L);
             table.delete(d);
           } catch (Throwable t) {
@@ -305,7 +113,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -313,24 +121,24 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
@@ -338,14 +146,14 @@ public class TestVisibilityLabelsWithDeletes {
   public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersionsNoTimestamp()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d1 = new Delete(row1);
             d1.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d1.addColumns(fam, qual);
@@ -358,8 +166,8 @@ public class TestVisibilityLabelsWithDeletes {
             table.delete(d2);
 
             Delete d3 = new Delete(row1);
-            d3.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + SECRET + "&" + TOPSECRET + ")"));
+            d3.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d3.addColumns(fam, qual);
             table.delete(d3);
           } catch (Throwable t) {
@@ -370,7 +178,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -378,24 +186,23 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
-  public void
-    testVisibilityLabelsWithDeleteColumnsWithNoMatchVisExpWithMultipleVersionsNoTimestamp()
+  public void testVisibilityLabelsWithDeleteColumnsNoMatchVisExpWithMultipleVersionsNoTimestamp()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addColumns(fam, qual);
@@ -407,8 +214,8 @@ public class TestVisibilityLabelsWithDeletes {
             table.delete(d);
 
             d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + SECRET + "&" + TOPSECRET + ")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addColumns(fam, qual);
             table.delete(d);
           } catch (Throwable t) {
@@ -419,7 +226,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -427,13 +234,13 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
@@ -441,14 +248,14 @@ public class TestVisibilityLabelsWithDeletes {
   public void testVisibilityLabelsWithDeleteFamilyWithMultipleVersionsNoTimestamp()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d1 = new Delete(row1);
             d1.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d1.addFamily(fam);
@@ -460,8 +267,8 @@ public class TestVisibilityLabelsWithDeletes {
             table.delete(d2);
 
             Delete d3 = new Delete(row1);
-            d3.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + SECRET + "&" + TOPSECRET + ")"));
+            d3.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d3.addFamily(fam);
             table.delete(d3);
           } catch (Throwable t) {
@@ -472,7 +279,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -480,19 +287,14 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteColumnsWithoutAndWithVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -506,7 +308,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -527,7 +329,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -544,12 +346,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testDeleteColumnsWithAndWithoutVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -564,7 +361,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -584,7 +381,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -601,12 +398,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testDeleteFamiliesWithoutAndWithVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -620,7 +412,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -641,7 +433,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -658,12 +450,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testDeleteFamiliesWithAndWithoutVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -678,7 +465,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -698,7 +485,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -715,12 +502,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testDeletesWithoutAndWithVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -734,7 +516,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             // The delete would not be able to apply it because of visibility mismatch
@@ -756,7 +538,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -774,13 +556,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testVisibilityLabelsWithDeleteFamilyWithPutsReAppearing() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual, value);
@@ -795,7 +571,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addFamily(fam);
@@ -808,7 +584,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -821,7 +597,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET));
             d.addFamily(fam);
@@ -834,13 +610,13 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL));
       scanner = table.getScanner(s);
       next = scanner.next(3);
       assertEquals(1, next.length);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET));
       scanner = table.getScanner(s);
       Result[] next1 = scanner.next(3);
@@ -850,13 +626,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testVisibilityLabelsWithDeleteColumnsWithPutsReAppearing() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual, value);
@@ -871,7 +641,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addColumns(fam, qual);
@@ -884,7 +654,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -897,7 +667,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET));
             d.addColumns(fam, qual);
@@ -910,13 +680,13 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL));
       scanner = table.getScanner(s);
       next = scanner.next(3);
       assertEquals(1, next.length);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET));
       scanner = table.getScanner(s);
       Result[] next1 = scanner.next(3);
@@ -926,13 +696,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testVisibilityCombinations() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual, 123L, value);
@@ -946,7 +710,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET));
             d.addColumns(fam, qual, 126L);
@@ -956,7 +720,7 @@ public class TestVisibilityLabelsWithDeletes {
           }
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addColumn(fam, qual, 123L);
@@ -969,24 +733,18 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL, SECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
       assertEquals(0, next.length);
     }
   }
+
   @Test
   public void testVisibilityLabelsWithDeleteColumnWithSpecificVersionWithPutsReAppearing()
       throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
-
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put1 = new Put(Bytes.toBytes("row1"));
       put1.addColumn(fam, qual, 123L, value);
@@ -998,7 +756,7 @@ public class TestVisibilityLabelsWithDeletes {
       table.put(createList(put1, put2));
 
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL, SECRET));
 
       ResultScanner scanner = table.getScanner(s);
@@ -1009,7 +767,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addColumn(fam, qual, 123L);
@@ -1017,7 +775,7 @@ public class TestVisibilityLabelsWithDeletes {
           }
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET));
             d.addColumn(fam, qual, 123L);
@@ -1030,7 +788,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL));
       scanner = table.getScanner(s);
       assertEquals(0, scanner.next(3).length);
@@ -1039,11 +797,10 @@ public class TestVisibilityLabelsWithDeletes {
   }
 
   @Test
-  public void
-    testVisibilityLabelsWithDeleteFamilyWithNoMatchingVisExpWithMultipleVersionsNoTimestamp()
+  public void testVisibilityLabelsWithDeleteFamilyNoMatchingVisExpWithMultipleVersionsNoTimestamp()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@@ -1058,12 +815,12 @@ public class TestVisibilityLabelsWithDeletes {
           d2.addFamily(fam);
 
           Delete d3 = new Delete(row1);
-          d3.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-              + SECRET + "&" + TOPSECRET + ")"));
+          d3.setCellVisibility(new CellVisibility(
+              "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
           d3.addFamily(fam);
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(createList(d1, d2, d3));
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1073,7 +830,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1081,13 +838,13 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
       scanner.close();
     }
   }
@@ -1095,7 +852,7 @@ public class TestVisibilityLabelsWithDeletes {
   @Test
   public void testDeleteFamilyAndDeleteColumnsWithAndWithoutVisibilityExp() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@@ -1108,7 +865,7 @@ public class TestVisibilityLabelsWithDeletes {
           d2.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
           d2.addColumns(fam, qual);
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(createList(d1, d2));
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1118,7 +875,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1126,41 +883,36 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
       scanner.close();
     }
   }
 
   private Table doPuts(TableName tableName) throws IOException, InterruptedIOException,
       RetriesExhaustedWithDetailsException, InterruptedException {
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    createTable(tableName, 5);
 
     List<Put> puts = new ArrayList<>(5);
     Put put = new Put(Bytes.toBytes("row1"));
@@ -1170,8 +922,8 @@ public class TestVisibilityLabelsWithDeletes {
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 124L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-    + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     put = new Put(Bytes.toBytes("row1"));
@@ -1181,21 +933,21 @@ public class TestVisibilityLabelsWithDeletes {
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 126L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-        + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 127L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-        + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     TEST_UTIL.getAdmin().flush(tableName);
     put = new Put(Bytes.toBytes("row2"));
     put.addColumn(fam, qual, 127L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET
-        + "&" + SECRET + ")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     Table table = TEST_UTIL.getConnection().getTable(tableName);
@@ -1203,14 +955,9 @@ public class TestVisibilityLabelsWithDeletes {
     return table;
   }
 
-  private Table doPutsWithDiffCols(TableName tableName) throws IOException,
-      InterruptedIOException, RetriesExhaustedWithDetailsException, InterruptedException {
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+  private Table doPutsWithDiffCols(TableName tableName) throws IOException, InterruptedIOException,
+      RetriesExhaustedWithDetailsException, InterruptedException {
+    createTable(tableName, 5);
 
     List<Put> puts = new ArrayList<>(5);
     Put put = new Put(Bytes.toBytes("row1"));
@@ -1220,8 +967,8 @@ public class TestVisibilityLabelsWithDeletes {
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 124L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-    + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     put = new Put(Bytes.toBytes("row1"));
@@ -1236,8 +983,8 @@ public class TestVisibilityLabelsWithDeletes {
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual2, 127L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-        + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     Table table = TEST_UTIL.getConnection().getTable(tableName);
@@ -1247,12 +994,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   private Table doPutsWithoutVisibility(TableName tableName) throws IOException,
       InterruptedIOException, RetriesExhaustedWithDetailsException, InterruptedException {
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    createTable(tableName, 5);
     List<Put> puts = new ArrayList<>(5);
     Put put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 123L, value);
@@ -1286,22 +1028,21 @@ public class TestVisibilityLabelsWithDeletes {
     return table;
   }
 
-
   @Test
   public void testDeleteColumnWithSpecificTimeStampUsingMultipleVersionsUnMatchingVisExpression()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
-                SECRET + "&" + TOPSECRET+")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addColumn(fam, qual, 125L);
             table.delete(d);
           } catch (Throwable t) {
@@ -1314,7 +1055,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1322,48 +1063,48 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteColumnWithLatestTimeStampUsingMultipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
             d.addColumn(fam, qual);
@@ -1378,7 +1119,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1386,36 +1127,36 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
-  @Test (timeout=180000)
+  @Test(timeout = 180000)
   public void testDeleteColumnWithLatestTimeStampWhenNoVersionMatches() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       Put put = new Put(Bytes.toBytes("row1"));
@@ -1426,9 +1167,9 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility(SECRET ));
+            d.setCellVisibility(new CellVisibility(SECRET));
             d.addColumn(fam, qual);
             table.delete(d);
           } catch (Throwable t) {
@@ -1441,7 +1182,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1449,34 +1190,34 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(128L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
 
       put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual, 129L, value);
@@ -1485,7 +1226,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       scanner = table.getScanner(s);
       next = scanner.next(3);
@@ -1493,23 +1234,24 @@ public class TestVisibilityLabelsWithDeletes {
       cellScanner = next[0].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(129L, current.getTimestamp());
     }
   }
+
   @Test
   public void testDeleteColumnWithLatestTimeStampUsingMultipleVersionsAfterCompaction()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
             d.addColumn(fam, qual);
@@ -1531,7 +1273,7 @@ public class TestVisibilityLabelsWithDeletes {
       // Sleep to ensure compaction happens. Need to do it in a better way
       Thread.sleep(5000);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1539,43 +1281,43 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteFamilyLatestTimeStampWithMulipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
             d.addFamily(fam);
@@ -1590,7 +1332,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1598,26 +1340,26 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteColumnswithMultipleColumnsWithMultipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPutsWithDiffCols(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@@ -1627,7 +1369,7 @@ public class TestVisibilityLabelsWithDeletes {
           d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
           d.addColumns(fam, qual, 125L);
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(d);
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1639,7 +1381,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1647,40 +1389,34 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertTrue(Bytes.equals(current.getQualifierArray(), current.getQualifierOffset(),
-          current.getQualifierLength(), qual1, 0, qual1.length));
+        current.getQualifierLength(), qual1, 0, qual1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       assertTrue(Bytes.equals(current.getQualifierArray(), current.getQualifierOffset(),
-          current.getQualifierLength(), qual2, 0, qual2.length));
+        current.getQualifierLength(), qual2, 0, qual2.length));
     }
   }
 
   @Test
   public void testDeleteColumnsWithDiffColsAndTags() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual1, 125L, value);
@@ -1703,7 +1439,7 @@ public class TestVisibilityLabelsWithDeletes {
           d2.addColumns(fam, qual1, 125L);
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(createList(d1, d2));
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1713,22 +1449,17 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
       assertEquals(1, next.length);
     }
   }
+
   @Test
   public void testDeleteColumnsWithDiffColsAndTags1() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual1, 125L, value);
@@ -1751,7 +1482,7 @@ public class TestVisibilityLabelsWithDeletes {
           d2.addColumns(fam, qual1, 126L);
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(createList(d1, d2));
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1761,24 +1492,25 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
       assertEquals(1, next.length);
     }
   }
+
   @Test
   public void testDeleteFamilyWithoutCellVisibilityWithMulipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPutsWithoutVisibility(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.addFamily(fam);
             table.delete(d);
@@ -1792,7 +1524,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1801,8 +1533,8 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
@@ -1810,13 +1542,13 @@ public class TestVisibilityLabelsWithDeletes {
   public void testDeleteFamilyLatestTimeStampWithMulipleVersionsWithoutCellVisibilityInPuts()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPutsWithoutVisibility(tableName)) {
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
             d.addFamily(fam);
@@ -1830,7 +1562,7 @@ public class TestVisibilityLabelsWithDeletes {
       SUPERUSER.runAs(actiona);
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1838,51 +1570,51 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteFamilySpecificTimeStampWithMulipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + SECRET + "&" + TOPSECRET + ")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addFamily(fam, 126L);
             table.delete(d);
           } catch (Throwable t) {
@@ -1895,7 +1627,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(6);
@@ -1903,41 +1635,41 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testScanAfterCompaction() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
-                SECRET + "&" + TOPSECRET+")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addFamily(fam, 126L);
             table.delete(d);
           } catch (Throwable t) {
@@ -1958,7 +1690,7 @@ public class TestVisibilityLabelsWithDeletes {
       Thread.sleep(5000);
       // Sleep to ensure compaction happens. Need to do it in a better way
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1966,31 +1698,31 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteFamilySpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     // Do not flush here.
     try (Table table = doPuts(tableName)) {
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + TOPSECRET + "&

<TRUNCATED>

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

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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 2f5abfe5d94cd420379e19c6784bb854e74dc478
Parents: 74fea40
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Jan 2 17:07:41 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:42:08 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/2f5abfe5/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 772a9d6..a753d23 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -31,6 +31,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -58,6 +59,8 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Unit testing of ReplicationAdmin
@@ -69,6 +72,8 @@ public class TestReplicationAdmin {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestReplicationAdmin.class);
 
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationAdmin.class);
+
   private final static HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
@@ -118,6 +123,70 @@ public class TestReplicationAdmin {
   }
 
   @Test
+  public void testConcurrentPeerOperations() throws Exception {
+    int threadNum = 5;
+    AtomicLong successCount = new AtomicLong(0);
+
+    // Test concurrent add peer operation
+    Thread[] addPeers = new Thread[threadNum];
+    for (int i = 0; i < threadNum; i++) {
+      addPeers[i] = new Thread(() -> {
+        try {
+          hbaseAdmin.addReplicationPeer(ID_ONE,
+            ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build());
+          successCount.incrementAndGet();
+        } catch (Exception e) {
+          LOG.debug("Got exception when add replication peer", e);
+        }
+      });
+      addPeers[i].start();
+    }
+    for (Thread addPeer : addPeers) {
+      addPeer.join();
+    }
+    assertEquals(1, successCount.get());
+
+    // Test concurrent remove peer operation
+    successCount.set(0);
+    Thread[] removePeers = new Thread[threadNum];
+    for (int i = 0; i < threadNum; i++) {
+      removePeers[i] = new Thread(() -> {
+        try {
+          hbaseAdmin.removeReplicationPeer(ID_ONE);
+          successCount.incrementAndGet();
+        } catch (Exception e) {
+          LOG.debug("Got exception when remove replication peer", e);
+        }
+      });
+      removePeers[i].start();
+    }
+    for (Thread removePeer : removePeers) {
+      removePeer.join();
+    }
+    assertEquals(1, successCount.get());
+
+    // Test concurrent add peer operation again
+    successCount.set(0);
+    addPeers = new Thread[threadNum];
+    for (int i = 0; i < threadNum; i++) {
+      addPeers[i] = new Thread(() -> {
+        try {
+          hbaseAdmin.addReplicationPeer(ID_ONE,
+            ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build());
+          successCount.incrementAndGet();
+        } catch (Exception e) {
+          LOG.debug("Got exception when add replication peer", e);
+        }
+      });
+      addPeers[i].start();
+    }
+    for (Thread addPeer : addPeers) {
+      addPeer.join();
+    }
+    assertEquals(1, successCount.get());
+  }
+
+  @Test
   public void testAddInvalidPeer() {
     ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_ONE);


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 1c5cbeca149eef9d9b1ef3a5894a74a06789c6d7
Parents: 6c947a3
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 5 18:28:44 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:43:05 2018 +0800

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


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

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


[40/46] hbase git commit: HBASE-19711 TestReplicationAdmin.testConcurrentPeerOperations hangs

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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 088927d1ab3ef9acca788431f5b837cbee867c69
Parents: 1c5cbec
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 5 15:39:06 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:43:05 2018 +0800

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


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


[02/46] hbase git commit: HBASE-19928 TestVisibilityLabelsOnNewVersionBehaviorTable fails

Posted by zh...@apache.org.
HBASE-19928 TestVisibilityLabelsOnNewVersionBehaviorTable fails


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: a3233572ac8049fdbfa7edc9a48ca80a8c3de1ae
Parents: 00653a4
Author: Michael Stack <st...@apache.org>
Authored: Sat Feb 3 10:17:16 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 10:17:16 2018 -0800

----------------------------------------------------------------------
 .../TestVisibilityLabelsOnNewVersionBehaviorTable.java    | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3233572/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
index c14438e..d3177f9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
@@ -19,12 +19,21 @@ package org.apache.hadoop.hbase.security.visibility;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
 
+@Category({SecurityTests.class, MediumTests.class})
 public class TestVisibilityLabelsOnNewVersionBehaviorTable extends TestVisibilityLabelsWithDeletes {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestVisibilityLabelsOnNewVersionBehaviorTable.class);
 
   @Override
   protected Table createTable(HColumnDescriptor fam) throws IOException {
@@ -35,5 +44,4 @@ public class TestVisibilityLabelsOnNewVersionBehaviorTable extends TestVisibilit
     TEST_UTIL.getHBaseAdmin().createTable(table);
     return TEST_UTIL.getConnection().getTable(tableName);
   }
-
 }


[03/46] hbase git commit: HBASE-19914 Refactor TestVisibilityLabelsOnNewVersionBehaviorTable

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/cb138c2d/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsWithDeletesTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsWithDeletesTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsWithDeletesTestBase.java
new file mode 100644
index 0000000..ee5d3cc
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsWithDeletesTestBase.java
@@ -0,0 +1,313 @@
+/**
+ * Licensed 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.security.visibility;
+
+import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+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.client.Table;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Tests visibility labels with deletes
+ */
+public abstract class VisibilityLabelsWithDeletesTestBase {
+
+  protected static final String TOPSECRET = "TOPSECRET";
+  protected static final String PUBLIC = "PUBLIC";
+  protected static final String PRIVATE = "PRIVATE";
+  protected static final String CONFIDENTIAL = "CONFIDENTIAL";
+  protected static final String SECRET = "SECRET";
+  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static final byte[] row1 = Bytes.toBytes("row1");
+  protected static final byte[] row2 = Bytes.toBytes("row2");
+  protected final static byte[] fam = Bytes.toBytes("info");
+  protected final static byte[] qual = Bytes.toBytes("qual");
+  protected final static byte[] qual1 = Bytes.toBytes("qual1");
+  protected final static byte[] qual2 = Bytes.toBytes("qual2");
+  protected final static byte[] value = Bytes.toBytes("value");
+  protected final static byte[] value1 = Bytes.toBytes("value1");
+  protected static Configuration conf;
+
+  @Rule
+  public final TestName testName = new TestName();
+  protected static User SUPERUSER;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    // setup configuration
+    conf = TEST_UTIL.getConfiguration();
+    VisibilityTestUtil.enableVisiblityLabels(conf);
+    conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
+      ScanLabelGenerator.class);
+    conf.set("hbase.superuser", "admin");
+    TEST_UTIL.startMiniCluster(2);
+    SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
+
+    // Wait for the labels table to become available
+    TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000);
+    addLabels();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  public static void addLabels() throws Exception {
+    PrivilegedExceptionAction<VisibilityLabelsResponse> action =
+      new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
+        @Override
+        public VisibilityLabelsResponse run() throws Exception {
+          String[] labels = { SECRET, TOPSECRET, CONFIDENTIAL, PUBLIC, PRIVATE };
+          try (Connection conn = ConnectionFactory.createConnection(conf)) {
+            VisibilityClient.addLabels(conn, labels);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+    SUPERUSER.runAs(action);
+  }
+
+  protected abstract Table createTable(byte[] fam) throws IOException;
+
+  protected final void setAuths() throws IOException, InterruptedException {
+    PrivilegedExceptionAction<VisibilityLabelsResponse> action =
+      new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
+        @Override
+        public VisibilityLabelsResponse run() throws Exception {
+          try (Connection conn = ConnectionFactory.createConnection(conf)) {
+            return VisibilityClient.setAuths(conn,
+              new String[] { CONFIDENTIAL, PRIVATE, SECRET, TOPSECRET }, SUPERUSER.getShortName());
+          } catch (Throwable e) {
+          }
+          return null;
+        }
+      };
+    SUPERUSER.runAs(action);
+  }
+
+  private Table createTableAndWriteDataWithLabels(String... labelExps) throws Exception {
+    Table table = createTable(fam);
+    int i = 1;
+    List<Put> puts = new ArrayList<>(labelExps.length);
+    for (String labelExp : labelExps) {
+      Put put = new Put(Bytes.toBytes("row" + i));
+      put.addColumn(fam, qual, HConstants.LATEST_TIMESTAMP, value);
+      put.setCellVisibility(new CellVisibility(labelExp));
+      puts.add(put);
+      table.put(put);
+      i++;
+    }
+    // table.put(puts);
+    return table;
+  }
+
+  private Table createTableAndWriteDataWithLabels(long[] timestamp, String... labelExps)
+      throws Exception {
+    Table table = createTable(fam);
+    int i = 1;
+    List<Put> puts = new ArrayList<>(labelExps.length);
+    for (String labelExp : labelExps) {
+      Put put = new Put(Bytes.toBytes("row" + i));
+      put.addColumn(fam, qual, timestamp[i - 1], value);
+      put.setCellVisibility(new CellVisibility(labelExp));
+      puts.add(put);
+      table.put(put);
+      TEST_UTIL.getAdmin().flush(table.getName());
+      i++;
+    }
+    return table;
+  }
+
+  @Test
+  public void testVisibilityLabelsWithDeleteColumns() throws Throwable {
+    setAuths();
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
+
+    try (Table table = createTableAndWriteDataWithLabels(SECRET + "&" + TOPSECRET, SECRET)) {
+      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table table = connection.getTable(tableName)) {
+            Delete d = new Delete(row1);
+            d.setCellVisibility(new CellVisibility(TOPSECRET + "&" + SECRET));
+            d.addColumns(fam, qual);
+            table.delete(d);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+      SUPERUSER.runAs(actiona);
+
+      TEST_UTIL.getAdmin().flush(tableName);
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+      CellScanner cellScanner = next[0].cellScanner();
+      cellScanner.advance();
+      Cell current = cellScanner.current();
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
+
+    }
+  }
+
+  @Test
+  public void testVisibilityLabelsWithDeleteFamily() throws Exception {
+    setAuths();
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
+    try (Table table = createTableAndWriteDataWithLabels(SECRET, CONFIDENTIAL + "|" + TOPSECRET)) {
+      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table table = connection.getTable(tableName)) {
+            Delete d = new Delete(row2);
+            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
+            d.addFamily(fam);
+            table.delete(d);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+      SUPERUSER.runAs(actiona);
+
+      TEST_UTIL.getAdmin().flush(tableName);
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+      CellScanner cellScanner = next[0].cellScanner();
+      cellScanner.advance();
+      Cell current = cellScanner.current();
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
+    }
+  }
+
+  @Test
+  public void testVisibilityLabelsWithDeleteFamilyVersion() throws Exception {
+    setAuths();
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
+    long[] ts = new long[] { 123L, 125L };
+    try (
+      Table table = createTableAndWriteDataWithLabels(ts, CONFIDENTIAL + "|" + TOPSECRET, SECRET)) {
+      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table table = connection.getTable(tableName)) {
+            Delete d = new Delete(row1);
+            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
+            d.addFamilyVersion(fam, 123L);
+            table.delete(d);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+      SUPERUSER.runAs(actiona);
+
+      TEST_UTIL.getAdmin().flush(tableName);
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+      CellScanner cellScanner = next[0].cellScanner();
+      cellScanner.advance();
+      Cell current = cellScanner.current();
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
+    }
+  }
+
+  @Test
+  public void testVisibilityLabelsWithDeleteColumnExactVersion() throws Exception {
+    setAuths();
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
+    long[] ts = new long[] { 123L, 125L };
+    try (
+      Table table = createTableAndWriteDataWithLabels(ts, CONFIDENTIAL + "|" + TOPSECRET, SECRET)) {
+      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table table = connection.getTable(tableName)) {
+            Delete d = new Delete(row1);
+            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
+            d.addColumn(fam, qual, 123L);
+            table.delete(d);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+      SUPERUSER.runAs(actiona);
+
+      TEST_UTIL.getAdmin().flush(tableName);
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+      CellScanner cellScanner = next[0].cellScanner();
+      cellScanner.advance();
+      Cell current = cellScanner.current();
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
+    }
+  }
+}


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 9e8400fdad871e5b02dcfb866a579be8b30e60ae
Parents: 12d321d
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 25 18:49:56 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/9e8400fd/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 9f4ad18..6c1c213 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -37,20 +36,14 @@ public class ReplicationFactory {
       args);
   }
 
-  public static ReplicationQueuesClient
-      getReplicationQueuesClient(ReplicationQueuesClientArguments args) throws Exception {
-    return (ReplicationQueuesClient) ConstructorUtils
-        .invokeConstructor(ReplicationQueuesClientZKImpl.class, args);
-  }
-
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
-                                                     Abortable abortable) {
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
+      Abortable abortable) {
     return getReplicationPeers(zk, conf, null, abortable);
   }
 
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
-                                                     final ReplicationQueuesClient queuesClient, Abortable abortable) {
-    return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable);
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
+      ReplicationQueueStorage queueStorage, Abortable abortable) {
+    return new ReplicationPeersZKImpl(zk, conf, queueStorage, abortable);
   }
 
   public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e8400fd/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/9e8400fd/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/9e8400fd/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/9e8400fd/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
deleted file mode 100644
index e00a7a2..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@InterfaceAudience.Private
-public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implements
-    ReplicationQueuesClient {
-
-  Logger LOG = LoggerFactory.getLogger(ReplicationQueuesClientZKImpl.class);
-
-  public ReplicationQueuesClientZKImpl(ReplicationQueuesClientArguments args) {
-    this(args.getZk(), args.getConf(), args.getAbortable());
-  }
-
-  public ReplicationQueuesClientZKImpl(final ZKWatcher zk, Configuration conf,
-                                       Abortable abortable) {
-    super(zk, conf, abortable);
-  }
-
-  @Override
-  public void init() throws ReplicationException {
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) {
-        ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Internal error while initializing a queues client", e);
-    }
-  }
-
-  @Override
-  public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
-    znode = ZNodePaths.joinZNode(znode, queueId);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of wals for queueId=" + queueId
-          + " and serverName=" + serverName, e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public List<String> getAllQueues(String serverName) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of queues for serverName=" + serverName, e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public Set<String> getAllWALs() throws KeeperException {
-    /**
-     * Load all wals in all replication queues from ZK. This method guarantees to return a
-     * snapshot which contains all WALs in the zookeeper at the start of this call even there
-     * is concurrent queue failover. However, some newly created WALs during the call may
-     * not be included.
-     */
-    for (int retry = 0; ; retry++) {
-      int v0 = getQueuesZNodeCversion();
-      List<String> rss = getListOfReplicators();
-      if (rss == null || rss.isEmpty()) {
-        LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
-        return ImmutableSet.of();
-      }
-      Set<String> wals = Sets.newHashSet();
-      for (String rs : rss) {
-        List<String> listOfPeers = getAllQueues(rs);
-        // if rs just died, this will be null
-        if (listOfPeers == null) {
-          continue;
-        }
-        for (String id : listOfPeers) {
-          List<String> peersWals = getLogsInQueue(rs, id);
-          if (peersWals != null) {
-            wals.addAll(peersWals);
-          }
-        }
-      }
-      int v1 = getQueuesZNodeCversion();
-      if (v0 == v1) {
-        return wals;
-      }
-      LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
-        v0, v1, retry));
-    }
-  }
-
-  public int getQueuesZNodeCversion() throws KeeperException {
-    try {
-      Stat stat = new Stat();
-      ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
-      return stat.getCversion();
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get stat of replication rs node", e);
-      throw e;
-    }
-  }
-
-  @Override
-  public int getHFileRefsNodeChangeVersion() throws KeeperException {
-    Stat stat = new Stat();
-    try {
-      ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get stat of replication hfile references node.", e);
-      throw e;
-    }
-    return stat.getCversion();
-  }
-
-  @Override
-  public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of all peers in hfile references node.", e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public List<String> getReplicableHFiles(String peerId) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e);
-      throw e;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e8400fd/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/9e8400fd/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/9e8400fd/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..6abe3f8
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.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.HBaseClassTestRule;
+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.ClassRule;
+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 {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestReplicationStateZKImpl.class);
+
+  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/9e8400fd/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/9e8400fd/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/9e8400fd/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/9e8400fd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index 5abd874..84abfeb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
@@ -147,28 +148,13 @@ public final class ReplicationPeerManager {
     }
   }
 
-  private ReplicationPeerConfig copy(ReplicationPeerConfig peerConfig) {
-    ReplicationPeerConfig copiedPeerConfig = new ReplicationPeerConfig();
-    copiedPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
-    copiedPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
-    copiedPeerConfig.setTableCFsMap(peerConfig.getTableCFsMap());
-    copiedPeerConfig.setNamespaces(peerConfig.getNamespaces());
-    copiedPeerConfig.setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap());
-    copiedPeerConfig.setExcludeNamespaces(peerConfig.getExcludeNamespaces());
-    copiedPeerConfig.setBandwidth(peerConfig.getBandwidth());
-    copiedPeerConfig.setReplicateAllUserTables(peerConfig.replicateAllUserTables());
-    copiedPeerConfig.setClusterKey(peerConfig.getClusterKey());
-    copiedPeerConfig.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
-    return copiedPeerConfig;
-  }
-
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
     if (peers.containsKey(peerId)) {
       // this should be a retry, just return
       return;
     }
-    ReplicationPeerConfig copiedPeerConfig = copy(peerConfig);
+    ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build();
     peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
     peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
   }
@@ -205,13 +191,14 @@ public final class ReplicationPeerManager {
     // the checking rules are too complicated here so we give up checking whether this is a retry.
     ReplicationPeerDescription desc = peers.get(peerId);
     ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
-    ReplicationPeerConfig newPeerConfig = copy(peerConfig);
+    ReplicationPeerConfigBuilder newPeerConfigBuilder =
+        ReplicationPeerConfig.newBuilder(peerConfig);
     // we need to use the new conf to overwrite the old one.
-    newPeerConfig.getConfiguration().putAll(oldPeerConfig.getConfiguration());
-    newPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
-    newPeerConfig.getPeerData().putAll(oldPeerConfig.getPeerData());
-    newPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
-
+    newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
+    ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build();
     peerStorage.updatePeerConfig(peerId, newPeerConfig);
     peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e8400fd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
index 5f1df44..7b62169 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
@@ -1,42 +1,43 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.hadoop.hbase.replication.master;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
-import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-
 import java.io.IOException;
 import java.util.Collections;
-import java.util.List;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.zookeeper.KeeperException;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+
 /**
  * Implementation of a file cleaner that checks if a hfile is still scheduled for replication before
  * deleting it from hfile archive directory.
@@ -45,7 +46,7 @@ import org.slf4j.LoggerFactory;
 public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationHFileCleaner.class);
   private ZKWatcher zkw;
-  private ReplicationQueuesClient rqc;
+  private ReplicationQueueStorage rqs;
   private boolean stopped = false;
 
   @Override
@@ -60,8 +61,8 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     try {
       // The concurrently created new hfile entries in ZK may not be included in the return list,
       // but they won't be deleted because they're not in the checking set.
-      hfileRefs = loadHFileRefsFromPeers();
-    } catch (KeeperException e) {
+      hfileRefs = rqs.getAllHFileRefs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable files");
       return Collections.emptyList();
     }
@@ -82,37 +83,6 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     });
   }
 
-  /**
-   * Load all hfile references in all replication queues from ZK. This method guarantees to return a
-   * snapshot which contains all hfile references in the zookeeper at the start of this call.
-   * However, some newly created hfile references during the call may not be included.
-   */
-  private Set<String> loadHFileRefsFromPeers() throws KeeperException {
-    Set<String> hfileRefs = Sets.newHashSet();
-    List<String> listOfPeers;
-    for (int retry = 0;; retry++) {
-      int v0 = rqc.getHFileRefsNodeChangeVersion();
-      hfileRefs.clear();
-      listOfPeers = rqc.getAllPeersFromHFileRefsQueue();
-      if (listOfPeers == null) {
-        LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions.");
-        return ImmutableSet.of();
-      }
-      for (String id : listOfPeers) {
-        List<String> peerHFileRefs = rqc.getReplicableHFiles(id);
-        if (peerHFileRefs != null) {
-          hfileRefs.addAll(peerHFileRefs);
-        }
-      }
-      int v1 = rqc.getHFileRefsNodeChangeVersion();
-      if (v0 == v1) {
-        return hfileRefs;
-      }
-      LOG.debug(String.format("Replication hfile references node cversion changed from "
-          + "%d to %d, retry = %d", v0, v1, retry));
-    }
-  }
-
   @Override
   public void setConf(Configuration config) {
     // If either replication or replication of bulk load hfiles is disabled, keep all members null
@@ -139,17 +109,15 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
   public void setConf(Configuration conf, ZKWatcher zk) {
     super.setConf(conf);
     try {
-      initReplicationQueuesClient(conf, zk);
+      initReplicationQueueStorage(conf, zk);
     } catch (Exception e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
 
-  private void initReplicationQueuesClient(Configuration conf, ZKWatcher zk)
-      throws Exception {
+  private void initReplicationQueueStorage(Configuration conf, ZKWatcher zk) {
     this.zkw = zk;
-    this.rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(
-        conf, new WarnOnlyAbortable(), zkw));
+    this.rqs = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
   }
 
   @Override
@@ -179,25 +147,12 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     }
 
     try {
-      hfileRefsFromQueue = loadHFileRefsFromPeers();
-    } catch (KeeperException e) {
+      hfileRefsFromQueue = rqs.getAllHFileRefs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable "
           + "file for " + fStat.getPath());
       return false;
     }
     return !hfileRefsFromQueue.contains(fStat.getPath().getName());
   }
-
-  private static class WarnOnlyAbortable implements Abortable {
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.warn("ReplicationHFileCleaner received abort, ignoring.  Reason: " + why);
-      LOG.debug(e.toString(), e);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e8400fd/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 d2e6d68..20de326 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -24,16 +23,14 @@ import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,7 +46,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
 public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogCleaner.class);
   private ZKWatcher zkw;
-  private ReplicationQueuesClient replicationQueues;
+  private ReplicationQueueStorage queueStorage;
   private boolean stopped = false;
   private Set<String> wals;
   private long readZKTimestamp = 0;
@@ -60,8 +57,8 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     try {
       // The concurrently created new WALs may not be included in the return list,
       // but they won't be deleted because they're not in the checking set.
-      wals = replicationQueues.getAllWALs();
-    } catch (KeeperException e) {
+      wals = queueStorage.getAllWALs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read zookeeper, skipping checking deletable files");
       wals = null;
     }
@@ -108,9 +105,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);
     }
@@ -130,18 +125,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;
-    }
-  }
 }


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: ab5277582d362f464ed42fd01a69a8df16b95a47
Parents: c85716f
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 27 20:13:42 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:41:30 2018 +0800

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


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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab527758/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
new file mode 100644
index 0000000..e4be422
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.io.IOException;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
+
+/**
+ * A thread which calls {@code reportProcedureDone} to tell master the result of a remote procedure.
+ */
+@InterfaceAudience.Private
+class RemoteProcedureResultReporter extends Thread {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RemoteProcedureResultReporter.class);
+
+  // Time to pause if master says 'please hold'. Make configurable if needed.
+  private static final int INIT_PAUSE_TIME_MS = 1000;
+
+  private static final int MAX_BATCH = 100;
+
+  private final HRegionServer server;
+
+  private final LinkedBlockingQueue<RemoteProcedureResult> results = new LinkedBlockingQueue<>();
+
+  public RemoteProcedureResultReporter(HRegionServer server) {
+    this.server = server;
+  }
+
+  public void complete(long procId, Throwable error) {
+    RemoteProcedureResult.Builder builder = RemoteProcedureResult.newBuilder().setProcId(procId);
+    if (error != null) {
+      builder.setStatus(RemoteProcedureResult.Status.ERROR).setError(
+        ForeignExceptionUtil.toProtoForeignException(server.getServerName().toString(), error));
+    } else {
+      builder.setStatus(RemoteProcedureResult.Status.SUCCESS);
+    }
+    results.add(builder.build());
+  }
+
+  @Override
+  public void run() {
+    ReportProcedureDoneRequest.Builder builder = ReportProcedureDoneRequest.newBuilder();
+    int tries = 0;
+    while (!server.isStopped()) {
+      if (builder.getResultCount() == 0) {
+        try {
+          builder.addResult(results.take());
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          continue;
+        }
+      }
+      while (builder.getResultCount() < MAX_BATCH) {
+        RemoteProcedureResult result = results.poll();
+        if (result == null) {
+          break;
+        }
+        builder.addResult(result);
+      }
+      ReportProcedureDoneRequest request = builder.build();
+      try {
+        server.reportProcedureDone(builder.build());
+        builder.clear();
+        tries = 0;
+      } catch (IOException e) {
+        boolean pause =
+          e instanceof ServerNotRunningYetException || e instanceof PleaseHoldException;
+        long pauseTime;
+        if (pause) {
+          // Do backoff else we flood the Master with requests.
+          pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
+        } else {
+          pauseTime = INIT_PAUSE_TIME_MS; // Reset.
+        }
+        LOG.info("Failed report procedure " + TextFormat.shortDebugString(request) + "; retry (#" +
+          tries + ")" + (pause ? " after " + pauseTime + "ms delay (Master is coming online...)."
+            : " immediately."),
+          e);
+        Threads.sleep(pauseTime);
+        tries++;
+      }
+    }
+  }
+}

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


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 9631c6a31d729580dff13632d0e00f53616aa6af
Parents: 2f5abfe
Author: zhangduo <zh...@apache.org>
Authored: Wed Jan 3 09:39:44 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:42:08 2018 +0800

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


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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/9631c6a3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
index 2a6069c..be70e6e 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
@@ -17,8 +17,13 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.io.IOException;
+import java.util.List;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompoundConfiguration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -38,4 +43,37 @@ public final class ReplicationUtils {
     return c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
       HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
   }
+
+  public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,
+      Configuration baseConf) throws ReplicationException {
+    Configuration otherConf;
+    try {
+      otherConf = HBaseConfiguration.createClusterConf(baseConf, peerConfig.getClusterKey());
+    } catch (IOException e) {
+      throw new ReplicationException("Can't get peer configuration for peer " + peerConfig, e);
+    }
+
+    if (!peerConfig.getConfiguration().isEmpty()) {
+      CompoundConfiguration compound = new CompoundConfiguration();
+      compound.add(otherConf);
+      compound.addStringMap(peerConfig.getConfiguration());
+      return compound;
+    }
+
+    return otherConf;
+  }
+
+  public static void removeAllQueues(ReplicationQueueStorage queueStorage, String peerId)
+      throws ReplicationException {
+    for (ServerName replicator : queueStorage.getListOfReplicators()) {
+      List<String> queueIds = queueStorage.getAllQueues(replicator);
+      for (String queueId : queueIds) {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        if (queueInfo.getPeerId().equals(peerId)) {
+          queueStorage.removeQueue(replicator, queueId);
+        }
+      }
+      queueStorage.removeReplicatorIfQueueIsEmpty(replicator);
+    }
+  }
 }

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/9631c6a3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
deleted file mode 100644
index f2c3ec9..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.cleaner;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
-import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
-import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Used to clean the replication queues belonging to the peer which does not exist.
- */
-@InterfaceAudience.Private
-public class ReplicationZKNodeCleaner {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleaner.class);
-  private final ReplicationQueueStorage queueStorage;
-  private final ReplicationPeerStorage peerStorage;
-  private final ReplicationQueueDeletor queueDeletor;
-
-  public ReplicationZKNodeCleaner(Configuration conf, ZKWatcher zkw, Abortable abortable)
-      throws IOException {
-    this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
-    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf);
-    this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, abortable);
-  }
-
-  /**
-   * @return undeletedQueues replicator with its queueIds for removed peers
-   * @throws IOException
-   */
-  public Map<ServerName, List<String>> getUnDeletedQueues() throws IOException {
-    Map<ServerName, List<String>> undeletedQueues = new HashMap<>();
-    try {
-      Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds());
-      List<ServerName> replicators = this.queueStorage.getListOfReplicators();
-      if (replicators == null || replicators.isEmpty()) {
-        return undeletedQueues;
-      }
-      for (ServerName replicator : replicators) {
-        List<String> queueIds = this.queueStorage.getAllQueues(replicator);
-        for (String queueId : queueIds) {
-          ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-          if (!peerIds.contains(queueInfo.getPeerId())) {
-            undeletedQueues.computeIfAbsent(replicator, (key) -> new ArrayList<>()).add(queueId);
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Undeleted replication queue for removed peer found: "
-                  + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]",
-                    queueInfo.getPeerId(), replicator, queueId));
-            }
-          }
-        }
-      }
-    } catch (ReplicationException ke) {
-      throw new IOException("Failed to get the replication queues of all replicators", ke);
-    }
-    return undeletedQueues;
-  }
-
-  /**
-   * @return undeletedHFileRefsQueue replicator with its undeleted queueIds for removed peers in
-   *         hfile-refs queue
-   */
-  public Set<String> getUnDeletedHFileRefsQueues() throws IOException {
-    Set<String> undeletedHFileRefsQueue = new HashSet<>();
-    String hfileRefsZNode = queueDeletor.getHfileRefsZNode();
-    try {
-      Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds());
-      List<String> listOfPeers = this.queueStorage.getAllPeersFromHFileRefsQueue();
-      Set<String> peers = new HashSet<>(listOfPeers);
-      peers.removeAll(peerIds);
-      if (!peers.isEmpty()) {
-        undeletedHFileRefsQueue.addAll(peers);
-      }
-    } catch (ReplicationException e) {
-      throw new IOException("Failed to get list of all peers from hfile-refs znode "
-          + hfileRefsZNode, e);
-    }
-    return undeletedHFileRefsQueue;
-  }
-
-  private class ReplicationQueueDeletor extends ReplicationStateZKBase {
-
-    ReplicationQueueDeletor(ZKWatcher zk, Configuration conf, Abortable abortable) {
-      super(zk, conf, abortable);
-    }
-
-    /**
-     * @param replicator The regionserver which has undeleted queue
-     * @param queueId The undeleted queue id
-     */
-    void removeQueue(final ServerName replicator, final String queueId) throws IOException {
-      String queueZnodePath =
-          ZNodePaths.joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator.getServerName()),
-            queueId);
-      try {
-        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-        if (!peerStorage.listPeerIds().contains(queueInfo.getPeerId())) {
-          ZKUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath);
-          LOG.info("Successfully removed replication queue, replicator: " + replicator
-              + ", queueId: " + queueId);
-        }
-      } catch (ReplicationException | KeeperException e) {
-        throw new IOException("Failed to delete queue, replicator: " + replicator + ", queueId: "
-            + queueId);
-      }
-    }
-
-    /**
-     * @param hfileRefsQueueId The undeleted hfile-refs queue id
-     * @throws IOException
-     */
-    void removeHFileRefsQueue(final String hfileRefsQueueId) throws IOException {
-      String node = ZNodePaths.joinZNode(this.hfileRefsZNode, hfileRefsQueueId);
-      try {
-        if (!peerStorage.listPeerIds().contains(hfileRefsQueueId)) {
-          ZKUtil.deleteNodeRecursively(this.zookeeper, node);
-          LOG.info("Successfully removed hfile-refs queue " + hfileRefsQueueId + " from path "
-              + hfileRefsZNode);
-        }
-      } catch (ReplicationException | KeeperException e) {
-        throw new IOException("Failed to delete hfile-refs queue " + hfileRefsQueueId
-            + " from path " + hfileRefsZNode, e);
-      }
-    }
-
-    String getHfileRefsZNode() {
-      return this.hfileRefsZNode;
-    }
-  }
-
-  /**
-   * Remove the undeleted replication queue's zk node for removed peers.
-   * @param undeletedQueues replicator with its queueIds for removed peers
-   * @throws IOException
-   */
-  public void removeQueues(final Map<ServerName, List<String>> undeletedQueues) throws IOException {
-    for (Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueues.entrySet()) {
-      ServerName replicator = replicatorAndQueueIds.getKey();
-      for (String queueId : replicatorAndQueueIds.getValue()) {
-        queueDeletor.removeQueue(replicator, queueId);
-      }
-    }
-  }
-
-  /**
-   * Remove the undeleted hfile-refs queue's zk node for removed peers.
-   * @param undeletedHFileRefsQueues replicator with its undeleted queueIds for removed peers in
-   *          hfile-refs queue
-   * @throws IOException
-   */
-  public void removeHFileRefsQueues(final Set<String> undeletedHFileRefsQueues) throws IOException {
-    for (String hfileRefsQueueId : undeletedHFileRefsQueues) {
-      queueDeletor.removeHFileRefsQueue(hfileRefsQueueId);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9631c6a3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
deleted file mode 100644
index 19ca804..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.cleaner;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Chore that will clean the replication queues belonging to the peer which does not exist.
- */
-@InterfaceAudience.Private
-public class ReplicationZKNodeCleanerChore extends ScheduledChore {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleanerChore.class);
-  private final ReplicationZKNodeCleaner cleaner;
-
-  public ReplicationZKNodeCleanerChore(Stoppable stopper, int period,
-      ReplicationZKNodeCleaner cleaner) {
-    super("ReplicationZKNodeCleanerChore", stopper, period);
-    this.cleaner = cleaner;
-  }
-
-  @Override
-  protected void chore() {
-    try {
-      Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
-      cleaner.removeQueues(undeletedQueues);
-    } catch (IOException e) {
-      LOG.warn("Failed to clean replication zk node", e);
-    }
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/9631c6a3/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index abaed31..d128a41 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -114,6 +114,7 @@ import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
@@ -752,7 +753,7 @@ public class HBaseFsck extends Configured implements Closeable {
    * @return 0 on success, non-zero on failure
    */
   public int onlineHbck()
-      throws IOException, KeeperException, InterruptedException {
+      throws IOException, KeeperException, InterruptedException, ReplicationException {
     // print hbase server version
     errors.print("Version: " + status.getHBaseVersion());
 
@@ -3570,8 +3571,8 @@ public class HBaseFsck extends Configured implements Closeable {
     return hbi;
   }
 
-  private void checkAndFixReplication() throws IOException {
-    ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, connection, errors);
+  private void checkAndFixReplication() throws ReplicationException {
+    ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, errors);
     checker.checkUnDeletedQueues();
 
     if (checker.hasUnDeletedQueues() && this.fixReplication) {
@@ -4859,8 +4860,8 @@ public class HBaseFsck extends Configured implements Closeable {
   };
 
 
-  public HBaseFsck exec(ExecutorService exec, String[] args) throws KeeperException, IOException,
-      InterruptedException {
+  public HBaseFsck exec(ExecutorService exec, String[] args)
+      throws KeeperException, IOException, InterruptedException, ReplicationException {
     long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN;
 
     boolean checkCorruptHFiles = false;

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/9631c6a3/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
deleted file mode 100644
index 4bcde0a..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.cleaner;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
-import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ MasterTests.class, MediumTests.class })
-public class TestReplicationZKNodeCleaner {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestReplicationZKNodeCleaner.class);
-
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  private final String ID_ONE = "1";
-  private final ServerName SERVER_ONE = ServerName.valueOf("server1", 8000, 1234);
-  private final String ID_TWO = "2";
-  private final ServerName SERVER_TWO = ServerName.valueOf("server2", 8000, 1234);
-
-  private final Configuration conf;
-  private final ZKWatcher zkw;
-  private final ReplicationQueueStorage repQueues;
-
-  public TestReplicationZKNodeCleaner() throws Exception {
-    conf = TEST_UTIL.getConfiguration();
-    zkw = new ZKWatcher(conf, "TestReplicationZKNodeCleaner", null);
-    repQueues = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setInt("hbase.master.cleaner.interval", 10000);
-    TEST_UTIL.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void testReplicationZKNodeCleaner() throws Exception {
-    // add queue for ID_ONE which isn't exist
-    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
-
-    ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
-    Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
-    assertEquals(1, undeletedQueues.size());
-    assertTrue(undeletedQueues.containsKey(SERVER_ONE));
-    assertEquals(1, undeletedQueues.get(SERVER_ONE).size());
-    assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE));
-
-    // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
-
-    undeletedQueues = cleaner.getUnDeletedQueues();
-    assertEquals(1, undeletedQueues.size());
-    assertTrue(undeletedQueues.containsKey(SERVER_ONE));
-    assertEquals(2, undeletedQueues.get(SERVER_ONE).size());
-    assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE));
-    assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_TWO + "-" + SERVER_TWO));
-
-    cleaner.removeQueues(undeletedQueues);
-    undeletedQueues = cleaner.getUnDeletedQueues();
-    assertEquals(0, undeletedQueues.size());
-  }
-
-  @Test
-  public void testReplicationZKNodeCleanerChore() throws Exception {
-    // add queue for ID_ONE which isn't exist
-    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
-    // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
-
-    // Wait the cleaner chore to run
-    Thread.sleep(20000);
-
-    ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
-    assertEquals(0, cleaner.getUnDeletedQueues().size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9631c6a3/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
new file mode 100644
index 0000000..e64255c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+import java.util.stream.Stream;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
+import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestHBaseFsckReplication {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    ReplicationPeerStorage peerStorage = ReplicationStorageFactory
+        .getReplicationPeerStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+    ReplicationQueueStorage queueStorage = ReplicationStorageFactory
+        .getReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+
+    String peerId1 = "1";
+    String peerId2 = "2";
+    peerStorage.addPeer(peerId1, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(),
+      true);
+    peerStorage.addPeer(peerId2, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(),
+      true);
+    for (int i = 0; i < 10; i++) {
+      queueStorage.addWAL(ServerName.valueOf("localhost", 10000 + i, 100000 + i), peerId1,
+        "file-" + i);
+    }
+    queueStorage.addWAL(ServerName.valueOf("localhost", 10000, 100000), peerId2, "file");
+    HBaseFsck fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), true);
+    HbckTestingUtil.assertNoErrors(fsck);
+
+    // should not remove anything since the replication peer is still alive
+    assertEquals(10, queueStorage.getListOfReplicators().size());
+    peerStorage.removePeer(peerId1);
+    // there should be orphan queues
+    assertEquals(10, queueStorage.getListOfReplicators().size());
+    fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), false);
+    HbckTestingUtil.assertErrors(fsck, Stream.generate(() -> {
+      return ERROR_CODE.UNDELETED_REPLICATION_QUEUE;
+    }).limit(10).toArray(ERROR_CODE[]::new));
+
+    // should not delete anything when fix is false
+    assertEquals(10, queueStorage.getListOfReplicators().size());
+
+    fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), true);
+    HbckTestingUtil.assertErrors(fsck, Stream.generate(() -> {
+      return ERROR_CODE.UNDELETED_REPLICATION_QUEUE;
+    }).limit(10).toArray(ERROR_CODE[]::new));
+
+    List<ServerName> replicators = queueStorage.getListOfReplicators();
+    // should not remove the server with queue for peerId2
+    assertEquals(1, replicators.size());
+    assertEquals(ServerName.valueOf("localhost", 10000, 100000), replicators.get(0));
+    for (String queueId : queueStorage.getAllQueues(replicators.get(0))) {
+      assertEquals(peerId2, queueId);
+    }
+  }
+}

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 3b734aefd790cfd178ff6fdd61b4616a6a8ada62
Parents: 762770b
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 20 20:57:37 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b734aef/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() {


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: e2a72ae9188a03c838d556f17140dcf2e64802d3
Parents: 3b603d2
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 15 21:06:44 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/e2a72ae9/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/e2a72ae9/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 861e3b2..dca0bec 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/e2a72ae9/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/e2a72ae9/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/e2a72ae9/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/e2a72ae9/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/e2a72ae9/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/e2a72ae9/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 0d2fd9bc..775bea7 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,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
@@ -142,6 +142,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;
@@ -333,8 +334,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,
@@ -3568,6 +3568,30 @@ public class HMaster extends HRegionServer implements MasterServices {
     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);
+    }
+  }
+
   /**
    * This method modifies the master's configuration in order to inject replication-related features
    */

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/e2a72ae9/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/e2a72ae9/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 2c87745..57a4535 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -36,10 +35,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.ipc.RemoteException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
@@ -49,6 +45,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProc
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
 /**
  * A remote procecdure dispatcher for regionservers.
@@ -225,7 +228,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);
   }
 
   /**
@@ -234,22 +240,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()) {
@@ -281,8 +293,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.
@@ -308,6 +319,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 {
@@ -317,17 +333,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);
       }
     }
@@ -368,8 +375,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.
@@ -390,16 +396,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);
@@ -449,7 +445,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) {
@@ -490,6 +485,11 @@ public class RSProcedureDispatcher
         submitTask(new CloseRegionRemoteCall(serverName, op));
       }
     }
+
+    @Override
+    public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) {
+      throw new UnsupportedOperationException();
+    }
   }
 
   // ==========================================================================
@@ -497,13 +497,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/e2a72ae9/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/e2a72ae9/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/e2a72ae9/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/e2a72ae9/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/e2a72ae9/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 b971d61..c5c1ac6 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
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
@@ -126,6 +127,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
+import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler;
 import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler;
 import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
@@ -209,6 +211,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;
@@ -1894,6 +1897,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);
@@ -3672,6 +3677,62 @@ public class HRegionServer extends HasThread implements
         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;
+        }
+      }
+    }
+  }
+
   /**
    * This method modifies the region server's configuration in order to inject replication-related
    * features

http://git-wip-us.apache.org/repos/asf/hbase/blob/e2a72ae9/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 fd28e05..dbfcdc6 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
@@ -42,7 +41,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
-
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -101,6 +99,7 @@ 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.net.Address;
+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;
@@ -143,6 +142,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -173,6 +173,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;
@@ -3440,23 +3441,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 =
@@ -3473,4 +3457,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/e2a72ae9/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/e2a72ae9/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 5003d51..f3a1f70 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
@@ -539,26 +539,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/e2a72ae9/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/e2a72ae9/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/e2a72ae9/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 ba781d6..abb0ddd 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;


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 12678c5d011b4e0cf0d6aa1098531c806d18b6c4
Parents: 3b734ae
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Dec 20 16:43:38 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/12678c5d/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 29d9489..05bb637 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
@@ -911,6 +911,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();
 
@@ -933,22 +954,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());
   }
@@ -1033,6 +1051,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/12678c5d/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 2c0db61..5d7e853 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
@@ -28,6 +28,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 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.MasterTests;
@@ -67,6 +69,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.
@@ -225,4 +306,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();
+      }
+    }
+  }
 }


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: e5dc52ac2f6c95b6f25a9b591c7211f572883a6a
Parents: e2a72ae
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 18 15:22:36 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5dc52ac/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 775bea7..20ae7ee 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
@@ -130,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;
@@ -143,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;
@@ -173,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;
@@ -335,15 +341,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;
@@ -1218,7 +1224,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));
@@ -2326,11 +2331,8 @@ public class HMaster extends HRegionServer implements MasterServices {
             return true;
           }
           Pair<RegionInfo, ServerName> pair =
-              new Pair(MetaTableAccessor.getRegionInfo(data),
+              new Pair<>(MetaTableAccessor.getRegionInfo(data),
                   MetaTableAccessor.getServerName(data,0));
-          if (pair == null) {
-            return false;
-          }
           if (!pair.getFirst().getTable().equals(tableName)) {
             return false;
           }
@@ -2786,7 +2788,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return initialized;
   }
 
@@ -2805,7 +2807,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureExecutor.getEnvironment().setEventReady(serverCrashProcessingEnabled, b);
   }
 
-  public ProcedureEvent getServerCrashProcessingEnabledEvent() {
+  public ProcedureEvent<?> getServerCrashProcessingEnabledEvent() {
     return serverCrashProcessingEnabled;
   }
 
@@ -3356,54 +3358,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
@@ -3422,17 +3406,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
@@ -3585,13 +3563,18 @@ 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);
     }
   }
 
+  @Override
+  public ReplicationManager getReplicationManager() {
+    return replicationManager;
+  }
+
   /**
    * This method modifies the master's configuration in order to inject replication-related features
    */

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5dc52ac/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/e5dc52ac/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 535f288..f572cef 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/e5dc52ac/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/e5dc52ac/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/e5dc52ac/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/e5dc52ac/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/e5dc52ac/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/e5dc52ac/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/e5dc52ac/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/e5dc52ac/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/e5dc52ac/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/e5dc52ac/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 c5c1ac6..677b8b4 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
@@ -145,6 +145,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;
@@ -3686,7 +3687,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/e5dc52ac/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/e5dc52ac/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 be91aa0..e88710e 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
@@ -41,6 +41,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;
@@ -368,7 +369,6 @@ public class MockNoopMasterServices implements MasterServices {
 
   @Override
   public ClusterConnection getClusterConnection() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -398,20 +398,24 @@ public class MockNoopMasterServices implements MasterServices {
   }
 
   @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
@@ -421,8 +425,9 @@ public class MockNoopMasterServices implements MasterServices {
   }
 
   @Override
-  public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
@@ -457,7 +462,6 @@ public class MockNoopMasterServices implements MasterServices {
 
   @Override
   public ProcedureEvent getInitializedEvent() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -470,4 +474,9 @@ public class MockNoopMasterServices implements MasterServices {
   public Connection createConnection(Configuration conf) throws IOException {
     return null;
   }
+
+  @Override
+  public ReplicationManager getReplicationManager() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5dc52ac/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) {
   }
 
 }


[16/46] 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/12d321d4/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/12d321d4/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/12d321d4/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/12d321d4/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 ba78e6d..29a577b 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,13 @@
  */
 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;
 import java.util.HashMap;
@@ -25,26 +32,24 @@ 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.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterReplicationEndpointForTest;
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -53,15 +58,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
@@ -73,8 +69,6 @@ public class TestReplicationAdmin {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestReplicationAdmin.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationAdmin.class);
-
   private final static HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
@@ -109,16 +103,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);
     }
   }
 
@@ -208,32 +203,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();
   }
 
   /**
@@ -429,7 +421,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/12d321d4/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 e88710e..4e66676 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,9 +19,9 @@ 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;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.ChoreService;
@@ -41,7 +41,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;
@@ -55,9 +55,8 @@ 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 {
+
   private final Configuration conf;
   private final MetricsMaster metricsMaster;
 
@@ -461,7 +460,7 @@ public class MockNoopMasterServices implements MasterServices {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return null;
   }
 
@@ -476,7 +475,7 @@ public class MockNoopMasterServices implements MasterServices {
   }
 
   @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/12d321d4/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 2e1c979..5769d0e 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
@@ -42,6 +42,7 @@ 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.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -275,7 +276,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/12d321d4/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 5bdcd45..bfed922 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
@@ -54,9 +53,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");


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 3b27510ac387a2e1e5510dbf85748c8e21699d71
Parents: 068dcb4
Author: zhangduo <zh...@apache.org>
Authored: Tue Dec 26 20:39:00 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:39:29 2018 +0800

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


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b27510a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
new file mode 100644
index 0000000..ab35b46
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+
+/**
+ * All the modification method will fail once in the test and should finally succeed.
+ */
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestReplicationProcedureRetry {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setClass(HConstants.MASTER_IMPL, MockHMaster.class, HMaster.class);
+    UTIL.startMiniCluster(3);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @After
+  public void tearDownAfterTest() throws IOException {
+    for (ReplicationPeerDescription desc : UTIL.getAdmin().listReplicationPeers()) {
+      UTIL.getAdmin().removeReplicationPeer(desc.getPeerId());
+    }
+  }
+
+  private void doTest() throws IOException {
+    Admin admin = UTIL.getAdmin();
+    String peerId = "1";
+    ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
+        .setClusterKey("localhost:" + UTIL.getZkCluster().getClientPort() + ":/hbase2").build();
+    admin.addReplicationPeer(peerId, peerConfig, true);
+
+    assertEquals(peerConfig.getClusterKey(),
+      admin.getReplicationPeerConfig(peerId).getClusterKey());
+    ReplicationPeerConfig newPeerConfig =
+        ReplicationPeerConfig.newBuilder(peerConfig).setBandwidth(123456).build();
+    admin.updateReplicationPeerConfig(peerId, newPeerConfig);
+    assertEquals(newPeerConfig.getBandwidth(),
+      admin.getReplicationPeerConfig(peerId).getBandwidth());
+
+    admin.disableReplicationPeer(peerId);
+    assertFalse(admin.listReplicationPeers().get(0).isEnabled());
+
+    admin.enableReplicationPeer(peerId);
+    assertTrue(admin.listReplicationPeers().get(0).isEnabled());
+
+    admin.removeReplicationPeer(peerId);
+    assertTrue(admin.listReplicationPeers().isEmpty());
+
+    // make sure that we have run into the mocked method
+    MockHMaster master = (MockHMaster) UTIL.getHBaseCluster().getMaster();
+    assertTrue(master.addPeerCalled);
+    assertTrue(master.removePeerCalled);
+    assertTrue(master.updatePeerConfigCalled);
+    assertTrue(master.enablePeerCalled);
+    assertTrue(master.disablePeerCalled);
+  }
+
+  @Test
+  public void testErrorBeforeUpdate() throws IOException, ReplicationException {
+    ((MockHMaster) UTIL.getHBaseCluster().getMaster()).reset(true);
+    doTest();
+  }
+
+  @Test
+  public void testErrorAfterUpdate() throws IOException, ReplicationException {
+    ((MockHMaster) UTIL.getHBaseCluster().getMaster()).reset(false);
+    doTest();
+  }
+
+  public static final class MockHMaster extends HMaster {
+
+    volatile boolean addPeerCalled;
+
+    volatile boolean removePeerCalled;
+
+    volatile boolean updatePeerConfigCalled;
+
+    volatile boolean enablePeerCalled;
+
+    volatile boolean disablePeerCalled;
+
+    private ReplicationPeerManager manager;
+
+    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+      super(conf);
+    }
+
+    private Object invokeWithError(InvocationOnMock invocation, boolean errorBeforeUpdate)
+        throws Throwable {
+      if (errorBeforeUpdate) {
+        throw new ReplicationException("mock error before update");
+      }
+      invocation.callRealMethod();
+      throw new ReplicationException("mock error after update");
+    }
+
+    public void reset(boolean errorBeforeUpdate) throws ReplicationException {
+      addPeerCalled = false;
+      removePeerCalled = false;
+      updatePeerConfigCalled = false;
+      enablePeerCalled = false;
+      disablePeerCalled = false;
+      ReplicationPeerManager m = super.getReplicationPeerManager();
+      manager = spy(m);
+      doAnswer(invocation -> {
+        if (!addPeerCalled) {
+          addPeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).addPeer(anyString(), any(ReplicationPeerConfig.class), anyBoolean());
+      doAnswer(invocation -> {
+        if (!removePeerCalled) {
+          removePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).removePeer(anyString());
+      doAnswer(invocation -> {
+        if (!updatePeerConfigCalled) {
+          updatePeerConfigCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).updatePeerConfig(anyString(), any(ReplicationPeerConfig.class));
+      doAnswer(invocation -> {
+        if (!enablePeerCalled) {
+          enablePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).enablePeer(anyString());
+      doAnswer(invocation -> {
+        if (!disablePeerCalled) {
+          disablePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).disablePeer(anyString());
+    }
+
+    @Override
+    public ReplicationPeerManager getReplicationPeerManager() {
+      return manager;
+    }
+  }
+}


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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fc2f857/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
index 7b2e73f..53b8ba3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
@@ -19,9 +19,7 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;

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

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

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


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

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


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

Branch: refs/heads/HBASE-19397-branch-2
Commit: 74fea408bd22b9ee5f78e531d08de2e13a483d4a
Parents: 3be1397
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 2 16:13:55 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:42:08 2018 +0800

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


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