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/08 08:57:17 UTC

[01/18] hbase git commit: HBASE-19940 TestMetaShutdownHandler flakey; ADDENDUM: more debug [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19064 0db757ea5 -> 785a77c11 (forced update)


HBASE-19940 TestMetaShutdownHandler flakey; ADDENDUM: more debug


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

Branch: refs/heads/HBASE-19064
Commit: 590bee78dc9cb08a7b2d0eb5205d9dae65a8800a
Parents: 77153d4
Author: Michael Stack <st...@apache.org>
Authored: Tue Feb 6 06:34:49 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Feb 6 06:35:36 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/590bee78/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
index 172c170..0d410fc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
@@ -297,10 +297,10 @@ public class JVMClusterUtil {
         for (RegionServerThread t : regionservers) {
           if (t.isAlive()) {
             LOG.warn("RegionServerThreads taking too long to stop, interrupting; thread dump "  +
-              "if > three attempts");
+              "if > three attempts: i=" + i);
             if (i > 3) {
               try {
-                Threads.threadDumpingIsAlive(t.getRegionServer().getThread());
+                Threads.threadDumpingIsAlive(t);
               } catch (InterruptedException e) {
                 e.printStackTrace();
               }


[18/18] hbase git commit: HBASE-19781 Add a new cluster state flag for synchronous replication

Posted by zh...@apache.org.
HBASE-19781 Add a new cluster state flag for synchronous replication


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

Branch: refs/heads/HBASE-19064
Commit: 343403d2cc7089a38dd5e78c5957beb64bd25625
Parents: 4d80122
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Jan 22 11:44:49 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 8 16:49:19 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  39 +++++
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  31 ++++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   7 +
 .../hbase/client/ConnectionImplementation.java  |   9 ++
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  26 +++
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  15 ++
 .../client/ShortCircuitMasterConnection.java    |   9 ++
 .../replication/ReplicationPeerConfigUtil.java  |  26 +--
 .../replication/ReplicationPeerDescription.java |  10 +-
 .../hbase/replication/SyncReplicationState.java |  48 ++++++
 .../hbase/shaded/protobuf/RequestConverter.java |  10 ++
 .../src/main/protobuf/Master.proto              |   4 +
 .../src/main/protobuf/MasterProcedure.proto     |   6 +-
 .../src/main/protobuf/Replication.proto         |  20 +++
 .../replication/ReplicationPeerStorage.java     |  18 ++-
 .../hbase/replication/ReplicationUtils.java     |   1 +
 .../replication/ZKReplicationPeerStorage.java   |  60 +++++--
 .../replication/TestReplicationStateBasic.java  |  23 ++-
 .../TestZKReplicationPeerStorage.java           |  12 +-
 .../hbase/coprocessor/MasterObserver.java       |  23 +++
 .../org/apache/hadoop/hbase/master/HMaster.java |  12 ++
 .../hbase/master/MasterCoprocessorHost.java     |  21 +++
 .../hadoop/hbase/master/MasterRpcServices.java  |  17 ++
 .../hadoop/hbase/master/MasterServices.java     |   9 ++
 .../procedure/PeerProcedureInterface.java       |   2 +-
 .../replication/ReplicationPeerManager.java     |  51 +++++-
 ...ransitPeerSyncReplicationStateProcedure.java | 159 +++++++++++++++++++
 .../hbase/security/access/AccessController.java |   8 +
 .../replication/TestReplicationAdmin.java       |  62 ++++++++
 .../hbase/master/MockNoopMasterServices.java    |  11 +-
 .../cleaner/TestReplicationHFileCleaner.java    |   4 +-
 .../TestReplicationTrackerZKImpl.java           |   6 +-
 .../TestReplicationSourceManager.java           |   3 +-
 .../security/access/TestAccessController.java   |  16 ++
 .../hbase/util/TestHBaseFsckReplication.java    |   5 +-
 .../src/main/ruby/hbase/replication_admin.rb    |  15 ++
 hbase-shell/src/main/ruby/shell.rb              |   1 +
 .../src/main/ruby/shell/commands/list_peers.rb  |   6 +-
 .../transit_peer_sync_replication_state.rb      |  44 +++++
 .../test/ruby/hbase/replication_admin_test.rb   |  24 +++
 40 files changed, 818 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 b8546fa..167d6f3 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
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -2648,6 +2649,44 @@ public interface Admin extends Abortable, Closeable {
   List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException;
 
   /**
+   * Transit current cluster to a new state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @param state a new state of current cluster
+   * @throws IOException if a remote or network exception occurs
+   */
+  void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws IOException;
+
+  /**
+   * Transit current cluster to a new state in a synchronous 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 state a new state of current cluster
+   * @throws IOException if a remote or network exception occurs
+   */
+  Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
+      SyncReplicationState state) throws IOException;
+
+  /**
+   * Get the current cluster state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @return the current cluster state
+   * @throws IOException if a remote or network exception occurs
+   */
+  default SyncReplicationState getReplicationPeerSyncReplicationState(String peerId)
+      throws IOException {
+    List<ReplicationPeerDescription> peers = listReplicationPeers(Pattern.compile(peerId));
+    if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) {
+      throw new IOException("Replication peer " + peerId + " does not exist");
+    }
+    return peers.get(0).getSyncReplicationState();
+  }
+
+  /**
    * Mark region server(s) as decommissioned to prevent additional regions from getting
    * assigned to them. Optionally unload the regions on the servers. If there are multiple servers
    * to be decommissioned, decommissioning them at the same time can prevent wasteful region

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 35cdd3f..895e7ff 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.RpcChannel;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -600,6 +602,35 @@ public interface AsyncAdmin {
       ReplicationPeerConfig peerConfig);
 
   /**
+   * Transit current cluster to a new state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @param state a new state of current cluster
+   */
+  CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
+      SyncReplicationState state);
+
+  /**
+   * Get the current cluster state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @return the current cluster state wrapped by a {@link CompletableFuture}.
+   */
+  default CompletableFuture<SyncReplicationState>
+      getReplicationPeerSyncReplicationState(String peerId) {
+    CompletableFuture<SyncReplicationState> future = new CompletableFuture<>();
+    listReplicationPeers(Pattern.compile(peerId)).whenComplete((peers, error) -> {
+      if (error != null) {
+        future.completeExceptionally(error);
+      } else if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) {
+        future.completeExceptionally(
+          new IOException("Replication peer " + peerId + " does not exist"));
+      } else {
+        future.complete(peers.get(0).getSyncReplicationState());
+      }
+    });
+    return future;
+  }
+
+  /**
    * Append the replicable table-cf config of the specified peer
    * @param peerId 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/343403d2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 9b2390c..44771fd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -414,6 +415,12 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
+      SyncReplicationState clusterState) {
+    return wrap(rawAdmin.transitReplicationPeerSyncReplicationState(peerId, clusterState));
+  }
+
+  @Override
   public CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
       Map<TableName, List<String>> tableCfs) {
     return wrap(rawAdmin.appendReplicationPeerTableCFs(peerId, tableCfs));

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 8807884..5407c6d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -123,6 +123,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 
@@ -1724,6 +1726,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
           MasterProtos.ClearDeadServersRequest request) throws ServiceException {
         return stub.clearDeadServers(controller, request);
       }
+
+      @Override
+      public TransitReplicationPeerSyncReplicationStateResponse
+        transitReplicationPeerSyncReplicationState(RpcController controller,
+          TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+        return stub.transitReplicationPeerSyncReplicationState(controller, request);
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 8685984..c01b891 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
@@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
@@ -206,6 +207,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Disab
 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.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 
@@ -3991,6 +3993,30 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
+  public void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws IOException {
+    get(transitReplicationPeerSyncReplicationStateAsync(peerId, state), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
+      SyncReplicationState state) throws IOException {
+    TransitReplicationPeerSyncReplicationStateResponse response =
+        executeCallable(new MasterCallable<TransitReplicationPeerSyncReplicationStateResponse>(
+          getConnection(), getRpcControllerFactory()) {
+          @Override
+          protected TransitReplicationPeerSyncReplicationStateResponse rpcCall() throws Exception {
+            return master.transitReplicationPeerSyncReplicationState(getRpcController(),
+              RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId,
+                state));
+          }
+        });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE");
+  }
+
+  @Override
   public void appendReplicationPeerTableCFs(String id,
       Map<TableName, List<String>> tableCfs)
       throws ReplicationException, IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 050bfe2..30a372d 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
@@ -78,6 +78,7 @@ import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -255,6 +256,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
@@ -1613,6 +1616,18 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
+    SyncReplicationState clusterState) {
+    return this
+      .<TransitReplicationPeerSyncReplicationStateRequest, TransitReplicationPeerSyncReplicationStateResponse> procedureCall(
+        RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId,
+          clusterState),
+        (s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done),
+        (resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId,
+          () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"));
+  }
+
+  @Override
   public CompletableFuture<Void> appendReplicationPeerTableCFs(String id,
       Map<TableName, List<String>> tableCfs) {
     if (tableCfs == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index 50690b4..7bb65d2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -166,6 +166,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 
@@ -638,4 +640,11 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
       throws ServiceException {
     return stub.splitRegion(controller, request);
   }
+
+  @Override
+  public TransitReplicationPeerSyncReplicationStateResponse
+    transitReplicationPeerSyncReplicationState(RpcController controller,
+      TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+    return stub.transitReplicationPeerSyncReplicationState(controller, request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 642149b..86b49ea 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
@@ -38,6 +38,7 @@ 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.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -391,25 +392,28 @@ public final class ReplicationPeerConfigUtil {
     return ProtobufUtil.prependPBMagic(bytes);
   }
 
-  public static ReplicationPeerDescription toReplicationPeerDescription(
-      ReplicationProtos.ReplicationPeerDescription desc) {
-    boolean enabled = ReplicationProtos.ReplicationState.State.ENABLED == desc.getState()
-        .getState();
+  public static ReplicationPeerDescription
+      toReplicationPeerDescription(ReplicationProtos.ReplicationPeerDescription desc) {
+    boolean enabled =
+        ReplicationProtos.ReplicationState.State.ENABLED == desc.getState().getState();
     ReplicationPeerConfig config = convert(desc.getConfig());
-    return new ReplicationPeerDescription(desc.getId(), enabled, config);
+    return new ReplicationPeerDescription(desc.getId(), enabled, config,
+        SyncReplicationState.valueOf(desc.getSyncReplicationState().getNumber()));
   }
 
-  public static ReplicationProtos.ReplicationPeerDescription toProtoReplicationPeerDescription(
-      ReplicationPeerDescription desc) {
+  public static ReplicationProtos.ReplicationPeerDescription
+      toProtoReplicationPeerDescription(ReplicationPeerDescription desc) {
     ReplicationProtos.ReplicationPeerDescription.Builder builder =
         ReplicationProtos.ReplicationPeerDescription.newBuilder();
     builder.setId(desc.getPeerId());
-    ReplicationProtos.ReplicationState.Builder stateBuilder = ReplicationProtos.ReplicationState
-        .newBuilder();
-    stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED
-        : ReplicationProtos.ReplicationState.State.DISABLED);
+    ReplicationProtos.ReplicationState.Builder stateBuilder =
+        ReplicationProtos.ReplicationState.newBuilder();
+    stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED :
+        ReplicationProtos.ReplicationState.State.DISABLED);
     builder.setState(stateBuilder.build());
     builder.setConfig(convert(desc.getPeerConfig()));
+    builder.setSyncReplicationState(
+      ReplicationProtos.SyncReplicationState.forNumber(desc.getSyncReplicationState().ordinal()));
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
index ba97d07..2d077c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
@@ -28,11 +28,14 @@ public class ReplicationPeerDescription {
   private final String id;
   private final boolean enabled;
   private final ReplicationPeerConfig config;
+  private final SyncReplicationState syncReplicationState;
 
-  public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config) {
+  public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config,
+      SyncReplicationState syncReplicationState) {
     this.id = id;
     this.enabled = enabled;
     this.config = config;
+    this.syncReplicationState = syncReplicationState;
   }
 
   public String getPeerId() {
@@ -47,11 +50,16 @@ public class ReplicationPeerDescription {
     return this.config;
   }
 
+  public SyncReplicationState getSyncReplicationState() {
+    return this.syncReplicationState;
+  }
+
   @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("id : ").append(id);
     builder.append(", enabled : " + enabled);
     builder.append(", config : " + config);
+    builder.append(", syncReplicationState : " + syncReplicationState);
     return builder.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
new file mode 100644
index 0000000..bd144e9
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.yetus.audience.InterfaceAudience;
+
+/**
+ * Used by synchronous replication. Indicate the state of the current cluster in a synchronous
+ * replication peer. The state may be one of {@link SyncReplicationState#ACTIVE},
+ * {@link SyncReplicationState#DOWNGRADE_ACTIVE} or
+ * {@link SyncReplicationState#STANDBY}.
+ * <p>
+ * For asynchronous replication, the state is {@link SyncReplicationState#NONE}.
+ */
+@InterfaceAudience.Public
+public enum SyncReplicationState {
+  NONE, ACTIVE, DOWNGRADE_ACTIVE, STANDBY;
+
+  public static SyncReplicationState valueOf(int value) {
+    switch (value) {
+      case 0:
+        return NONE;
+      case 1:
+        return ACTIVE;
+      case 2:
+        return DOWNGRADE_ACTIVE;
+      case 3:
+        return STANDBY;
+      default:
+        throw new IllegalArgumentException("Unknown synchronous replication state " + value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 8ac7058..659be2a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
@@ -146,6 +147,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Enabl
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 
 /**
@@ -1874,4 +1876,12 @@ public final class RequestConverter {
     }
     return pbServers;
   }
+
+  public static TransitReplicationPeerSyncReplicationStateRequest
+    buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
+      SyncReplicationState state) {
+    return TransitReplicationPeerSyncReplicationStateRequest.newBuilder().setPeerId(peerId)
+      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
+      .build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 3a236c0..c2ab180 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -962,6 +962,10 @@ service MasterService {
   rpc ListReplicationPeers(ListReplicationPeersRequest)
     returns(ListReplicationPeersResponse);
 
+  /** Transit the state of current cluster in a synchronous replication peer */
+  rpc TransitReplicationPeerSyncReplicationState(TransitReplicationPeerSyncReplicationStateRequest)
+    returns(TransitReplicationPeerSyncReplicationStateResponse);
+
   /** Returns a list of ServerNames marked as decommissioned. */
   rpc ListDecommissionedRegionServers(ListDecommissionedRegionServersRequest)
     returns(ListDecommissionedRegionServersResponse);

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 83099c3..1dffd33 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -409,4 +409,8 @@ message AddPeerStateData {
 
 message UpdatePeerConfigStateData {
   required ReplicationPeer peer_config = 1;
-}
\ No newline at end of file
+}
+
+message TransitPeerSyncReplicationStateStateData {
+  required SyncReplicationState syncReplicationState = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 44295d8..de7b742 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -63,12 +63,23 @@ message ReplicationState {
 }
 
 /**
+ * Indicate the state of the current cluster in a synchronous replication peer.
+ */
+enum SyncReplicationState {
+  NONE = 0;
+  ACTIVE = 1;
+  DOWNGRADE_ACTIVE = 2;
+  STANDBY = 3;
+}
+
+/**
  * Used by replication. Description of the replication peer.
  */
 message ReplicationPeerDescription {
   required string id = 1;
   required ReplicationState state = 2;
   required ReplicationPeer config = 3;
+  optional SyncReplicationState syncReplicationState = 4;
 }
 
 /**
@@ -137,3 +148,12 @@ message ListReplicationPeersRequest {
 message ListReplicationPeersResponse {
   repeated ReplicationPeerDescription peer_desc = 1;
 }
+
+message TransitReplicationPeerSyncReplicationStateRequest {
+  required string peer_id = 1;
+  required SyncReplicationState syncReplicationState = 2;
+}
+
+message TransitReplicationPeerSyncReplicationStateResponse {
+  required uint64 proc_id = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 1adda02..d2538ab 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
@@ -31,8 +31,8 @@ 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;
+  void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled,
+      SyncReplicationState syncReplicationState) throws ReplicationException;
 
   /**
    * Remove a replication peer.
@@ -70,4 +70,18 @@ public interface ReplicationPeerStorage {
    * @throws ReplicationException if there are errors accessing the storage service.
    */
   ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException;
+
+  /**
+   * Set the state of current cluster in a synchronous replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void setPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws ReplicationException;
+
+  /**
+   * Get the state of current cluster in a synchronous replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  SyncReplicationState getPeerSyncReplicationState(String peerId)
+      throws ReplicationException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 11507aa..d633be9 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
@@ -22,6 +22,7 @@ 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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 a53500a..338ce3f 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
@@ -22,6 +22,7 @@ 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;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
@@ -51,6 +52,8 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
   public static final byte[] DISABLED_ZNODE_BYTES =
     toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
 
+  public static final String SYNCHRONOUS_REPLICATION_STATE_ZNODE = "sync-rep-state";
+
   /**
    * The name of the znode that contains the replication status of a remote slave (i.e. peer)
    * cluster.
@@ -79,21 +82,29 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
     return ZNodePaths.joinZNode(peersZNode, peerId);
   }
 
+  @VisibleForTesting
+  public String getSyncReplicationStateNode(String peerId) {
+    return ZNodePaths.joinZNode(getPeerNode(peerId), SYNCHRONOUS_REPLICATION_STATE_ZNODE);
+  }
+
   @Override
-  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException {
+  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled,
+      SyncReplicationState syncReplicationState) throws ReplicationException {
+    List<ZKUtilOp> multiOps = Arrays.asList(
+      ZKUtilOp.createAndFailSilent(getPeerNode(peerId),
+        ReplicationPeerConfigUtil.toByteArray(peerConfig)),
+      ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
+        enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES),
+      ZKUtilOp.createAndFailSilent(getSyncReplicationStateNode(peerId),
+        Bytes.toBytes(syncReplicationState.ordinal())));
     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);
+      ZKUtil.multiOrSequential(zookeeper, multiOps, 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 + ", peerConfig=>" + peerConfig + ", state=" +
+            (enabled ? "ENABLED" : "DISABLED") + ", syncReplicationState=" + syncReplicationState,
+        e);
     }
   }
 
@@ -166,4 +177,31 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
           "Failed to parse replication peer config for peer with id=" + peerId, e);
     }
   }
+
+  @Override
+  public void setPeerSyncReplicationState(String peerId, SyncReplicationState clusterState)
+      throws ReplicationException {
+    byte[] clusterStateBytes = Bytes.toBytes(clusterState.ordinal());
+    try {
+      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId), clusterStateBytes);
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Unable to change the cluster state for the synchronous replication peer with id=" +
+              peerId,
+          e);
+    }
+  }
+
+  @Override
+  public SyncReplicationState getPeerSyncReplicationState(String peerId)
+      throws ReplicationException {
+    byte[] data;
+    try {
+      data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException(
+          "Error getting cluster state for the synchronous replication peer with id=" + peerId, e);
+    }
+    return SyncReplicationState.valueOf(Bytes.toInt(data));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 fccffb5..fe658a3 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
@@ -160,7 +160,8 @@ public abstract class TestReplicationStateBasic {
     assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
     assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
     rp.getPeerStorage().addPeer(ID_ONE,
-            ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true,
+      SyncReplicationState.NONE);
     rqs.addPeerToHFileRefs(ID_ONE);
     rqs.addHFileRefs(ID_ONE, files1);
     assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
@@ -183,10 +184,12 @@ public abstract class TestReplicationStateBasic {
   public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
     rp.init();
     rp.getPeerStorage().addPeer(ID_ONE,
-      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true,
+      SyncReplicationState.NONE);
     rqs.addPeerToHFileRefs(ID_ONE);
     rp.getPeerStorage().addPeer(ID_TWO,
-      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true,
+      SyncReplicationState.NONE);
     rqs.addPeerToHFileRefs(ID_TWO);
 
     List<Pair<Path, Path>> files1 = new ArrayList<>(3);
@@ -235,9 +238,13 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(0);
 
     // Add some peers
-    rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true);
+    rp.getPeerStorage().addPeer(ID_ONE,
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true,
+      SyncReplicationState.NONE);
     assertNumberOfPeers(1);
-    rp.getPeerStorage().addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), true);
+    rp.getPeerStorage().addPeer(ID_TWO,
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true,
+      SyncReplicationState.NONE);
     assertNumberOfPeers(2);
 
     assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationUtils
@@ -247,7 +254,9 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(1);
 
     // Add one peer
-    rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true);
+    rp.getPeerStorage().addPeer(ID_ONE,
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true,
+      SyncReplicationState.NONE);
     rp.addPeer(ID_ONE);
     assertNumberOfPeers(2);
     assertTrue(rp.getPeer(ID_ONE).isPeerEnabled());
@@ -311,7 +320,7 @@ public abstract class TestReplicationStateBasic {
       // Add peers for the corresponding queues so they are not orphans
       rp.getPeerStorage().addPeer("qId" + i,
         ReplicationPeerConfig.newBuilder().setClusterKey("localhost:2818:/bogus" + i).build(),
-        true);
+        true, SyncReplicationState.NONE);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 3290fb0..1258695 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
@@ -87,8 +87,9 @@ public class TestZKReplicationPeerStorage {
     Random rand = new Random(seed);
     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())
+        .setRemoteWALDir(Long.toHexString(rand.nextLong())).setNamespaces(randNamespaces(rand))
+        .setExcludeNamespaces(randNamespaces(rand)).setTableCFsMap(randTableCFs(rand))
+        .setExcludeTableCFsMap(randTableCFs(rand)).setReplicateAllUserTables(rand.nextBoolean())
         .setBandwidth(rand.nextInt(1000)).build();
   }
 
@@ -139,7 +140,8 @@ public class TestZKReplicationPeerStorage {
   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);
+      STORAGE.addPeer(Integer.toString(i), getConfig(i), i % 2 == 0,
+        SyncReplicationState.valueOf(i % 4));
     }
     List<String> peerIds = STORAGE.listPeerIds();
     assertEquals(peerCount, peerIds.size());
@@ -163,6 +165,10 @@ public class TestZKReplicationPeerStorage {
     for (int i = 0; i < peerCount; i++) {
       assertEquals(i % 2 != 0, STORAGE.isPeerEnabled(Integer.toString(i)));
     }
+    for (int i = 0; i < peerCount; i++) {
+      assertEquals(SyncReplicationState.valueOf(i % 4),
+        STORAGE.getPeerSyncReplicationState(Integer.toString(i)));
+    }
     String toRemove = Integer.toString(peerCount / 2);
     STORAGE.removePeer(toRemove);
     peerIds = STORAGE.listPeerIds();

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index a17bc9f..8d2b55f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
@@ -1232,6 +1233,28 @@ public interface MasterObserver {
       String regex) throws IOException {}
 
   /**
+   * Called before transit current cluster state for the specified synchronous replication peer
+   * @param ctx the environment to interact with the framework and master
+   * @param peerId a short name that identifies the peer
+   * @param state a new state
+   */
+  default void preTransitReplicationPeerSyncReplicationState(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
+      SyncReplicationState state) throws IOException {
+  }
+
+  /**
+   * Called after transit current cluster state for the specified synchronous replication peer
+   * @param ctx the environment to interact with the framework and master
+   * @param peerId a short name that identifies the peer
+   * @param state a new state
+   */
+  default void postTransitReplicationPeerSyncReplicationState(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
+      SyncReplicationState state) throws IOException {
+  }
+
+  /**
    * Called before new LockProcedure is queued.
    * @param ctx the environment to interact with the framework and master
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 ecf6d63..ddffbec 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
@@ -135,6 +135,7 @@ 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.ReplicationPeerManager;
+import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure;
 import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
@@ -169,6 +170,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader;
@@ -3416,6 +3418,16 @@ public class HMaster extends HRegionServer implements MasterServices {
     return peers;
   }
 
+  @Override
+  public long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
+    throws ReplicationException, IOException {
+    LOG.info(
+      getClientIdAuditPrefix() +
+        " transit current cluster state to {} in a synchronous replication peer id={}",
+      state, peerId);
+    return executePeerProcedure(new TransitPeerSyncReplicationStateProcedure(peerId, state));
+  }
+
   /**
    * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
    * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 7f17227..ba5e70a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -1524,6 +1525,26 @@ public class MasterCoprocessorHost
     });
   }
 
+  public void preTransitReplicationPeerSyncReplicationState(final String peerId,
+      final SyncReplicationState clusterState) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preTransitReplicationPeerSyncReplicationState(this, peerId, clusterState);
+      }
+    });
+  }
+
+  public void postTransitReplicationPeerSyncReplicationState(final String peerId,
+      final SyncReplicationState clusterState) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postTransitReplicationPeerSyncReplicationState(this, peerId, clusterState);
+      }
+    });
+  }
+
   public void preRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
       LockType type, String description) throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 377a9c6..f5bf117 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
@@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
@@ -285,6 +286,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@@ -1958,6 +1961,20 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public TransitReplicationPeerSyncReplicationStateResponse
+    transitReplicationPeerSyncReplicationState(RpcController controller,
+      TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+    try {
+      long procId = master.transitReplicationPeerSyncReplicationState(request.getPeerId(),
+        SyncReplicationState.valueOf(request.getSyncReplicationState().getNumber()));
+      return TransitReplicationPeerSyncReplicationStateResponse.newBuilder().setProcId(procId)
+        .build();
+    } catch (ReplicationException | IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
       ListReplicationPeersRequest request) throws ServiceException {
     ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 9d371bd..5c6f2dd 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
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 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.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -480,6 +481,14 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
+   * Set current cluster state for a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @param clusterState state of current cluster
+   */
+  long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState clusterState)
+      throws ReplicationException, IOException;
+
+  /**
    * @return {@link LockManager} to lock namespaces/tables/regions.
    */
   LockManager getLockManager();

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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
index 4abc9ad..fc5348e 100644
--- 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
@@ -25,7 +25,7 @@ import org.apache.yetus.audience.InterfaceStability;
 public interface PeerProcedureInterface {
 
   enum PeerOperationType {
-    ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH
+    ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH, TRANSIT_SYNC_REPLICATION_STATE
   }
 
   String getPeerId();

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 d715e2e..9336fbd 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.master.replication;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.EnumMap;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -43,6 +45,7 @@ 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.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -61,6 +64,16 @@ public class ReplicationPeerManager {
 
   private final ConcurrentMap<String, ReplicationPeerDescription> peers;
 
+  private final EnumMap<SyncReplicationState, EnumSet<SyncReplicationState>> allowedTransition =
+    new EnumMap<SyncReplicationState, EnumSet<SyncReplicationState>>(SyncReplicationState.class) {
+      {
+        put(SyncReplicationState.ACTIVE, EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE));
+        put(SyncReplicationState.STANDBY, EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE));
+        put(SyncReplicationState.DOWNGRADE_ACTIVE,
+          EnumSet.of(SyncReplicationState.STANDBY, SyncReplicationState.ACTIVE));
+      }
+    };
+
   ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage,
       ConcurrentMap<String, ReplicationPeerDescription> peers) {
     this.peerStorage = peerStorage;
@@ -163,6 +176,17 @@ public class ReplicationPeerManager {
     }
   }
 
+  public void preTransitPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    SyncReplicationState fromState = desc.getSyncReplicationState();
+    EnumSet<SyncReplicationState> allowedToStates = allowedTransition.get(fromState);
+    if (allowedToStates == null || !allowedToStates.contains(state)) {
+      throw new DoNotRetryIOException("Can not transit current cluster state from " + fromState +
+          " to " + state + " for peer id=" + peerId);
+    }
+  }
+
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
     if (peers.containsKey(peerId)) {
@@ -170,8 +194,12 @@ public class ReplicationPeerManager {
       return;
     }
     ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build();
-    peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
+    SyncReplicationState syncReplicationState =
+        StringUtils.isBlank(peerConfig.getRemoteWALDir()) ? SyncReplicationState.NONE
+            : SyncReplicationState.DOWNGRADE_ACTIVE;
+    peerStorage.addPeer(peerId, copiedPeerConfig, enabled, syncReplicationState);
+    peers.put(peerId,
+      new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState));
   }
 
   public void removePeer(String peerId) throws ReplicationException {
@@ -190,7 +218,8 @@ public class ReplicationPeerManager {
       return;
     }
     peerStorage.setPeerState(peerId, enabled);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig()));
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig(),
+      desc.getSyncReplicationState()));
   }
 
   public void enablePeer(String peerId) throws ReplicationException {
@@ -215,7 +244,8 @@ public class ReplicationPeerManager {
     newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
     ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build();
     peerStorage.updatePeerConfig(peerId, newPeerConfig);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
+    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig,
+      desc.getSyncReplicationState()));
   }
 
   public List<ReplicationPeerDescription> listPeers(Pattern pattern) {
@@ -231,6 +261,14 @@ public class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
+  public void transitPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws ReplicationException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    peerStorage.setPeerSyncReplicationState(peerId, state);
+    peers.put(peerId,
+      new ReplicationPeerDescription(peerId, desc.isEnabled(), desc.getPeerConfig(), state));
+  }
+
   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
@@ -350,10 +388,11 @@ public class ReplicationPeerManager {
     for (String peerId : peerStorage.listPeerIds()) {
       ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
       boolean enabled = peerStorage.isPeerEnabled(peerId);
-      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig));
+      SyncReplicationState state = peerStorage.getPeerSyncReplicationState(peerId);
+      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig, state));
     }
     return new ReplicationPeerManager(peerStorage,
-        ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
+      ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
new file mode 100644
index 0000000..d26eecc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+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.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TransitPeerSyncReplicationStateStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+/**
+ * The procedure for transit current cluster state for a synchronous replication peer.
+ */
+@InterfaceAudience.Private
+public class TransitPeerSyncReplicationStateProcedure extends ModifyPeerProcedure {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TransitPeerSyncReplicationStateProcedure.class);
+
+  private SyncReplicationState state;
+
+  public TransitPeerSyncReplicationStateProcedure() {
+  }
+
+  public TransitPeerSyncReplicationStateProcedure(String peerId, SyncReplicationState state) {
+    super(peerId);
+    this.state = state;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.TRANSIT_SYNC_REPLICATION_STATE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preTransitReplicationPeerSyncReplicationState(peerId, state);
+    }
+    env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, state);
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().transitPeerSyncReplicationState(peerId, state);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env)
+    throws IOException, ReplicationException {
+    LOG.info("Successfully transit current cluster state to {} in synchronous replication peer {}",
+      state, peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      env.getMasterCoprocessorHost().postTransitReplicationPeerSyncReplicationState(peerId, state);
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(TransitPeerSyncReplicationStateStateData.newBuilder()
+      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
+      .build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    TransitPeerSyncReplicationStateStateData data =
+      serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
+    state = SyncReplicationState.valueOf(data.getSyncReplicationState().getNumber());
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    switch (state) {
+      case PRE_PEER_MODIFICATION:
+        try {
+          prePeerModification(env);
+        } catch (IOException 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("{} failed to call prePeerModification for peer {}, retry", getClass().getName(),
+            peerId, e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
+        return Flow.HAS_MORE_STATE;
+      case UPDATE_PEER_STORAGE:
+        try {
+          updatePeerStorage(env);
+        } catch (ReplicationException e) {
+          LOG.warn("{} update peer storage for peer {} failed, retry", getClass().getName(), peerId,
+            e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
+        return Flow.HAS_MORE_STATE;
+      case REFRESH_PEER_ON_RS:
+        // TODO: Need add child procedure for every RegionServer
+        setNextState(PeerModificationState.POST_PEER_MODIFICATION);
+        return Flow.HAS_MORE_STATE;
+      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("{} 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;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+  }
+
+  private void releaseLatch() {
+    ProcedurePrepareLatch.releaseLatch(latch, this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 1fbf01d..decfb4c 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
@@ -112,6 +112,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
@@ -2501,6 +2502,13 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   @Override
+  public void preTransitReplicationPeerSyncReplicationState(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
+      SyncReplicationState clusterState) throws IOException {
+    requirePermission(ctx, "transitSyncReplicationPeerState", Action.ADMIN);
+  }
+
+  @Override
   public void preListReplicationPeers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       String regex) throws IOException {
     requirePermission(ctx, "listReplicationPeers", Action.ADMIN);

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 e471100..a7710e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 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.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterReplicationEndpointForTest;
 import org.apache.hadoop.hbase.replication.regionserver.TestReplicator.ReplicationEndpointForTest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -983,4 +984,65 @@ public class TestReplicationAdmin {
       // OK
     }
   }
+
+  @Test
+  public void testTransitSyncReplicationPeerState() throws Exception {
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_ONE);
+    hbaseAdmin.addReplicationPeer(ID_ONE, builder.build());
+    assertEquals(SyncReplicationState.NONE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE));
+
+    try {
+      hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE,
+        SyncReplicationState.DOWNGRADE_ACTIVE);
+      fail("Can't transit cluster state if replication peer don't config remote wal dir");
+    } catch (Exception e) {
+      // OK
+    }
+
+    String rootDir = "hdfs://srv1:9999/hbase";
+    builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_SECOND);
+    builder.setRemoteWALDir(rootDir);
+    hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    // Disable and enable peer don't affect SyncReplicationState
+    hbaseAdmin.disableReplicationPeer(ID_SECOND);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+    hbaseAdmin.enableReplicationPeer(ID_SECOND);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
+    assertEquals(SyncReplicationState.ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    try {
+      hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND,
+        SyncReplicationState.STANDBY);
+      fail("Can't transit cluster state from ACTIVE to STANDBY");
+    } catch (Exception e) {
+      // OK
+    }
+
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.STANDBY);
+    assertEquals(SyncReplicationState.STANDBY,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    try {
+      hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
+      fail("Can't transit cluster state from STANDBY to ACTIVE");
+    } catch (Exception e) {
+      // OK
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 48e9e8d..b7d54d6 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
@@ -54,11 +54,10 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 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.SyncReplicationState;
 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;
@@ -481,4 +480,10 @@ public class MockNoopMasterServices implements MasterServices {
   public ReplicationPeerManager getReplicationPeerManager() {
     return null;
   }
-}
+
+  @Override
+  public long transitReplicationPeerSyncReplicationState(String peerId,
+    SyncReplicationState clusterState) throws ReplicationException, IOException {
+    return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 08dd428..24b930c 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
@@ -50,6 +50,7 @@ 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.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -109,7 +110,8 @@ public class TestReplicationHFileCleaner {
   public void setup() throws ReplicationException, IOException {
     root = TEST_UTIL.getDataTestDirOnTestFS();
     rp.getPeerStorage().addPeer(peerId,
-      ReplicationPeerConfig.newBuilder().setClusterKey(TEST_UTIL.getClusterKey()).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(TEST_UTIL.getClusterKey()).build(), true,
+      SyncReplicationState.NONE);
     rq.addPeerToHFileRefs(peerId);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 52b914e..69121f8 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
@@ -154,11 +154,13 @@ public class TestReplicationTrackerZKImpl {
   public void testPeerNameControl() throws Exception {
     int exists = 0;
     rp.getPeerStorage().addPeer("6",
-      ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true,
+      SyncReplicationState.NONE);
 
     try {
       rp.getPeerStorage().addPeer("6",
-        ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true);
+        ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true,
+        SyncReplicationState.NONE);
     } catch (ReplicationException e) {
       if (e.getCause() instanceof KeeperException.NodeExistsException) {
         exists++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 a53cba3..db62b49 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,6 +70,7 @@ 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.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -584,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.getPeerStorage().addPeer(peerId, peerConfig, true);
+    rp.getPeerStorage().addPeer(peerId, peerConfig, true, SyncReplicationState.NONE);
     try {
       manager.addPeer(peerId);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 ab9fb17..b2b5f1c 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
@@ -117,6 +117,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
@@ -2938,6 +2939,21 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Test
+  public void testTransitSyncReplicationPeerState() throws Exception {
+    AccessTestAction action = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preTransitReplicationPeerSyncReplicationState(
+          ObserverContextImpl.createAndPrepare(CP_ENV), "test", SyncReplicationState.NONE);
+        return null;
+      }
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
+
+  @Test
   public void testListReplicationPeers() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override


[12/18] hbase git commit: HBASE-19078 Add a remote peer cluster wal directory config for synchronous replication

Posted by zh...@apache.org.
HBASE-19078 Add a remote peer cluster wal directory config for synchronous replication

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

Branch: refs/heads/HBASE-19064
Commit: 4b10af083bc21d0ca69666e81f9d04947e405615
Parents: 8f260d0
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Jan 13 18:55:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 8 16:48:23 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfigUtil.java  |  6 ++
 .../replication/ReplicationPeerConfig.java      | 21 +++++-
 .../ReplicationPeerConfigBuilder.java           |  7 ++
 .../src/main/protobuf/Replication.proto         |  1 +
 .../replication/ReplicationPeerManager.java     | 15 ++++
 .../replication/TestReplicationAdmin.java       | 77 ++++++++++++++++++++
 .../src/main/ruby/hbase/replication_admin.rb    | 17 +++--
 hbase-shell/src/main/ruby/hbase_constants.rb    |  1 +
 .../src/main/ruby/shell/commands/add_peer.rb    | 21 +++++-
 .../src/main/ruby/shell/commands/list_peers.rb  | 19 ++++-
 .../test/ruby/hbase/replication_admin_test.rb   | 16 ++++
 11 files changed, 188 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/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 a234a9b..642149b 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
@@ -315,6 +315,9 @@ public final class ReplicationPeerConfigUtil {
         excludeNamespacesList.stream().map(ByteString::toStringUtf8).collect(Collectors.toSet()));
     }
 
+    if (peer.hasRemoteWALDir()) {
+      builder.setRemoteWALDir(peer.getRemoteWALDir());
+    }
     return builder.build();
   }
 
@@ -371,6 +374,9 @@ public final class ReplicationPeerConfigUtil {
       }
     }
 
+    if (peerConfig.getRemoteWALDir() != null) {
+      builder.setRemoteWALDir(peerConfig.getRemoteWALDir());
+    }
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/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 bf8d030..4c10c46 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
@@ -46,6 +46,8 @@ public class ReplicationPeerConfig {
   private Map<TableName, ? extends Collection<String>> excludeTableCFsMap = null;
   private Set<String> excludeNamespaces = null;
   private long bandwidth = 0;
+  // Used by synchronous replication
+  private String remoteWALDir;
 
   private ReplicationPeerConfig(ReplicationPeerConfigBuilderImpl builder) {
     this.clusterKey = builder.clusterKey;
@@ -64,6 +66,7 @@ public class ReplicationPeerConfig {
         builder.excludeNamespaces != null ? Collections.unmodifiableSet(builder.excludeNamespaces)
             : null;
     this.bandwidth = builder.bandwidth;
+    this.remoteWALDir = builder.remoteWALDir;
   }
 
   private Map<TableName, List<String>>
@@ -210,6 +213,10 @@ public class ReplicationPeerConfig {
     return this;
   }
 
+  public String getRemoteWALDir() {
+    return this.remoteWALDir;
+  }
+
   public static ReplicationPeerConfigBuilder newBuilder() {
     return new ReplicationPeerConfigBuilderImpl();
   }
@@ -223,7 +230,8 @@ public class ReplicationPeerConfig {
         .setReplicateAllUserTables(peerConfig.replicateAllUserTables())
         .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap())
         .setExcludeNamespaces(peerConfig.getExcludeNamespaces())
-        .setBandwidth(peerConfig.getBandwidth());
+        .setBandwidth(peerConfig.getBandwidth())
+        .setRemoteWALDir(peerConfig.getRemoteWALDir());
     return builder;
   }
 
@@ -250,6 +258,8 @@ public class ReplicationPeerConfig {
 
     private long bandwidth = 0;
 
+    private String remoteWALDir = null;
+
     @Override
     public ReplicationPeerConfigBuilder setClusterKey(String clusterKey) {
       this.clusterKey = clusterKey;
@@ -313,6 +323,12 @@ public class ReplicationPeerConfig {
     }
 
     @Override
+    public ReplicationPeerConfigBuilder setRemoteWALDir(String dir) {
+      this.remoteWALDir = dir;
+      return this;
+    }
+
+    @Override
     public ReplicationPeerConfig build() {
       // It would be nice to validate the configuration, but we have to work with "old" data
       // from ZK which makes it much more difficult.
@@ -341,6 +357,9 @@ public class ReplicationPeerConfig {
       }
     }
     builder.append("bandwidth=").append(bandwidth);
+    if (this.remoteWALDir != null) {
+      builder.append(",remoteWALDir=").append(remoteWALDir);
+    }
     return builder.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
index 0b2f2e2..eac98c6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
@@ -138,6 +138,13 @@ public interface ReplicationPeerConfigBuilder {
   ReplicationPeerConfigBuilder setExcludeNamespaces(Set<String> namespaces);
 
   /**
+   * Set the remote peer cluster's wal directory. Used by synchronous replication.
+   * @param dir the remote peer cluster's wal directory
+   * @return {@code this}
+   */
+  ReplicationPeerConfigBuilder setRemoteWALDir(String dir);
+
+  /**
    * Builds the configuration object from the current state of {@code this}.
    * @return A {@link ReplicationPeerConfig} instance.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/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 9f7b4c2..44295d8 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -48,6 +48,7 @@ message ReplicationPeer {
   optional bool replicate_all = 8;
   repeated TableCF exclude_table_cfs = 9;
   repeated bytes exclude_namespaces = 10;
+  optional string remoteWALDir = 11;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/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 19fc7f4..d715e2e 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
@@ -146,6 +146,21 @@ public class ReplicationPeerManager {
           oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
           " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
     }
+
+    if (!isStringEquals(peerConfig.getRemoteWALDir(), oldPeerConfig.getRemoteWALDir())) {
+      throw new DoNotRetryIOException(
+          "Changing the remote wal dir on an existing peer is not allowed. Existing remote wal " +
+              "dir '" + oldPeerConfig.getRemoteWALDir() + "' for peer " + peerId +
+              " does not match new remote wal dir '" + peerConfig.getRemoteWALDir() + "'");
+    }
+
+    if (oldPeerConfig.getRemoteWALDir() != null) {
+      if (!ReplicationUtils.isKeyConfigEqual(oldPeerConfig, peerConfig)) {
+        throw new DoNotRetryIOException(
+            "Changing the replicated namespace/table config on a synchronous replication "
+                + "peer(peerId: " + peerId + ") is not allowed.");
+      }
+    }
   }
 
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/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 685c560..e471100 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
@@ -906,4 +906,81 @@ public class TestReplicationAdmin {
       // OK
     }
   }
+
+  @Test
+  public void testPeerRemoteWALDir() throws Exception {
+    String rootDir = "hdfs://srv1:9999/hbase";
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_ONE);
+    hbaseAdmin.addReplicationPeer(ID_ONE, builder.build());
+
+    ReplicationPeerConfig rpc = hbaseAdmin.getReplicationPeerConfig(ID_ONE);
+    assertNull(rpc.getRemoteWALDir());
+
+    try {
+      builder.setRemoteWALDir("hdfs://srv2:8888/hbase");
+      hbaseAdmin.updateReplicationPeerConfig(ID_ONE, builder.build());
+      fail("Change remote wal dir is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_SECOND);
+    builder.setRemoteWALDir(rootDir);
+    hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+
+    rpc = hbaseAdmin.getReplicationPeerConfig(ID_SECOND);
+    assertEquals(rootDir, rpc.getRemoteWALDir());
+
+    try {
+      builder.setRemoteWALDir("hdfs://srv2:8888/hbase");
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail("Change remote wal dir is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      builder.setRemoteWALDir(null);
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail("Change remote wal dir is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      builder = ReplicationPeerConfig.newBuilder(rpc);
+      builder.setReplicateAllUserTables(false);
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail(
+        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      builder = ReplicationPeerConfig.newBuilder(rpc);
+      Set<String> namespaces = new HashSet<>();
+      namespaces.add("ns1");
+      builder.setExcludeNamespaces(namespaces);
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail(
+        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      builder = ReplicationPeerConfig.newBuilder(rpc);
+      Map<TableName, List<String>> tableCfs = new HashMap<>();
+      tableCfs.put(TableName.valueOf(name.getMethodName()), new ArrayList<>());
+      builder.setExcludeTableCFsMap(tableCfs);
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail(
+        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index b9d4a0c..ba7d191 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -64,16 +64,20 @@ module Hbase
         table_cfs = args.fetch(TABLE_CFS, nil)
         namespaces = args.fetch(NAMESPACES, nil)
         peer_state = args.fetch(STATE, nil)
+        remote_wal_dir = args.fetch(REMOTE_WAL_DIR, nil)
 
         # Create and populate a ReplicationPeerConfig
-        builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
-          .newBuilder()
+        builder = ReplicationPeerConfig.newBuilder()
         builder.set_cluster_key(cluster_key)
 
         unless endpoint_classname.nil?
           builder.set_replication_endpoint_impl(endpoint_classname)
         end
 
+        unless remote_wal_dir.nil?
+          builder.setRemoteWALDir(remote_wal_dir)
+        end
+
         unless config.nil?
           builder.putAllConfiguration(config)
         end
@@ -228,8 +232,7 @@ module Hbase
           namespaces.each do |n|
             ns_set.add(n)
           end
-          builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
-            .newBuilder(rpc)
+          builder = ReplicationPeerConfig.newBuilder(rpc)
           builder.setNamespaces(ns_set)
           @admin.updateReplicationPeerConfig(id, builder.build)
         end
@@ -248,8 +251,7 @@ module Hbase
               ns_set.remove(n)
             end
           end
-          builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
-            .newBuilder(rpc)
+          builder = ReplicationPeerConfig.newBuilder(rpc)
           builder.setNamespaces(ns_set)
           @admin.updateReplicationPeerConfig(id, builder.build)
         end
@@ -361,8 +363,7 @@ module Hbase
 
       # Create and populate a ReplicationPeerConfig
       replication_peer_config = get_peer_config(id)
-      builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
-        .newBuilder(replication_peer_config)
+      builder = ReplicationPeerConfig.newBuilder(replication_peer_config)
       unless config.nil?
         builder.putAllConfiguration(config)
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/hbase-shell/src/main/ruby/hbase_constants.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb
index 28484cb..2870dfb 100644
--- a/hbase-shell/src/main/ruby/hbase_constants.rb
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -77,6 +77,7 @@ module HBaseConstants
   VALUE = 'VALUE'.freeze
   ENDPOINT_CLASSNAME = 'ENDPOINT_CLASSNAME'.freeze
   CLUSTER_KEY = 'CLUSTER_KEY'.freeze
+  REMOTE_WAL_DIR = 'REMOTE_WAL_DIR'.freeze
   TABLE_CFS = 'TABLE_CFS'.freeze
   NAMESPACES = 'NAMESPACES'.freeze
   STATE = 'STATE'.freeze

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
index eb2da83..4b6f294 100644
--- a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
@@ -35,7 +35,7 @@ to the peer cluster.
 An optional parameter for table column families identifies which tables and/or column families
 will be replicated to the peer cluster.
 
-Notice: Set a namespace in the peer config means that all tables in this namespace
+Note: Set a namespace in the peer config means that all tables in this namespace
 will be replicated to the peer cluster. So if you already have set a namespace in peer config,
 then you can't set this namespace's tables in the peer config again.
 
@@ -74,6 +74,25 @@ the key TABLE_CFS.
 Note: Either CLUSTER_KEY or ENDPOINT_CLASSNAME must be specified. If ENDPOINT_CLASSNAME is specified, CLUSTER_KEY is
 optional and should only be specified if a particular custom endpoint requires it.
 
+The default replication peer is asynchronous. You can also add a synchronous replication peer
+with REMOTE_WAL_DIR parameter. Meanwhile, synchronous replication peer also support other optional
+config for asynchronous replication peer.
+
+Examples:
+
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase"
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    STATE => "ENABLED", REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase"
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    STATE => "DISABLED", REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase"
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase", NAMESPACES => ["ns1", "ns2"]
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase", TABLE_CFS => { "table1" => [] }
+
+Note: The REMOTE_WAL_DIR is not allowed to change.
+
 EOF
       end
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
index 522d23d..caeab86 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
@@ -39,7 +39,8 @@ EOF
         peers = replication_admin.list_peers
 
         formatter.header(%w[PEER_ID CLUSTER_KEY ENDPOINT_CLASSNAME
-                            STATE REPLICATE_ALL NAMESPACES TABLE_CFS BANDWIDTH])
+                            REMOTE_ROOT_DIR STATE REPLICATE_ALL
+                            NAMESPACES TABLE_CFS BANDWIDTH])
 
         peers.each do |peer|
           id = peer.getPeerId
@@ -52,8 +53,20 @@ EOF
             namespaces = replication_admin.show_peer_namespaces(config)
             tableCFs = replication_admin.show_peer_tableCFs_by_config(config)
           end
-          formatter.row([id, config.getClusterKey,
-                         config.getReplicationEndpointImpl, state,
+          cluster_key = 'nil'
+          unless config.getClusterKey.nil?
+            cluster_key = config.getClusterKey
+          end
+          endpoint_classname = 'nil'
+          unless config.getReplicationEndpointImpl.nil?
+            endpoint_classname = config.getReplicationEndpointImpl
+          end
+          remote_root_dir = 'nil'
+          unless config.getRemoteWALDir.nil?
+            remote_root_dir = config.getRemoteWALDir
+          end
+          formatter.row([id, cluster_key, endpoint_classname,
+                         remote_root_dir, state,
                          config.replicateAllUserTables, namespaces, tableCFs,
                          config.getBandwidth])
         end

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b10af08/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
index 0f84396..7f2b6ae 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -97,6 +97,22 @@ module Hbase
       command(:remove_peer, @peer_id)
     end
 
+    define_test "add_peer: remote wal dir" do
+      cluster_key = "server1.cie.com:2181:/hbase"
+      remote_wal_dir = "hdfs://srv1:9999/hbase"
+      args = { CLUSTER_KEY => cluster_key, REMOTE_WAL_DIR => remote_wal_dir }
+      command(:add_peer, @peer_id, args)
+
+      assert_equal(1, command(:list_peers).length)
+      peer = command(:list_peers).get(0)
+      assert_equal(@peer_id, peer.getPeerId)
+      assert_equal(cluster_key, peer.getPeerConfig.getClusterKey)
+      assert_equal(remote_wal_dir, peer.getPeerConfig.getRemoteWALDir)
+
+      # cleanup for future tests
+      command(:remove_peer, @peer_id)
+    end
+
     define_test "add_peer: single zk cluster key with enabled/disabled state" do
       cluster_key = "server1.cie.com:2181:/hbase"
 


[16/18] hbase git commit: HBASE-19747 Introduce a special WALProvider for synchronous replication

Posted by zh...@apache.org.
HBASE-19747 Introduce a special WALProvider for synchronous replication


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

Branch: refs/heads/HBASE-19064
Commit: 4d80122b11061edd8e48b4979bfc2203e5c42bc8
Parents: 4b10af0
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 19 18:38:39 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 8 16:49:19 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AbstractFSWAL.java   |   7 +
 .../hbase/regionserver/wal/AsyncFSWAL.java      |   1 -
 .../hbase/regionserver/wal/DualAsyncFSWAL.java  |   4 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |   3 -
 .../regionserver/PeerActionListener.java        |  33 +++
 .../SynchronousReplicationPeerProvider.java     |  35 +++
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |   1 +
 .../hadoop/hbase/wal/AsyncFSWALProvider.java    |  18 +-
 .../hbase/wal/NettyAsyncFSWALConfigHelper.java  |   8 +-
 .../hbase/wal/RegionGroupingProvider.java       |  13 +-
 .../wal/SynchronousReplicationWALProvider.java  | 225 +++++++++++++++++++
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  37 ++-
 .../org/apache/hadoop/hbase/wal/WALKeyImpl.java |  16 +-
 .../regionserver/TestCompactionPolicy.java      |   1 +
 .../regionserver/TestFailedAppendAndSync.java   | 122 +++++-----
 .../hadoop/hbase/regionserver/TestHRegion.java  |  24 +-
 .../TestHRegionWithInMemoryFlush.java           |   7 -
 .../hbase/regionserver/TestRegionIncrement.java |  20 +-
 .../hbase/regionserver/TestWALLockup.java       |   1 +
 .../regionserver/wal/AbstractTestWALReplay.java |   1 +
 .../regionserver/wal/ProtobufLogTestHelper.java |  44 +++-
 .../hbase/regionserver/wal/TestAsyncFSWAL.java  |  13 +-
 .../regionserver/wal/TestAsyncWALReplay.java    |   4 +-
 .../wal/TestCombinedAsyncWriter.java            |   3 +-
 .../hbase/regionserver/wal/TestFSHLog.java      |  15 +-
 .../hbase/regionserver/wal/TestWALReplay.java   |   1 +
 .../apache/hadoop/hbase/wal/IOTestProvider.java |   2 -
 .../TestSynchronousReplicationWALProvider.java  | 153 +++++++++++++
 28 files changed, 659 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 14fbe10..31b1c54 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -430,6 +430,13 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     this.implClassName = getClass().getSimpleName();
   }
 
+  /**
+   * Used to initialize the WAL. Usually just call rollWriter to create the first log writer.
+   */
+  public void init() throws IOException {
+    rollWriter();
+  }
+
   @Override
   public void registerWALActionsListener(WALActionsListener listener) {
     this.listeners.add(listener);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index 8e57441..ac72dc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -247,7 +247,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE);
     waitOnShutdownInSeconds = conf.getInt(ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS,
       DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS);
-    rollWriter();
   }
 
   private static boolean waitingRoll(int epochAndState) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
index 6bf9e02..f92ce93 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
@@ -38,14 +38,14 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
 
   private final Path remoteWalDir;
 
-  public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteRootDir,
+  public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteWalDir,
       String logDir, String archiveDir, Configuration conf, List<WALActionsListener> listeners,
       boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup,
       Class<? extends Channel> channelClass) throws FailedLogCloseException, IOException {
     super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix,
         eventLoopGroup, channelClass);
     this.remoteFs = remoteFs;
-    this.remoteWalDir = new Path(remoteRootDir, logDir);
+    this.remoteWalDir = remoteWalDir;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 7ff0453..3fef09b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -212,9 +212,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
       5);
     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 2);
 
-    // rollWriter sets this.hdfs_out if it can.
-    rollWriter();
-
     // This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is
     // put on the ring buffer.
     String hostingThreadName = Thread.currentThread().getName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
new file mode 100644
index 0000000..74ad626
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.yetus.audience.InterfaceAudience;
+
+/**
+ * Get notification for replication peer events. Mainly used for telling the
+ * {@link org.apache.hadoop.hbase.wal.SynchronousReplicationWALProvider} to close some WAL if not
+ * used any more.
+ * <p>
+ * TODO: Also need a synchronous peer state change notification.
+ */
+@InterfaceAudience.Private
+public interface PeerActionListener {
+
+  default void peerRemoved(String peerId) {}
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
new file mode 100644
index 0000000..b4e04fb
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.Optional;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Get the peer id and remote root dir if the region is synchronously replicated.
+ */
+@InterfaceAudience.Private
+public interface SynchronousReplicationPeerProvider {
+
+  /**
+   * Return the peer id and remote WAL directory if the region is synchronously replicated.
+   */
+  Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index 231afd5..3eb8f8f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -137,6 +137,7 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
         if (walCopy == null) {
           walCopy = createWAL();
           wal = walCopy;
+          walCopy.init();
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
index 9c62bed..84e859d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.wal;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,12 +30,10 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
 
 /**
  * A WAL provider that use {@link AsyncFSWAL}.
@@ -61,6 +58,7 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
   private EventLoopGroup eventLoopGroup;
 
   private Class<? extends Channel> channelClass;
+
   @Override
   protected AsyncFSWAL createWAL() throws IOException {
     return new AsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf),
@@ -73,15 +71,9 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
   @Override
   protected void doInit(Configuration conf) throws IOException {
     Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
-        NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
-    if (eventLoopGroupAndChannelClass != null) {
-      eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
-      channelClass = eventLoopGroupAndChannelClass.getSecond();
-    } else {
-      eventLoopGroup = new NioEventLoopGroup(1,
-          new DefaultThreadFactory("AsyncFSWAL", true, Thread.MAX_PRIORITY));
-      channelClass = NioSocketChannel.class;
-    }
+      NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
+    eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
+    channelClass = eventLoopGroupAndChannelClass.getSecond();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
index 0836b5d..00ccb71 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
 
 /**
  * Helper class for passing netty event loop config to {@link AsyncFSWALProvider}.
@@ -55,7 +58,10 @@ public class NettyAsyncFSWALConfigHelper {
   static Pair<EventLoopGroup, Class<? extends Channel>> getEventLoopConfig(Configuration conf) {
     String name = conf.get(EVENT_LOOP_CONFIG);
     if (StringUtils.isBlank(name)) {
-      return null;
+      // create new event loop group if config is empty
+      return Pair.<EventLoopGroup, Class<? extends Channel>> newPair(
+        new NioEventLoopGroup(0, new DefaultThreadFactory("AsyncFSWAL", true, Thread.MAX_PRIORITY)),
+        NioSocketChannel.class);
     }
     return EVENT_LOOP_CONFIG_MAP.get(name);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
index 28817e9..0b7b8da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
@@ -31,6 +31,7 @@ import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 // imports for classes still in regionserver.wal
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -132,6 +133,7 @@ public class RegionGroupingProvider implements WALProvider {
 
   private RegionGroupingStrategy strategy;
   private WALFactory factory;
+  private Configuration conf;
   private List<WALActionsListener> listeners = new ArrayList<>();
   private String providerId;
   private Class<? extends WALProvider> providerClass;
@@ -141,6 +143,7 @@ public class RegionGroupingProvider implements WALProvider {
     if (null != strategy) {
       throw new IllegalStateException("WALProvider.init should only be called once.");
     }
+    this.conf = conf;
     this.factory = factory;
     StringBuilder sb = new StringBuilder().append(factory.factoryId);
     if (providerId != null) {
@@ -156,11 +159,11 @@ public class RegionGroupingProvider implements WALProvider {
   }
 
   private WALProvider createProvider(String group) throws IOException {
-    if (META_WAL_PROVIDER_ID.equals(providerId)) {
-      return factory.createProvider(providerClass, META_WAL_PROVIDER_ID);
-    } else {
-      return factory.createProvider(providerClass, group);
-    }
+    WALProvider provider = WALFactory.createProvider(providerClass);
+    provider.init(factory, conf,
+      META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : group);
+    provider.addWALActionsListener(new MetricsWAL());
+    return provider;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
new file mode 100644
index 0000000..f60599f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.wal;
+
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALArchiveDirectoryName;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALDirectoryName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
+import org.apache.hadoop.hbase.replication.regionserver.SynchronousReplicationPeerProvider;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.KeyLocker;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Streams;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
+/**
+ * The special {@link WALProvider} for synchronous replication.
+ * <p>
+ * It works like an interceptor, when getting WAL, first it will check if the given region should be
+ * replicated synchronously, if so it will return a special WAL for it, otherwise it will delegate
+ * the request to the normal {@link WALProvider}.
+ */
+@InterfaceAudience.Private
+public class SynchronousReplicationWALProvider implements WALProvider, PeerActionListener {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(SynchronousReplicationWALProvider.class);
+
+  private static final String LOG_SUFFIX = ".syncrep";
+
+  private final WALProvider provider;
+
+  private final SynchronousReplicationPeerProvider peerProvider;
+
+  private WALFactory factory;
+
+  private Configuration conf;
+
+  private List<WALActionsListener> listeners = new ArrayList<>();
+
+  private EventLoopGroup eventLoopGroup;
+
+  private Class<? extends Channel> channelClass;
+
+  private AtomicBoolean initialized = new AtomicBoolean(false);
+
+  private final ConcurrentMap<String, DualAsyncFSWAL> peerId2WAL = new ConcurrentHashMap<>();
+
+  private final KeyLocker<String> createLock = new KeyLocker<>();
+
+  SynchronousReplicationWALProvider(WALProvider provider,
+      SynchronousReplicationPeerProvider peerProvider) {
+    this.provider = provider;
+    this.peerProvider = peerProvider;
+  }
+
+  @Override
+  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
+    if (!initialized.compareAndSet(false, true)) {
+      throw new IllegalStateException("WALProvider.init should only be called once.");
+    }
+    provider.init(factory, conf, providerId);
+    this.conf = conf;
+    this.factory = factory;
+    Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
+      NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
+    eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
+    channelClass = eventLoopGroupAndChannelClass.getSecond();
+  }
+
+  private String getLogPrefix(String peerId) {
+    return factory.factoryId + WAL_FILE_NAME_DELIMITER + peerId;
+  }
+
+  private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException {
+    Path remoteWALDirPath = new Path(remoteWALDir);
+    FileSystem remoteFs = remoteWALDirPath.getFileSystem(conf);
+    return new DualAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), remoteFs,
+        CommonFSUtils.getWALRootDir(conf), new Path(remoteWALDirPath, peerId),
+        getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
+        conf, listeners, true, getLogPrefix(peerId), LOG_SUFFIX, eventLoopGroup, channelClass);
+  }
+
+  private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException {
+    DualAsyncFSWAL wal = peerId2WAL.get(peerId);
+    if (wal != null) {
+      return wal;
+    }
+    Lock lock = createLock.acquireLock(peerId);
+    try {
+      wal = peerId2WAL.get(peerId);
+      if (wal == null) {
+        wal = createWAL(peerId, remoteWALDir);
+        peerId2WAL.put(peerId, wal);
+        wal.init();
+      }
+      return wal;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public WAL getWAL(RegionInfo region) throws IOException {
+    Optional<Pair<String, String>> peerIdAndRemoteWALDir =
+      peerProvider.getPeerIdAndRemoteWALDir(region);
+    if (peerIdAndRemoteWALDir.isPresent()) {
+      Pair<String, String> pair = peerIdAndRemoteWALDir.get();
+      return getWAL(pair.getFirst(), pair.getSecond());
+    } else {
+      return provider.getWAL(region);
+    }
+  }
+
+  private Stream<WAL> getWALStream() {
+    return Streams.concat(peerId2WAL.values().stream(), provider.getWALs().stream());
+  }
+
+  @Override
+  public List<WAL> getWALs() {
+    return getWALStream().collect(Collectors.toList());
+  }
+
+  @Override
+  public void shutdown() throws IOException {
+    // save the last exception and rethrow
+    IOException failure = null;
+    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
+      try {
+        wal.shutdown();
+      } catch (IOException e) {
+        LOG.error("Shutdown WAL failed", e);
+        failure = e;
+      }
+    }
+    provider.shutdown();
+    if (failure != null) {
+      throw failure;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // save the last exception and rethrow
+    IOException failure = null;
+    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
+      try {
+        wal.close();
+      } catch (IOException e) {
+        LOG.error("Close WAL failed", e);
+        failure = e;
+      }
+    }
+    provider.close();
+    if (failure != null) {
+      throw failure;
+    }
+  }
+
+  @Override
+  public long getNumLogFiles() {
+    return peerId2WAL.size() + provider.getNumLogFiles();
+  }
+
+  @Override
+  public long getLogFileSize() {
+    return peerId2WAL.values().stream().mapToLong(DualAsyncFSWAL::getLogFileSize).sum() +
+      provider.getLogFileSize();
+  }
+
+  @Override
+  public void peerRemoved(String peerId) {
+    WAL wal = peerId2WAL.remove(peerId);
+    if (wal != null) {
+      try {
+        wal.close();
+      } catch (IOException e) {
+        LOG.error("Close WAL failed", e);
+      }
+    }
+  }
+
+  @Override
+  public void addWALActionsListener(WALActionsListener listener) {
+    listeners.add(listener);
+    provider.addWALActionsListener(listener);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 1410b53..4e519ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.replication.regionserver.SynchronousReplicationPeerProvider;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -130,13 +131,10 @@ public class WALFactory {
     }
   }
 
-  WALProvider createProvider(Class<? extends WALProvider> clazz, String providerId)
-      throws IOException {
-    LOG.info("Instantiating WALProvider of type " + clazz);
+  static WALProvider createProvider(Class<? extends WALProvider> clazz) throws IOException {
+    LOG.info("Instantiating WALProvider of type {}", clazz);
     try {
-      final WALProvider result = clazz.getDeclaredConstructor().newInstance();
-      result.init(this, conf, providerId);
-      return result;
+      return clazz.newInstance();
     } catch (Exception e) {
       LOG.error("couldn't set up WALProvider, the configured class is " + clazz);
       LOG.debug("Exception details for failure to load WALProvider.", e);
@@ -148,9 +146,10 @@ public class WALFactory {
    * instantiate a provider from a config property. requires conf to have already been set (as well
    * as anything the provider might need to read).
    */
-  WALProvider getProvider(String key, String defaultValue, String providerId) throws IOException {
-    Class<? extends WALProvider> clazz = getProviderClass(key, defaultValue);
-    WALProvider provider = createProvider(clazz, providerId);
+  private WALProvider getProvider(String key, String defaultValue, String providerId)
+      throws IOException {
+    WALProvider provider = createProvider(getProviderClass(key, defaultValue));
+    provider.init(this, conf, providerId);
     provider.addWALActionsListener(new MetricsWAL());
     return provider;
   }
@@ -182,6 +181,26 @@ public class WALFactory {
   }
 
   /**
+   * A temporary constructor for testing synchronous replication.
+   * <p>
+   * Remove it once we can integrate the synchronous replication logic in RS.
+   */
+  @VisibleForTesting
+  WALFactory(Configuration conf, String factoryId, SynchronousReplicationPeerProvider peerProvider)
+      throws IOException {
+    timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000);
+    /* TODO Both of these are probably specific to the fs wal provider */
+    logReaderClass = conf.getClass("hbase.regionserver.hlog.reader.impl", ProtobufLogReader.class,
+      AbstractFSWALProvider.Reader.class);
+    this.conf = conf;
+    this.factoryId = factoryId;
+    WALProvider provider = createProvider(getProviderClass(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
+    this.provider = new SynchronousReplicationWALProvider(provider, peerProvider);
+    this.provider.addWALActionsListener(new MetricsWAL());
+    this.provider.init(this, conf, null);
+  }
+
+  /**
    * Shutdown all WALs and clean up any underlying storage.
    * Use only when you will not need to replay and edits that have gone to any wals from this
    * factory.

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
index 7abd4a5..d24bd2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
@@ -135,13 +135,21 @@ public class WALKeyImpl implements WALKey {
   }
 
   @VisibleForTesting
-  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename,
-                long logSeqNum,
+  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
       final long now, UUID clusterId) {
     List<UUID> clusterIds = new ArrayList<>(1);
     clusterIds.add(clusterId);
-    init(encodedRegionName, tablename, logSeqNum, now, clusterIds,
-        HConstants.NO_NONCE, HConstants.NO_NONCE, null, null);
+    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, HConstants.NO_NONCE,
+      HConstants.NO_NONCE, null, null);
+  }
+
+  @VisibleForTesting
+  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
+      final long now, UUID clusterId, MultiVersionConcurrencyControl mvcc) {
+    List<UUID> clusterIds = new ArrayList<>(1);
+    clusterIds.add(clusterId);
+    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, HConstants.NO_NONCE,
+      HConstants.NO_NONCE, mvcc, null);
   }
 
   // TODO: Fix being able to pass in sequenceid.

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index ca4b227..939f35c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -100,6 +100,7 @@ public class TestCompactionPolicy {
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
 
     hlog = new FSHLog(fs, basedir, logName, conf);
+    hlog.init();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     region = HRegion.createHRegion(info, basedir, conf, htd, hlog);
     region.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index e27a986..f9eb534 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -102,65 +102,64 @@ public class TestFailedAppendAndSync {
     return name.getMethodName();
   }
 
-  /**
-   * Reproduce locking up that happens when we get an exceptions appending and syncing.
-   * See HBASE-14317.
-   * First I need to set up some mocks for Server and RegionServerServices. I also need to
-   * set up a dodgy WAL that will throw an exception when we go to append to it.
-   */
-  @Test (timeout=300000)
-  public void testLockupAroundBadAssignSync() throws IOException {
+  // Dodgy WAL. Will throw exceptions when flags set.
+  class DodgyFSLog extends FSHLog {
+    volatile boolean throwSyncException = false;
+    volatile boolean throwAppendException = false;
     final AtomicLong rolls = new AtomicLong(0);
-    // Dodgy WAL. Will throw exceptions when flags set.
-    class DodgyFSLog extends FSHLog {
-      volatile boolean throwSyncException = false;
-      volatile boolean throwAppendException = false;
 
-      public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
-      throws IOException {
-        super(fs, root, logDir, conf);
-      }
-
-      @Override
-      public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
-        byte [][] regions = super.rollWriter(force);
-        rolls.getAndIncrement();
-        return regions;
-      }
+    public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
+        throws IOException {
+      super(fs, root, logDir, conf);
+    }
 
-      @Override
-      protected Writer createWriterInstance(Path path) throws IOException {
-        final Writer w = super.createWriterInstance(path);
-          return new Writer() {
-            @Override
-            public void close() throws IOException {
-              w.close();
-            }
+    @Override
+    public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
+      byte[][] regions = super.rollWriter(force);
+      rolls.getAndIncrement();
+      return regions;
+    }
 
-            @Override
-            public void sync() throws IOException {
-              if (throwSyncException) {
-                throw new IOException("FAKE! Failed to replace a bad datanode...");
-              }
-              w.sync();
-            }
+    @Override
+    protected Writer createWriterInstance(Path path) throws IOException {
+      final Writer w = super.createWriterInstance(path);
+      return new Writer() {
+        @Override
+        public void close() throws IOException {
+          w.close();
+        }
 
-            @Override
-            public void append(Entry entry) throws IOException {
-              if (throwAppendException) {
-                throw new IOException("FAKE! Failed to replace a bad datanode...");
-              }
-              w.append(entry);
-            }
+        @Override
+        public void sync() throws IOException {
+          if (throwSyncException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...");
+          }
+          w.sync();
+        }
 
-            @Override
-            public long getLength() {
-              return w.getLength();
-              }
-            };
+        @Override
+        public void append(Entry entry) throws IOException {
+          if (throwAppendException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...");
           }
-      }
+          w.append(entry);
+        }
 
+        @Override
+        public long getLength() {
+          return w.getLength();
+        }
+      };
+    }
+  }
+  /**
+   * Reproduce locking up that happens when we get an exceptions appending and syncing.
+   * See HBASE-14317.
+   * First I need to set up some mocks for Server and RegionServerServices. I also need to
+   * set up a dodgy WAL that will throw an exception when we go to append to it.
+   */
+  @Test (timeout=300000)
+  public void testLockupAroundBadAssignSync() throws IOException {
     // Make up mocked server and services.
     Server server = mock(Server.class);
     when(server.getConfiguration()).thenReturn(CONF);
@@ -172,6 +171,7 @@ public class TestFailedAppendAndSync {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + getName());
     DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
+    dodgyWAL.init();
     LogRoller logRoller = new LogRoller(server, services);
     logRoller.addWAL(dodgyWAL);
     logRoller.start();
@@ -192,7 +192,7 @@ public class TestFailedAppendAndSync {
       } catch (IOException ioe) {
         fail();
       }
-      long rollsCount = rolls.get();
+      long rollsCount = dodgyWAL.rolls.get();
       try {
         dodgyWAL.throwAppendException = true;
         dodgyWAL.throwSyncException = false;
@@ -202,8 +202,10 @@ public class TestFailedAppendAndSync {
       } catch (IOException ioe) {
         threwOnAppend = true;
       }
-      while (rollsCount == rolls.get()) Threads.sleep(100);
-      rollsCount = rolls.get();
+      while (rollsCount == dodgyWAL.rolls.get()) {
+        Threads.sleep(100);
+      }
+      rollsCount = dodgyWAL.rolls.get();
 
       // When we get to here.. we should be ok. A new WAL has been put in place. There were no
       // appends to sync. We should be able to continue.
@@ -217,14 +219,16 @@ public class TestFailedAppendAndSync {
       } catch (IOException ioe) {
         threwOnBoth = true;
       }
-      while (rollsCount == rolls.get()) Threads.sleep(100);
+      while (rollsCount == dodgyWAL.rolls.get()) {
+        Threads.sleep(100);
+      }
 
       // Again, all should be good. New WAL and no outstanding unsync'd edits so we should be able
       // to just continue.
 
       // So, should be no abort at this stage. Verify.
-      Mockito.verify(server, Mockito.atLeast(0)).
-        abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
+      Mockito.verify(server, Mockito.atLeast(0)).abort(Mockito.anyString(),
+        Mockito.any(Throwable.class));
       try {
         dodgyWAL.throwAppendException = false;
         dodgyWAL.throwSyncException = true;
@@ -239,8 +243,8 @@ public class TestFailedAppendAndSync {
       // happens. If it don't we'll timeout the whole test. That is fine.
       while (true) {
         try {
-          Mockito.verify(server, Mockito.atLeast(1)).
-            abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
+          Mockito.verify(server, Mockito.atLeast(1)).abort(Mockito.anyString(),
+            Mockito.any(Throwable.class));
           break;
         } catch (WantedButNotInvoked t) {
           Threads.sleep(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 88e1aa2..df24e0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -217,7 +217,6 @@ public class TestHRegion {
   protected static HBaseTestingUtility TEST_UTIL;
   public static Configuration CONF ;
   private String dir;
-  private static FileSystem FILESYSTEM;
   private final int MAX_VERSIONS = 2;
 
   // Test names
@@ -239,7 +238,6 @@ public class TestHRegion {
   @Before
   public void setup() throws IOException {
     TEST_UTIL = HBaseTestingUtility.createLocalHTU();
-    FILESYSTEM = TEST_UTIL.getTestFileSystem();
     CONF = TEST_UTIL.getConfiguration();
     dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
     method = name.getMethodName();
@@ -341,6 +339,7 @@ public class TestHRegion {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + "testMemstoreSnapshotSize");
     MyFaultyFSLog faultyLog = new MyFaultyFSLog(fs, rootDir, "testMemstoreSnapshotSize", CONF);
+    faultyLog.init();
     HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, faultyLog,
         COLUMN_FAMILY_BYTES);
 
@@ -352,7 +351,6 @@ public class TestHRegion {
     Put put = new Put(value);
     put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
     faultyLog.setFailureType(FaultyFSLog.FailureType.SYNC);
-
     boolean threwIOE = false;
     try {
       region.put(put);
@@ -388,6 +386,7 @@ public class TestHRegion {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + testName);
     FSHLog hLog = new FSHLog(fs, rootDir, testName, CONF);
+    hLog.init();
     HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog,
         COLUMN_FAMILY_BYTES);
     HStore store = region.getStore(COLUMN_FAMILY_BYTES);
@@ -1162,6 +1161,7 @@ public class TestHRegion {
     FailAppendFlushMarkerWAL wal =
       new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
         method, walConf);
+    wal.init();
     this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family);
     try {
@@ -1193,7 +1193,7 @@ public class TestHRegion {
       wal.flushActions = new FlushAction [] {FlushAction.COMMIT_FLUSH};
       wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
             method, walConf);
-
+      wal.init();
       this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
         HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family);
       region.put(put);
@@ -2445,6 +2445,7 @@ public class TestHRegion {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + "testDataInMemoryWithoutWAL");
     FSHLog hLog = new FSHLog(fs, rootDir, "testDataInMemoryWithoutWAL", CONF);
+    hLog.init();
     // This chunk creation is done throughout the code base. Do we want to move it into core?
     // It is missing from this test. W/o it we NPE.
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
@@ -2497,9 +2498,9 @@ public class TestHRegion {
     RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
     // Because the preBatchMutate returns void, we can't do usual Mockito when...then form. Must
     // do below format (from Mockito doc).
-    Mockito.doAnswer(new Answer() {
+    Mockito.doAnswer(new Answer<Void>() {
       @Override
-      public Object answer(InvocationOnMock invocation) throws Throwable {
+      public Void answer(InvocationOnMock invocation) throws Throwable {
         MiniBatchOperationInProgress<Mutation> mb = invocation.getArgument(0);
         mb.addOperationsFromCP(0, new Mutation[]{addPut});
         return null;
@@ -3793,9 +3794,12 @@ public class TestHRegion {
 
         boolean previousEmpty = res.isEmpty();
         res.clear();
-        InternalScanner scanner = region.getScanner(scan);
-        while (scanner.next(res))
-          ;
+        try (InternalScanner scanner = region.getScanner(scan)) {
+          boolean moreRows;
+          do {
+            moreRows = scanner.next(res);
+          } while (moreRows);
+        }
         if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
           assertEquals("i=" + i, expectedCount, res.size());
           long timestamp = res.get(0).getTimestamp();
@@ -3891,7 +3895,7 @@ public class TestHRegion {
             region.put(put);
             numPutsFinished++;
             if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
-              System.out.println("put iteration = " + numPutsFinished);
+              LOG.debug("put iteration = {}", numPutsFinished);
               Delete delete = new Delete(row, (long) numPutsFinished - 30);
               region.delete(delete);
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
index dfe52d0..58f62e3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
@@ -27,25 +27,18 @@ import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A test similar to TestHRegion, but with in-memory flush families.
  * Also checks wal truncation after in-memory compaction.
  */
 @Category({VerySlowRegionServerTests.class, LargeTests.class})
-@SuppressWarnings("deprecation")
 public class TestHRegionWithInMemoryFlush extends TestHRegion{
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestHRegionWithInMemoryFlush.class);
 
-  // Do not spin up clusters in here. If you need to spin up a cluster, do it
-  // over in TestHRegionOnCluster.
-  private static final Logger LOG = LoggerFactory.getLogger(TestHRegionWithInMemoryFlush.class);
-
   /**
    * @return A region on which you must call
    *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
index 8b96fa7..e5006ea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Scan;
@@ -36,7 +37,6 @@ import org.apache.hadoop.hbase.client.TestIncrementsFromClientSide;
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -81,12 +81,12 @@ public class TestRegionIncrement {
   }
 
   private HRegion getRegion(final Configuration conf, final String tableName) throws IOException {
-    WAL wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
-      TEST_UTIL.getDataTestDir().toString(), conf);
+    FSHLog wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
+        TEST_UTIL.getDataTestDir().toString(), conf);
+    wal.init();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-    return (HRegion)TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName),
-      HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, tableName, conf,
-      false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);
+    return TEST_UTIL.createLocalHRegion(TableName.valueOf(tableName), HConstants.EMPTY_BYTE_ARRAY,
+      HConstants.EMPTY_BYTE_ARRAY, false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);
   }
 
   private void closeRegion(final HRegion region) throws IOException {
@@ -170,8 +170,6 @@ public class TestRegionIncrement {
 
   /**
    * Have each thread update its own Cell. Avoid contention with another thread.
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
   public void testUnContendedSingleCellIncrement()
@@ -209,13 +207,9 @@ public class TestRegionIncrement {
 
   /**
    * Have each thread update its own Cell. Avoid contention with another thread.
-   * This is
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
-  public void testContendedAcrossCellsIncrement()
-  throws IOException, InterruptedException {
+  public void testContendedAcrossCellsIncrement() throws IOException, InterruptedException {
     final HRegion region = getRegion(TEST_UTIL.getConfiguration(),
         TestIncrementsFromClientSide.filterStringSoTableNameSafe(this.name.getMethodName()));
     long startTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index ca65914..8913343 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -214,6 +214,7 @@ public class TestWALLockup {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + getName());
     DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
+    dodgyWAL.init();
     Path originalWAL = dodgyWAL.getCurrentFileName();
     // I need a log roller running.
     LogRoller logRoller = new LogRoller(server, services);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index b1e304e..7600fe9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -1097,6 +1097,7 @@ public abstract class AbstractTestWALReplay {
 
   private MockWAL createMockWAL() throws IOException {
     MockWAL wal = new MockWAL(fs, hbaseRootDir, logName, conf);
+    wal.init();
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
index ecd8e6c..49633cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -41,7 +42,7 @@ import org.apache.hadoop.hbase.wal.WALProvider;
 /**
  * Helper class for testing protobuf log.
  */
-final class ProtobufLogTestHelper {
+public final class ProtobufLogTestHelper {
 
   private ProtobufLogTestHelper() {
   }
@@ -54,17 +55,22 @@ final class ProtobufLogTestHelper {
     return RegionInfoBuilder.newBuilder(tableName).setRegionId(1024).build();
   }
 
+  private static WAL.Entry generateEdit(int i, RegionInfo hri, TableName tableName, byte[] row,
+      int columnCount, long timestamp, MultiVersionConcurrencyControl mvcc) {
+    WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, i, timestamp,
+        HConstants.DEFAULT_CLUSTER_ID, mvcc);
+    WALEdit edit = new WALEdit();
+    int prefix = i;
+    IntStream.range(0, columnCount).mapToObj(j -> toValue(prefix, j))
+        .map(value -> new KeyValue(row, row, row, timestamp, value)).forEachOrdered(edit::add);
+    return new WAL.Entry(key, edit);
+  }
+
   public static void doWrite(WALProvider.Writer writer, boolean withTrailer, TableName tableName,
       int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
     RegionInfo hri = toRegionInfo(tableName);
     for (int i = 0; i < recordCount; i++) {
-      WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, i, timestamp,
-          HConstants.DEFAULT_CLUSTER_ID);
-      WALEdit edit = new WALEdit();
-      int prefix = i;
-      IntStream.range(0, columnCount).mapToObj(j -> toValue(prefix, j))
-          .map(value -> new KeyValue(row, row, row, timestamp, value)).forEachOrdered(edit::add);
-      writer.append(new WAL.Entry(key, edit));
+      writer.append(generateEdit(i, hri, tableName, row, columnCount, timestamp, null));
     }
     writer.sync();
     if (withTrailer) {
@@ -72,14 +78,24 @@ final class ProtobufLogTestHelper {
     }
   }
 
-  public static void doRead(ProtobufLogReader reader, boolean withTrailer, TableName tableName,
-      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+  public static void doWrite(WAL wal, RegionInfo hri, TableName tableName, int columnCount,
+      int recordCount, byte[] row, long timestamp, MultiVersionConcurrencyControl mvcc)
+      throws IOException {
+    for (int i = 0; i < recordCount; i++) {
+      WAL.Entry entry = generateEdit(i, hri, tableName, row, columnCount, timestamp, mvcc);
+      wal.append(hri, entry.getKey(), entry.getEdit(), true);
+    }
+    wal.sync();
+  }
+
+  public static void doRead(ProtobufLogReader reader, boolean withTrailer, RegionInfo hri,
+      TableName tableName, int columnCount, int recordCount, byte[] row, long timestamp)
+      throws IOException {
     if (withTrailer) {
       assertNotNull(reader.trailer);
     } else {
       assertNull(reader.trailer);
     }
-    RegionInfo hri = toRegionInfo(tableName);
     for (int i = 0; i < recordCount; ++i) {
       WAL.Entry entry = reader.next();
       assertNotNull(entry);
@@ -96,4 +112,10 @@ final class ProtobufLogTestHelper {
     }
     assertNull(reader.next());
   }
+
+  public static void doRead(ProtobufLogReader reader, boolean withTrailer, TableName tableName,
+      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+    doRead(reader, withTrailer, toRegionInfo(tableName), tableName, columnCount, recordCount, row,
+      timestamp);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
index 450c01b..5f0f77c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
@@ -67,8 +67,10 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL {
   protected AbstractFSWAL<?> newWAL(FileSystem fs, Path rootDir, String logDir, String archiveDir,
       Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
       String prefix, String suffix) throws IOException {
-    return new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix,
-        suffix, GROUP, CHANNEL_CLASS);
+    AsyncFSWAL wal = new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners,
+        failIfWALExists, prefix, suffix, GROUP, CHANNEL_CLASS);
+    wal.init();
+    return wal;
   }
 
   @Override
@@ -76,15 +78,16 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL {
       String archiveDir, Configuration conf, List<WALActionsListener> listeners,
       boolean failIfWALExists, String prefix, String suffix, final Runnable action)
       throws IOException {
-    return new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix,
-        suffix, GROUP, CHANNEL_CLASS) {
+    AsyncFSWAL wal = new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners,
+        failIfWALExists, prefix, suffix, GROUP, CHANNEL_CLASS) {
 
       @Override
       void atHeadOfRingBufferEventHandlerAppend() {
         action.run();
         super.atHeadOfRingBufferEventHandlerAppend();
       }
-
     };
+    wal.init();
+    return wal;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
index 80b7477..0740954 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
@@ -66,7 +66,9 @@ public class TestAsyncWALReplay extends AbstractTestWALReplay {
 
   @Override
   protected WAL createWAL(Configuration c, Path hbaseRootDir, String logName) throws IOException {
-    return new AsyncFSWAL(FileSystem.get(c), hbaseRootDir, logName,
+    AsyncFSWAL wal = new AsyncFSWAL(FileSystem.get(c), hbaseRootDir, logName,
         HConstants.HREGION_OLDLOGDIR_NAME, c, null, true, null, null, GROUP, CHANNEL_CLASS);
+    wal.init();
+    return wal;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
index d74f9d8..36dbe0f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
@@ -77,8 +77,7 @@ public class TestCombinedAsyncWriter {
     CHANNEL_CLASS = NioSocketChannel.class;
     UTIL.startMiniDFSCluster(3);
     UTIL.getTestFileSystem().mkdirs(UTIL.getDataTestDirOnTestFS());
-    WALS =
-      new WALFactory(UTIL.getConfiguration(), null, TestCombinedAsyncWriter.class.getSimpleName());
+    WALS = new WALFactory(UTIL.getConfiguration(), TestCombinedAsyncWriter.class.getSimpleName());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
index 9c789ba..3502428 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
@@ -72,8 +72,10 @@ public class TestFSHLog extends AbstractTestFSWAL {
   protected AbstractFSWAL<?> newWAL(FileSystem fs, Path rootDir, String walDir, String archiveDir,
       Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
       String prefix, String suffix) throws IOException {
-    return new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix,
-        suffix);
+    FSHLog wal =
+      new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
+    wal.init();
+    return wal;
   }
 
   @Override
@@ -81,8 +83,8 @@ public class TestFSHLog extends AbstractTestFSWAL {
       String archiveDir, Configuration conf, List<WALActionsListener> listeners,
       boolean failIfWALExists, String prefix, String suffix, final Runnable action)
       throws IOException {
-    return new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix,
-        suffix) {
+    FSHLog wal = new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists,
+        prefix, suffix) {
 
       @Override
       void atHeadOfRingBufferEventHandlerAppend() {
@@ -90,6 +92,8 @@ public class TestFSHLog extends AbstractTestFSWAL {
         super.atHeadOfRingBufferEventHandlerAppend();
       }
     };
+    wal.init();
+    return wal;
   }
 
   @Test
@@ -98,6 +102,7 @@ public class TestFSHLog extends AbstractTestFSWAL {
     final String name = this.name.getMethodName();
     FSHLog log = new FSHLog(FS, FSUtils.getRootDir(CONF), name, HConstants.HREGION_OLDLOGDIR_NAME,
         CONF, null, true, null, null);
+    log.init();
     try {
       Field ringBufferEventHandlerField = FSHLog.class.getDeclaredField("ringBufferEventHandler");
       ringBufferEventHandlerField.setAccessible(true);
@@ -140,7 +145,7 @@ public class TestFSHLog extends AbstractTestFSWAL {
     try (FSHLog log =
         new FSHLog(FS, FSUtils.getRootDir(CONF), name, HConstants.HREGION_OLDLOGDIR_NAME, CONF,
             null, true, null, null)) {
-
+      log.init();
       log.registerWALActionsListener(new WALActionsListener() {
         @Override
         public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit)

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index 649e981..66e19a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -48,6 +48,7 @@ public class TestWALReplay extends AbstractTestWALReplay {
   @Override
   protected WAL createWAL(Configuration c, Path hbaseRootDir, String logName) throws IOException {
     FSHLog wal = new FSHLog(FileSystem.get(c), hbaseRootDir, logName, c);
+    wal.init();
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
index 3928d9c..f996ce0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -104,8 +104,6 @@ public class IOTestProvider implements WALProvider {
     this.factory = factory;
     this.conf = conf;
     this.providerId = providerId != null ? providerId : DEFAULT_PROVIDER_ID;
-
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d80122b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
new file mode 100644
index 0000000..e6031c6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.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.wal;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogTestHelper;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestSynchronousReplicationWALProvider {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID = "1";
+
+  private static String REMOTE_WAL_DIR = "/RemoteWAL";
+
+  private static TableName TABLE = TableName.valueOf("table");
+
+  private static TableName TABLE_NO_REP = TableName.valueOf("table-no-rep");
+
+  private static RegionInfo REGION = RegionInfoBuilder.newBuilder(TABLE).build();
+
+  private static RegionInfo REGION_NO_REP = RegionInfoBuilder.newBuilder(TABLE_NO_REP).build();
+
+  private static WALFactory FACTORY;
+
+  private static Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
+    if (info.getTable().equals(TABLE)) {
+      return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR));
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.startMiniDFSCluster(3);
+    FACTORY = new WALFactory(UTIL.getConfiguration(), "test",
+        TestSynchronousReplicationWALProvider::getPeerIdAndRemoteWALDir);
+    UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID));
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws IOException {
+    FACTORY.close();
+    UTIL.shutdownMiniDFSCluster();
+  }
+
+  private void testReadWrite(DualAsyncFSWAL wal) throws Exception {
+    int recordCount = 100;
+    int columnCount = 10;
+    byte[] row = Bytes.toBytes("testRow");
+    long timestamp = System.currentTimeMillis();
+    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
+    ProtobufLogTestHelper.doWrite(wal, REGION, TABLE, columnCount, recordCount, row, timestamp,
+      mvcc);
+    Path localFile = wal.getCurrentFileName();
+    Path remoteFile = new Path(REMOTE_WAL_DIR + "/" + PEER_ID, localFile.getName());
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
+      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
+      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    wal.rollWriter();
+    DistributedFileSystem dfs = (DistributedFileSystem) UTIL.getDFSCluster().getFileSystem();
+    UTIL.waitFor(5000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return dfs.isFileClosed(localFile) && dfs.isFileClosed(remoteFile);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        StringBuilder sb = new StringBuilder();
+        if (!dfs.isFileClosed(localFile)) {
+          sb.append(localFile + " has not been closed yet.");
+        }
+        if (!dfs.isFileClosed(remoteFile)) {
+          sb.append(remoteFile + " has not been closed yet.");
+        }
+        return sb.toString();
+      }
+    });
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
+      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
+      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    WAL walNoRep = FACTORY.getWAL(REGION_NO_REP);
+    assertThat(walNoRep, not(instanceOf(DualAsyncFSWAL.class)));
+    DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION);
+    assertEquals(2, FACTORY.getWALs().size());
+    testReadWrite(wal);
+    SynchronousReplicationWALProvider walProvider =
+      (SynchronousReplicationWALProvider) FACTORY.getWALProvider();
+    walProvider.peerRemoved(PEER_ID);
+    assertEquals(1, FACTORY.getWALs().size());
+  }
+}


[10/18] hbase git commit: HBASE-19956 Remove category as a consideration timing out tests; set all test to timeout at 10minutes regardless

Posted by zh...@apache.org.
HBASE-19956 Remove category as a consideration timing out tests; set all test to timeout at 10minutes regardless


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

Branch: refs/heads/HBASE-19064
Commit: 277ce3d8a3d68f87e82c98fdeb3a3cb3dee33036
Parents: f5dbdf0
Author: Michael Stack <st...@apache.org>
Authored: Wed Feb 7 19:27:49 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Feb 7 23:10:12 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HBaseClassTestRule.java  | 15 ++-------------
 hbase-server/pom.xml                                 |  2 +-
 2 files changed, 3 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/277ce3d8/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
index 734ce3f..bcde826 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
@@ -57,20 +57,9 @@ public final class HBaseClassTestRule implements TestRule {
 
   private static long getTimeoutInSeconds(Class<?> clazz) {
     Category[] categories = clazz.getAnnotationsByType(Category.class);
-    if (categories.length == 0) {
-      throw new IllegalArgumentException(clazz.getName() + " is not annotated with @Category");
-    }
     for (Class<?> c : categories[0].value()) {
-      if (c == SmallTests.class) {
-        // See SmallTests. Supposed to run 15 seconds.
-        // Lots of these timeout on Jenkins... a stall of ten or twenty seconds mess up what looks
-        // fine when run local.
-        return 60;
-      } else if (c == MediumTests.class) {
-        // See MediumTests. Supposed to run 50 seconds.
-        return 180;
-      } else if (c == LargeTests.class) {
-        // Let large tests have a ten minute timeout.
+      if (c == SmallTests.class || c == MediumTests.class || c == LargeTests.class) {
+        // All tests have a 10minute timeout.
         return TimeUnit.MINUTES.toSeconds(10);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/277ce3d8/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 20186a7..d2cf916 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -36,7 +36,7 @@
     <license.bundles.jquery>true</license.bundles.jquery>
   </properties>
   <build>
-    <!-- Makes sure the resources get added before they are processed
+    <!-- Make sure resources get added before they are processed
       by placing this first -->
     <resources>
       <!-- Add the build webabpps to the classpth -->


[13/18] hbase git commit: HBASE-19935 Only allow table replication for sync replication for now

Posted by zh...@apache.org.
HBASE-19935 Only allow table replication for sync replication for now


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

Branch: refs/heads/HBASE-19064
Commit: 785a77c115d5088f397ac96e6d758ad3fff4af8d
Parents: 00d450e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Feb 6 16:00:59 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 8 16:49:19 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfig.java      |  9 +++
 .../replication/ReplicationPeerManager.java     | 34 ++++++++-
 .../replication/TestReplicationAdmin.java       | 73 ++++++++++++++------
 .../wal/TestCombinedAsyncWriter.java            |  6 ++
 .../wal/TestSyncReplicationWALProvider.java     |  6 ++
 5 files changed, 102 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/785a77c1/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 4c10c46..69565a7 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,6 +25,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -217,6 +219,13 @@ public class ReplicationPeerConfig {
     return this.remoteWALDir;
   }
 
+  /**
+   * Use remote wal dir to decide whether a peer is sync replication peer
+   */
+  public boolean isSyncReplication() {
+    return !StringUtils.isBlank(this.remoteWALDir);
+  }
+
   public static ReplicationPeerConfigBuilder newBuilder() {
     return new ReplicationPeerConfigBuilderImpl();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/785a77c1/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 9336fbd..6bfd9c9 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
@@ -167,7 +167,7 @@ public class ReplicationPeerManager {
               " does not match new remote wal dir '" + peerConfig.getRemoteWALDir() + "'");
     }
 
-    if (oldPeerConfig.getRemoteWALDir() != null) {
+    if (oldPeerConfig.isSyncReplication()) {
       if (!ReplicationUtils.isKeyConfigEqual(oldPeerConfig, peerConfig)) {
         throw new DoNotRetryIOException(
             "Changing the replicated namespace/table config on a synchronous replication "
@@ -195,8 +195,8 @@ public class ReplicationPeerManager {
     }
     ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build();
     SyncReplicationState syncReplicationState =
-        StringUtils.isBlank(peerConfig.getRemoteWALDir()) ? SyncReplicationState.NONE
-            : SyncReplicationState.DOWNGRADE_ACTIVE;
+        copiedPeerConfig.isSyncReplication() ? SyncReplicationState.DOWNGRADE_ACTIVE
+            : SyncReplicationState.NONE;
     peerStorage.addPeer(peerId, copiedPeerConfig, enabled, syncReplicationState);
     peers.put(peerId,
       new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState));
@@ -316,9 +316,37 @@ public class ReplicationPeerManager {
         peerConfig.getTableCFsMap());
     }
 
+    if (peerConfig.isSyncReplication()) {
+      checkPeerConfigForSyncReplication(peerConfig);
+    }
+
     checkConfiguredWALEntryFilters(peerConfig);
   }
 
+  private void checkPeerConfigForSyncReplication(ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    // This is used to reduce the difficulty for implementing the sync replication state transition
+    // as we need to reopen all the related regions.
+    // TODO: Add namespace, replicat_all flag back
+    if (peerConfig.replicateAllUserTables()) {
+      throw new DoNotRetryIOException(
+          "Only support replicated table config for sync replication peer");
+    }
+    if (peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) {
+      throw new DoNotRetryIOException(
+          "Only support replicated table config for sync replication peer");
+    }
+    if (peerConfig.getTableCFsMap() == null || peerConfig.getTableCFsMap().isEmpty()) {
+      throw new DoNotRetryIOException("Need config replicated tables for sync replication peer");
+    }
+    for (List<String> cfs : peerConfig.getTableCFsMap().values()) {
+      if (cfs != null && !cfs.isEmpty()) {
+        throw new DoNotRetryIOException(
+            "Only support replicated table config for sync replication peer");
+      }
+    }
+  }
+
   /**
    * Set a namespace in the peer config means that all tables in this namespace will be replicated
    * to the peer cluster.

http://git-wip-us.apache.org/repos/asf/hbase/blob/785a77c1/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 a7710e7..d462dbd 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
@@ -26,6 +26,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -910,6 +911,8 @@ public class TestReplicationAdmin {
 
   @Test
   public void testPeerRemoteWALDir() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+
     String rootDir = "hdfs://srv1:9999/hbase";
     ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_ONE);
@@ -929,57 +932,74 @@ public class TestReplicationAdmin {
     builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_SECOND);
     builder.setRemoteWALDir(rootDir);
-    hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
 
-    rpc = hbaseAdmin.getReplicationPeerConfig(ID_SECOND);
-    assertEquals(rootDir, rpc.getRemoteWALDir());
+    try {
+      hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+      fail("Only support replicated table config for sync replication");
+    } catch (Exception e) {
+      // OK
+    }
 
+    builder.setReplicateAllUserTables(false);
     try {
-      builder.setRemoteWALDir("hdfs://srv2:8888/hbase");
-      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
-      fail("Change remote wal dir is not allowed");
+      Set<String> namespaces = new HashSet<String>();
+      namespaces.add("ns1");
+      builder.setNamespaces(namespaces);
+      hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+      fail("Only support replicated table config for sync replication");
     } catch (Exception e) {
       // OK
     }
 
+    builder.setNamespaces(null);
     try {
-      builder.setRemoteWALDir(null);
-      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
-      fail("Change remote wal dir is not allowed");
+      hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+      fail("Only support replicated table config for sync replication, and tables can't be empty");
     } catch (Exception e) {
       // OK
     }
 
+    Map<TableName, List<String>> tableCfs = new HashMap<>();
     try {
-      builder = ReplicationPeerConfig.newBuilder(rpc);
-      builder.setReplicateAllUserTables(false);
+      tableCfs.put(tableName, Arrays.asList("cf1"));
+      builder.setTableCFsMap(tableCfs);
+      hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+      fail("Only support replicated table config for sync replication");
+    } catch (Exception e) {
+      // OK
+    }
+
+    tableCfs = new HashMap<>();
+    tableCfs.put(tableName, new ArrayList<>());
+    builder.setTableCFsMap(tableCfs);
+    hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+    rpc = hbaseAdmin.getReplicationPeerConfig(ID_SECOND);
+    assertEquals(rootDir, rpc.getRemoteWALDir());
+
+    try {
+      builder.setRemoteWALDir("hdfs://srv2:8888/hbase");
       hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
-      fail(
-        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+      fail("Change remote wal dir is not allowed");
     } catch (Exception e) {
       // OK
     }
 
     try {
-      builder = ReplicationPeerConfig.newBuilder(rpc);
-      Set<String> namespaces = new HashSet<>();
-      namespaces.add("ns1");
-      builder.setExcludeNamespaces(namespaces);
+      builder.setRemoteWALDir(null);
       hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
-      fail(
-        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+      fail("Change remote wal dir is not allowed");
     } catch (Exception e) {
       // OK
     }
 
     try {
       builder = ReplicationPeerConfig.newBuilder(rpc);
-      Map<TableName, List<String>> tableCfs = new HashMap<>();
-      tableCfs.put(TableName.valueOf(name.getMethodName()), new ArrayList<>());
-      builder.setExcludeTableCFsMap(tableCfs);
+      tableCfs = new HashMap<>();
+      tableCfs.put(TableName.valueOf("ns1:" + name.getMethodName()), new ArrayList<>());
+      builder.setTableCFsMap(tableCfs);
       hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
       fail(
-        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+        "Change replicated table config on an existing synchronous peer is not allowed");
     } catch (Exception e) {
       // OK
     }
@@ -987,8 +1007,11 @@ public class TestReplicationAdmin {
 
   @Test
   public void testTransitSyncReplicationPeerState() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+
     ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_ONE);
+    builder.setReplicateAllUserTables(false);
     hbaseAdmin.addReplicationPeer(ID_ONE, builder.build());
     assertEquals(SyncReplicationState.NONE,
       hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE));
@@ -1005,6 +1028,10 @@ public class TestReplicationAdmin {
     builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_SECOND);
     builder.setRemoteWALDir(rootDir);
+    builder.setReplicateAllUserTables(false);
+    Map<TableName, List<String>> tableCfs = new HashMap<>();
+    tableCfs.put(tableName, new ArrayList<>());
+    builder.setTableCFsMap(tableCfs);
     hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
     assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
       hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));

http://git-wip-us.apache.org/repos/asf/hbase/blob/785a77c1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
index 36dbe0f..07aa6a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.wal.AsyncFSWALProvider.AsyncWriter;
 import org.apache.hadoop.hbase.wal.WALFactory;
 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;
@@ -51,6 +53,10 @@ import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
 @Category({ RegionServerTests.class, MediumTests.class })
 public class TestCombinedAsyncWriter {
 
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestCombinedAsyncWriter.class);
+
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
   private static EventLoopGroup EVENT_LOOP_GROUP;

http://git-wip-us.apache.org/repos/asf/hbase/blob/785a77c1/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
index 60a9e13..f09e51e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertThat;
 import java.io.IOException;
 import java.util.Optional;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
@@ -41,12 +42,17 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ RegionServerTests.class, MediumTests.class })
 public class TestSyncReplicationWALProvider {
 
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestSyncReplicationWALProvider.class);
+
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
   private static String PEER_ID = "1";


[02/18] hbase git commit: HBASE-19940 TestMetaShutdownHandler flakey; ADDENDUM: yet more debug

Posted by zh...@apache.org.
    HBASE-19940 TestMetaShutdownHandler flakey; ADDENDUM: yet more debug


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

Branch: refs/heads/HBASE-19064
Commit: 001f9cc5ea7d00fceb24cbcbf6399ecf272882e4
Parents: 590bee7
Author: Michael Stack <st...@apache.org>
Authored: Tue Feb 6 09:47:30 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Feb 6 09:47:30 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/util/JVMClusterUtil.java    | 8 ++------
 1 file changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/001f9cc5/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
index 0d410fc..ee7ecf3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
@@ -297,13 +297,9 @@ public class JVMClusterUtil {
         for (RegionServerThread t : regionservers) {
           if (t.isAlive()) {
             LOG.warn("RegionServerThreads taking too long to stop, interrupting; thread dump "  +
-              "if > three attempts: i=" + i);
+              "if > 3 attempts: i=" + i);
             if (i > 3) {
-              try {
-                Threads.threadDumpingIsAlive(t);
-              } catch (InterruptedException e) {
-                e.printStackTrace();
-              }
+              Threads.printThreadInfo(System.out, "Thread dump " + t.getName());
             }
             t.interrupt();
           }


[05/18] hbase git commit: HBASE-19942 Fix flaky TestSimpleRpcScheduler

Posted by zh...@apache.org.
HBASE-19942 Fix flaky TestSimpleRpcScheduler


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

Branch: refs/heads/HBASE-19064
Commit: a5b86dd77ad2866b0ab3c1774738ecb15e8719c7
Parents: 380083e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Feb 6 17:52:05 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Wed Feb 7 07:54:39 2018 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java | 6 +++---
 .../org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java    | 2 ++
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a5b86dd7/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
index 5572725..e49f576 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
@@ -145,9 +145,9 @@ public abstract class RpcExecutor {
       queueClass = LinkedBlockingQueue.class;
     }
 
-    LOG.info("RpcExecutor " + name + " using " + callQueueType
-        + " as call queue; numCallQueues=" + numCallQueues + "; maxQueueLength=" + maxQueueLength
-        + "; handlerCount=" + handlerCount);
+    LOG.info("RpcExecutor " + this.name + " using " + this.queueClass
+        + " as call queue; numCallQueues=" + this.numCallQueues + "; maxQueueLength="
+        + maxQueueLength + "; handlerCount=" + this.handlerCount);
   }
 
   protected int computeNumCallQueues(final int handlerCount, final float callQueuesHandlersFactor) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5b86dd7/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index 875223d..3fab7fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -428,6 +428,8 @@ public class TestSimpleRpcScheduler {
 
     schedConf.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 0);
     schedConf.setInt("hbase.ipc.server.max.callqueue.length", 5);
+    schedConf.set(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY,
+      RpcExecutor.CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
 
     PriorityFunction priority = mock(PriorityFunction.class);
     when(priority.getPriority(any(), any(), any())).thenReturn(HConstants.NORMAL_QOS);


[07/18] hbase git commit: HBASE-19929 Call RS.stop on a session expired RS may hang

Posted by zh...@apache.org.
HBASE-19929 Call RS.stop on a session expired RS may hang


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

Branch: refs/heads/HBASE-19064
Commit: dcbb331792c210a71e4cebe004c8477b34993770
Parents: d8b999e
Author: zhangduo <zh...@apache.org>
Authored: Wed Feb 7 15:52:04 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Feb 7 15:52:04 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/util/DrainBarrier.java  | 134 -------
 .../hadoop/hbase/util/TestDrainBarrier.java     | 127 -------
 .../hbase/regionserver/HRegionServer.java       |   2 +-
 .../hadoop/hbase/regionserver/LogRoller.java    |  40 +-
 .../hbase/regionserver/wal/AbstractFSWAL.java   |  45 +--
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  42 ++-
 .../TestShutdownWhileWALBroken.java             | 164 ++++++++
 .../hbase/regionserver/TestWALLockup.java       | 378 +++++--------------
 8 files changed, 324 insertions(+), 608 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dcbb3317/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DrainBarrier.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DrainBarrier.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DrainBarrier.java
deleted file mode 100644
index b64ebdf..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DrainBarrier.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.util;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * A simple barrier that can be used by classes that need to wait for some operations to
- * finish before stopping/closing/etc. forever.
- */
-@InterfaceAudience.Private
-public class DrainBarrier {
-  /**
-   * Contains the number of outstanding operations, as well as flags.
-   * Initially, the number of operations is 1. Each beginOp increments, and endOp decrements it.
-   * beginOp does not proceed when it sees the draining flag. When stop is called, it atomically
-   * decrements the number of operations (the initial 1) and sets the draining flag. If stop did
-   * the decrement to zero, that means there are no more operations outstanding, so stop is done.
-   * Otherwise, stop blocks, and the endOp that decrements the count to 0 unblocks it.
-   */
-  private final AtomicLong valueAndFlags = new AtomicLong(inc(0));
-  private final static long DRAINING_FLAG = 0x1;
-  private final static int FLAG_BIT_COUNT = 1;
-
-  /**
-   * Tries to start an operation.
-   * @return false iff the stop is in progress, and the operation cannot be started.
-   */
-  public boolean beginOp() {
-    long oldValAndFlags;
-    do {
-      oldValAndFlags = valueAndFlags.get();
-      if (isDraining(oldValAndFlags)) return false;
-    } while (!valueAndFlags.compareAndSet(oldValAndFlags, inc(oldValAndFlags)));
-    return true;
-  }
-
-  /**
-   * Ends the operation. Unblocks the blocked caller of stop, if necessary.
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
-      justification="First, we do change the state before notify, 2nd, it doesn't even matter")
-  public void endOp() {
-    long oldValAndFlags;
-    do {
-      oldValAndFlags = valueAndFlags.get();
-      long unacceptableCount = isDraining(oldValAndFlags) ? 0 : 1;
-      if (getValue(oldValAndFlags) == unacceptableCount) {
-        throw new AssertionError("endOp called without corresponding beginOp call ("
-          + "the current count is " + unacceptableCount + ")");
-      }
-    } while (!valueAndFlags.compareAndSet(oldValAndFlags, dec(oldValAndFlags)));
-    if (getValue(oldValAndFlags) == 1) {
-      synchronized (this) { this.notifyAll(); }
-    }
-  }
-
-  /**
-   * Blocks new operations from starting, waits for the current ones to drain.
-   * If someone already called it, returns immediately, which is currently unavoidable as
-   * most of the users stop and close things right and left, and hope for the best.
-   * stopAndWaitForOpsOnce asserts instead.
-   * @throws InterruptedException the wait for operations has been interrupted.
-   */
-  public void stopAndDrainOps() throws InterruptedException {
-    stopAndDrainOps(true);
-  }
-
-  /**
-   * Blocks new operations from starting, waits for the current ones to drain.
-   * Can only be called once.
-   * @throws InterruptedException the wait for operations has been interrupted.
-   */
-  public void stopAndDrainOpsOnce() throws InterruptedException {
-    stopAndDrainOps(false);
-  }
-
-  /**
-   * @param ignoreRepeatedCalls If this is true and somebody already called stop, this method
-   *                            will return immediately if true; if this is false and somebody
-   *                            already called stop, it will assert.
-   */
-  // Justification for warnings - wait is not unconditional, and contrary to what WA_NOT_IN_LOOP
-  // description says we are not waiting on multiple conditions.
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings({"UW_UNCOND_WAIT", "WA_NOT_IN_LOOP"})
-  private void stopAndDrainOps(boolean ignoreRepeatedCalls) throws InterruptedException {
-    long oldValAndFlags;
-    do {
-      oldValAndFlags = valueAndFlags.get();
-      if (isDraining(oldValAndFlags)) {
-        if (ignoreRepeatedCalls) return;
-        throw new AssertionError("stopAndWaitForOpsOnce called more than once");
-      }
-    } while (!valueAndFlags.compareAndSet(oldValAndFlags, dec(oldValAndFlags) | DRAINING_FLAG));
-    if (getValue(oldValAndFlags) == 1) return; // There were no operations outstanding.
-    synchronized (this) { this.wait(); }
-  }
-
-  // Helper methods.
-  private static final boolean isDraining(long valueAndFlags) {
-    return (valueAndFlags & DRAINING_FLAG) == DRAINING_FLAG;
-  }
-
-  private static final long getValue(long valueAndFlags) {
-    return valueAndFlags >> FLAG_BIT_COUNT;
-  }
-
-  private static final long inc(long valueAndFlags) {
-    return valueAndFlags + (1 << FLAG_BIT_COUNT); // Not checking for overflow.
-  }
-
-  private static final long dec(long valueAndFlags) {
-    return valueAndFlags - (1 << FLAG_BIT_COUNT); // Negative overflow checked outside.
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dcbb3317/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java
deleted file mode 100644
index 5c3d053..0000000
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.util;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestDrainBarrier {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestDrainBarrier.class);
-
-  @Test
-  public void testBeginEndStopWork() throws Exception {
-    DrainBarrier barrier = new DrainBarrier();
-    assertTrue(barrier.beginOp());
-    assertTrue(barrier.beginOp());
-    barrier.endOp();
-    barrier.endOp();
-    barrier.stopAndDrainOps();
-    assertFalse(barrier.beginOp());
-  }
-
-  @Test
-  public void testUnmatchedEndAssert() throws Exception {
-    DrainBarrier barrier = new DrainBarrier();
-    try {
-      barrier.endOp();
-      throw new Error("Should have asserted");
-    } catch (AssertionError e) {
-    }
-
-    barrier.beginOp();
-    barrier.beginOp();
-    barrier.endOp();
-    barrier.endOp();
-    try {
-      barrier.endOp();
-      throw new Error("Should have asserted");
-    } catch (AssertionError e) {
-    }
-  }
-
-  @Test
-  public void testStopWithoutOpsDoesntBlock() throws Exception {
-    DrainBarrier barrier = new DrainBarrier();
-    barrier.stopAndDrainOpsOnce();
-
-    barrier = new DrainBarrier();
-    barrier.beginOp();
-    barrier.endOp();
-    barrier.stopAndDrainOpsOnce();
-  }
-
-  @Test
-  /** This test tests blocking and can have false positives in very bad timing cases. */
-  public void testStopIsBlockedByOps() throws Exception {
-    final DrainBarrier barrier = new DrainBarrier();
-    barrier.beginOp();
-    barrier.beginOp();
-    barrier.beginOp();
-    barrier.endOp();
-
-    Thread stoppingThread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          barrier.stopAndDrainOpsOnce();
-        } catch (InterruptedException e) {
-          fail("Should not have happened");
-        }
-      }
-    });
-    stoppingThread.start();
-
-    // First "end" should not unblock the thread, but the second should.
-    barrier.endOp();
-    stoppingThread.join(1000);
-    assertTrue(stoppingThread.isAlive());
-    barrier.endOp();
-    stoppingThread.join(30000); // When not broken, will be a very fast wait; set safe value.
-    assertFalse(stoppingThread.isAlive());
-  }
-
-  @Test
-  public void testMultipleStopOnceAssert() throws Exception {
-    DrainBarrier barrier = new DrainBarrier();
-    barrier.stopAndDrainOpsOnce();
-    try {
-      barrier.stopAndDrainOpsOnce();
-      throw new Error("Should have asserted");
-    } catch (AssertionError e) {
-    }
-  }
-
-  @Test
-  public void testMultipleSloppyStopsHaveNoEffect() throws Exception {
-    DrainBarrier barrier = new DrainBarrier();
-    barrier.stopAndDrainOps();
-    barrier.stopAndDrainOps();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dcbb3317/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 3a93c76..0d59b12 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
@@ -1181,7 +1181,7 @@ public class HRegionServer extends HasThread implements
 
   @VisibleForTesting
   protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
-  throws IOException {
+      throws IOException {
     RegionServerStatusService.BlockingInterface rss = rssStub;
     if (rss == null) {
       // the current server could be stopping.

http://git-wip-us.apache.org/repos/asf/hbase/blob/dcbb3317/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
index 7a247cf..55c5219 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
@@ -132,6 +132,23 @@ public class LogRoller extends HasThread implements Closeable {
     }
   }
 
+  private void abort(String reason, Throwable cause) {
+    // close all WALs before calling abort on RS.
+    // This is because AsyncFSWAL replies on us for rolling a new writer to make progress, and if we
+    // failed, AsyncFSWAL may be stuck, so we need to close it to let the upper layer know that it
+    // is already broken.
+    for (WAL wal : walNeedsRoll.keySet()) {
+      // shutdown rather than close here since we are going to abort the RS and the wals need to be
+      // split when recovery
+      try {
+        wal.shutdown();
+      } catch (IOException e) {
+        LOG.warn("Failed to shutdown wal", e);
+      }
+    }
+    server.abort(reason, cause);
+  }
+
   @Override
   public void run() {
     while (running) {
@@ -153,10 +170,8 @@ public class LogRoller extends HasThread implements Closeable {
           continue;
         }
         // Time for periodic roll
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Wal roll period " + this.rollperiod + "ms elapsed");
-        }
-      } else if (LOG.isDebugEnabled()) {
+        LOG.debug("Wal roll period {} ms elapsed", this.rollperiod);
+      } else {
         LOG.debug("WAL roll requested");
       }
       rollLock.lock(); // FindBugs UL_UNRELEASED_LOCK_EXCEPTION_PATH
@@ -170,20 +185,22 @@ public class LogRoller extends HasThread implements Closeable {
               entry.getValue().booleanValue());
           walNeedsRoll.put(wal, Boolean.FALSE);
           if (regionsToFlush != null) {
-            for (byte [] r: regionsToFlush) scheduleFlush(r);
+            for (byte[] r : regionsToFlush) {
+              scheduleFlush(r);
+            }
           }
         }
       } catch (FailedLogCloseException e) {
-        server.abort("Failed log close in log roller", e);
+        abort("Failed log close in log roller", e);
       } catch (java.net.ConnectException e) {
-        server.abort("Failed log close in log roller", e);
+        abort("Failed log close in log roller", e);
       } catch (IOException ex) {
         // Abort if we get here.  We probably won't recover an IOE. HBASE-1132
-        server.abort("IOE in log roller",
+        abort("IOE in log roller",
           ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex);
       } catch (Exception ex) {
         LOG.error("Log rolling failed", ex);
-        server.abort("Log rolling failed", ex);
+        abort("Log rolling failed", ex);
       } finally {
         try {
           rollLog.set(false);
@@ -211,9 +228,8 @@ public class LogRoller extends HasThread implements Closeable {
       }
     }
     if (!scheduled) {
-      LOG.warn("Failed to schedule flush of " +
-        Bytes.toString(encodedRegionName) + ", region=" + r + ", requester=" +
-        requester);
+      LOG.warn("Failed to schedule flush of {}, region={}, requester={}",
+        Bytes.toString(encodedRegionName), r, requester);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/dcbb3317/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 54a5cd3..14fbe10 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -17,12 +17,11 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
 import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
 
 import com.lmax.disruptor.RingBuffer;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -46,7 +45,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantLock;
-
 import org.apache.commons.lang3.mutable.MutableLong;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -66,7 +64,6 @@ import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.DrainBarrier;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -84,6 +81,7 @@ import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -173,9 +171,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
    */
   protected final SequenceIdAccounting sequenceIdAccounting = new SequenceIdAccounting();
 
-  /** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */
-  protected final DrainBarrier closeBarrier = new DrainBarrier();
-
   protected final long slowSyncNs;
 
   private final long walSyncTimeoutNs;
@@ -452,32 +447,22 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
 
   @Override
   public Long startCacheFlush(byte[] encodedRegionName, Set<byte[]> families) {
-    if (!closeBarrier.beginOp()) {
-      LOG.info("Flush not started for " + Bytes.toString(encodedRegionName) + "; server closing.");
-      return null;
-    }
     return this.sequenceIdAccounting.startCacheFlush(encodedRegionName, families);
   }
 
   @Override
   public Long startCacheFlush(byte[] encodedRegionName, Map<byte[], Long> familyToSeq) {
-    if (!closeBarrier.beginOp()) {
-      LOG.info("Flush not started for " + Bytes.toString(encodedRegionName) + "; server closing.");
-      return null;
-    }
     return this.sequenceIdAccounting.startCacheFlush(encodedRegionName, familyToSeq);
   }
 
   @Override
   public void completeCacheFlush(byte[] encodedRegionName) {
     this.sequenceIdAccounting.completeCacheFlush(encodedRegionName);
-    closeBarrier.endOp();
   }
 
   @Override
   public void abortCacheFlush(byte[] encodedRegionName) {
     this.sequenceIdAccounting.abortCacheFlush(encodedRegionName);
-    closeBarrier.endOp();
   }
 
   @Override
@@ -715,7 +700,11 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     // Now we have published the ringbuffer, halt the current thread until we get an answer back.
     try {
       if (syncFuture != null) {
-        syncFuture.get(walSyncTimeoutNs);
+        if (closed) {
+          throw new IOException("WAL has been closed");
+        } else {
+          syncFuture.get(walSyncTimeoutNs);
+        }
       }
     } catch (TimeoutIOException tioe) {
       // SyncFuture reuse by thread, if TimeoutIOException happens, ringbuffer
@@ -755,10 +744,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
         LOG.debug("WAL closed. Skipping rolling of writer");
         return regionsToFlush;
       }
-      if (!closeBarrier.beginOp()) {
-        LOG.debug("WAL closing. Skipping rolling of writer");
-        return regionsToFlush;
-      }
       try (TraceScope scope = TraceUtil.createTrace("FSHLog.rollWriter")) {
         Path oldPath = getOldPath();
         Path newPath = getNewPath();
@@ -783,8 +768,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
         throw new IOException(
             "Underlying FileSystem can't meet stream requirements. See RS log " + "for details.",
             exception);
-      } finally {
-        closeBarrier.endOp();
       }
       return regionsToFlush;
     } finally {
@@ -818,20 +801,18 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
       return;
     }
     closed = true;
-    try {
-      // Prevent all further flushing and rolling.
-      closeBarrier.stopAndDrainOps();
-    } catch (InterruptedException e) {
-      LOG.error("Exception while waiting for cache flushes and log rolls", e);
-      Thread.currentThread().interrupt();
-    }
     // Tell our listeners that the log is closing
     if (!this.listeners.isEmpty()) {
       for (WALActionsListener i : this.listeners) {
         i.logCloseRequested();
       }
     }
-    doShutdown();
+    rollWriterLock.lock();
+    try {
+      doShutdown();
+    } finally {
+      rollWriterLock.unlock();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/dcbb3317/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index faf3b77..19d89df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -608,19 +608,8 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
 
   @Override
   protected AsyncWriter createWriterInstance(Path path) throws IOException {
-    try {
-      return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup,
-        channelClass);
-    } catch (IOException e) {
-      // this usually means master already think we are dead so let's fail all the pending
-      // syncs. The shutdown process of RS will wait for all regions to be closed before calling
-      // WAL.close so if we do not wake up the thread blocked by sync here it will cause dead
-      // lock.
-      if (e.getMessage().contains("Parent directory doesn't exist:")) {
-        syncFutures.forEach(f -> f.done(f.getTxid(), e));
-      }
-      throw e;
-    }
+    return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup,
+      channelClass);
   }
 
   private void waitForSafePoint() {
@@ -675,17 +664,34 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     closeExecutor.shutdown();
     try {
       if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) {
-        LOG.error("We have waited " + waitOnShutdownInSeconds + " seconds but"
-          + " the close of async writer doesn't complete."
-          + "Please check the status of underlying filesystem"
-          + " or increase the wait time by the config \""
-          + ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS + "\"");
+        LOG.error("We have waited " + waitOnShutdownInSeconds + " seconds but" +
+          " the close of async writer doesn't complete." +
+          "Please check the status of underlying filesystem" +
+          " or increase the wait time by the config \"" + ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS +
+          "\"");
       }
     } catch (InterruptedException e) {
       LOG.error("The wait for close of async writer is interrupted");
       Thread.currentThread().interrupt();
     }
     IOException error = new IOException("WAL has been closed");
+    long nextCursor = waitingConsumePayloadsGatingSequence.get() + 1;
+    // drain all the pending sync requests
+    for (long cursorBound = waitingConsumePayloads.getCursor(); nextCursor <= cursorBound;
+      nextCursor++) {
+      if (!waitingConsumePayloads.isPublished(nextCursor)) {
+        break;
+      }
+      RingBufferTruck truck = waitingConsumePayloads.get(nextCursor);
+      switch (truck.type()) {
+        case SYNC:
+          syncFutures.add(truck.unloadSync());
+          break;
+        default:
+          break;
+      }
+    }
+    // and fail them
     syncFutures.forEach(f -> f.done(f.getTxid(), error));
     if (!(consumeExecutor instanceof EventLoop)) {
       consumeExecutor.shutdown();

http://git-wip-us.apache.org/repos/asf/hbase/blob/dcbb3317/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java
new file mode 100644
index 0000000..6c9b5e3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.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.regionserver;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+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.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.YouAreDeadException;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.zookeeper.KeeperException.SessionExpiredException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * See HBASE-19929 for more details.
+ */
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestShutdownWhileWALBroken {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestShutdownWhileWALBroken.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestShutdownWhileWALBroken.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName TABLE_NAME = TableName.valueOf("TestShutdownWhileWALBroken");
+
+  private static byte[] CF = Bytes.toBytes("CF");
+
+  @Parameter
+  public String walType;
+
+  @Parameters(name = "{index}: WAL={0}")
+  public static List<Object[]> params() {
+    return Arrays.asList(new Object[] { "asyncfs" }, new Object[] { "filesystem" });
+  }
+
+  public static final class MyRegionServer extends HRegionServer {
+
+    private final CountDownLatch latch = new CountDownLatch(1);
+
+    public MyRegionServer(Configuration conf) throws IOException {
+      super(conf);
+    }
+
+    @Override
+    protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
+        throws IOException {
+      try {
+        super.tryRegionServerReport(reportStartTime, reportEndTime);
+      } catch (YouAreDeadException e) {
+        LOG.info("Caught YouAreDeadException, ignore", e);
+      }
+    }
+
+    @Override
+    public void abort(String reason, Throwable cause) {
+      if (cause instanceof SessionExpiredException) {
+        // called from ZKWatcher, let's wait a bit to make sure that we call stop before calling
+        // abort.
+        try {
+          latch.await();
+        } catch (InterruptedException e) {
+        }
+      } else {
+        // abort from other classes, usually LogRoller, now we can make progress on abort.
+        latch.countDown();
+      }
+      super.abort(reason, cause);
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    UTIL.getConfiguration().setClass(HConstants.REGION_SERVER_IMPL, MyRegionServer.class,
+      HRegionServer.class);
+    UTIL.getConfiguration().set(WALFactory.WAL_PROVIDER, walType);
+    UTIL.getConfiguration().set(WALFactory.META_WAL_PROVIDER, walType);
+    UTIL.startMiniCluster(2);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    UTIL.createMultiRegionTable(TABLE_NAME, CF);
+    try (Table table = UTIL.getConnection().getTable(TABLE_NAME)) {
+      UTIL.loadTable(table, CF);
+    }
+    int numRegions = UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).size();
+    RegionServerThread rst0 = UTIL.getMiniHBaseCluster().getRegionServerThreads().get(0);
+    RegionServerThread rst1 = UTIL.getMiniHBaseCluster().getRegionServerThreads().get(1);
+    HRegionServer liveRS;
+    RegionServerThread toKillRSThread;
+    if (rst1.getRegionServer().getRegions(TableName.META_TABLE_NAME).isEmpty()) {
+      liveRS = rst0.getRegionServer();
+      toKillRSThread = rst1;
+    } else {
+      liveRS = rst1.getRegionServer();
+      toKillRSThread = rst0;
+    }
+    assertTrue(liveRS.getRegions(TABLE_NAME).size() < numRegions);
+    UTIL.expireSession(toKillRSThread.getRegionServer().getZooKeeper(), false);
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return liveRS.getRegions(TABLE_NAME).size() == numRegions;
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Failover is not finished yet";
+      }
+    });
+    toKillRSThread.getRegionServer().stop("Stop for test");
+    // make sure that we can successfully quit
+    toKillRSThread.join();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dcbb3317/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 1e59248b..ca65914 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -33,7 +33,6 @@ 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.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -43,9 +42,9 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.regionserver.wal.DamagedWALException;
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
-import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
 import org.apache.hadoop.hbase.util.Threads;
@@ -67,11 +66,12 @@ import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
 /**
- * Testing for lock up of WAL subsystem.
- * Copied from TestHRegion.
+ * Testing for lock up of FSHLog.
  */
-@Category({MediumTests.class})
+@Category({ RegionServerTests.class, MediumTests.class })
 public class TestWALLockup {
 
   @ClassRule
@@ -79,14 +79,15 @@ public class TestWALLockup {
       HBaseClassTestRule.forClass(TestWALLockup.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestWALLockup.class);
-  @Rule public TestName name = new TestName();
+
+  @Rule
+  public TestName name = new TestName();
 
   private static final String COLUMN_FAMILY = "MyCF";
   private static final byte [] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY);
 
   HRegion region = null;
-  // Do not run unit tests in parallel (? Why not?  It don't work?  Why not?  St.Ack)
-  private static HBaseTestingUtility TEST_UTIL;
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Configuration CONF ;
   private String dir;
 
@@ -95,7 +96,6 @@ public class TestWALLockup {
 
   @Before
   public void setup() throws IOException {
-    TEST_UTIL = HBaseTestingUtility.createLocalHTU();
     CONF = TEST_UTIL.getConfiguration();
     // Disable block cache.
     CONF.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
@@ -110,100 +110,99 @@ public class TestWALLockup {
     TEST_UTIL.cleanupTestDir();
   }
 
-  String getName() {
+  private String getName() {
     return name.getMethodName();
   }
 
-  /**
-   * Reproduce locking up that happens when we get an inopportune sync during setup for
-   * zigzaglatch wait. See HBASE-14317. If below is broken, we will see this test timeout because
-   * it is locked up.
-   * <p>First I need to set up some mocks for Server and RegionServerServices. I also need to
-   * set up a dodgy WAL that will throw an exception when we go to append to it.
-   */
-  @Test (timeout=20000)
-  public void testLockupWhenSyncInMiddleOfZigZagSetup() throws IOException {
-    // A WAL that we can have throw exceptions when a flag is set.
-    class DodgyFSLog extends FSHLog {
-      // Set this when want the WAL to start throwing exceptions.
-      volatile boolean throwException = false;
+  // A WAL that we can have throw exceptions when a flag is set.
+  private static final class DodgyFSLog extends FSHLog {
+    // Set this when want the WAL to start throwing exceptions.
+    volatile boolean throwException = false;
 
-      // Latch to hold up processing until after another operation has had time to run.
-      CountDownLatch latch = new CountDownLatch(1);
+    // Latch to hold up processing until after another operation has had time to run.
+    CountDownLatch latch = new CountDownLatch(1);
 
-      public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
-      throws IOException {
-        super(fs, root, logDir, conf);
-      }
+    public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
+        throws IOException {
+      super(fs, root, logDir, conf);
+    }
 
-      @Override
-      protected void afterCreatingZigZagLatch() {
-        // If throwException set, then append will throw an exception causing the WAL to be
-        // rolled. We'll come in here. Hold up processing until a sync can get in before
-        // the zigzag has time to complete its setup and get its own sync in. This is what causes
-        // the lock up we've seen in production.
-        if (throwException) {
-          try {
-            LOG.info("LATCHED");
-            // So, timing can have it that the test can run and the bad flush below happens
-            // before we get here. In this case, we'll be stuck waiting on this latch but there
-            // is nothing in the WAL pipeline to get us to the below beforeWaitOnSafePoint...
-            // because all WALs have rolled. In this case, just give up on test.
-            if (!this.latch.await(5, TimeUnit.SECONDS)) {
-              LOG.warn("GIVE UP! Failed waiting on latch...Test is ABORTED!");
-            }
-          } catch (InterruptedException e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
+    @Override
+    protected void afterCreatingZigZagLatch() {
+      // If throwException set, then append will throw an exception causing the WAL to be
+      // rolled. We'll come in here. Hold up processing until a sync can get in before
+      // the zigzag has time to complete its setup and get its own sync in. This is what causes
+      // the lock up we've seen in production.
+      if (throwException) {
+        try {
+          LOG.info("LATCHED");
+          // So, timing can have it that the test can run and the bad flush below happens
+          // before we get here. In this case, we'll be stuck waiting on this latch but there
+          // is nothing in the WAL pipeline to get us to the below beforeWaitOnSafePoint...
+          // because all WALs have rolled. In this case, just give up on test.
+          if (!this.latch.await(5, TimeUnit.SECONDS)) {
+            LOG.warn("GIVE UP! Failed waiting on latch...Test is ABORTED!");
           }
+        } catch (InterruptedException e) {
         }
       }
+    }
 
-      @Override
-      protected void beforeWaitOnSafePoint() {
-        if (throwException) {
-          LOG.info("COUNTDOWN");
-          // Don't countdown latch until someone waiting on it otherwise, the above
-          // afterCreatingZigZagLatch will get to the latch and no one will ever free it and we'll
-          // be stuck; test won't go down
-          while (this.latch.getCount() <= 0) Threads.sleep(1);
-          this.latch.countDown();
-        }
+    @Override
+    protected void beforeWaitOnSafePoint() {
+      if (throwException) {
+        LOG.info("COUNTDOWN");
+        // Don't countdown latch until someone waiting on it otherwise, the above
+        // afterCreatingZigZagLatch will get to the latch and no one will ever free it and we'll
+        // be stuck; test won't go down
+        while (this.latch.getCount() <= 0)
+          Threads.sleep(1);
+        this.latch.countDown();
       }
+    }
 
-      @Override
-      protected Writer createWriterInstance(Path path) throws IOException {
-        final Writer w = super.createWriterInstance(path);
-        return new Writer() {
-          @Override
-          public void close() throws IOException {
-            w.close();
-          }
+    @Override
+    protected Writer createWriterInstance(Path path) throws IOException {
+      final Writer w = super.createWriterInstance(path);
+      return new Writer() {
+        @Override
+        public void close() throws IOException {
+          w.close();
+        }
 
-          @Override
-          public void sync() throws IOException {
-            if (throwException) {
-              throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
-            }
-            w.sync();
+        @Override
+        public void sync() throws IOException {
+          if (throwException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
           }
+          w.sync();
+        }
 
-          @Override
-          public void append(Entry entry) throws IOException {
-            if (throwException) {
-              throw new IOException("FAKE! Failed to replace a bad datanode...APPEND");
-            }
-            w.append(entry);
+        @Override
+        public void append(Entry entry) throws IOException {
+          if (throwException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...APPEND");
           }
+          w.append(entry);
+        }
 
-          @Override
-          public long getLength() {
-            return w.getLength();
-          }
-        };
-      }
+        @Override
+        public long getLength() {
+          return w.getLength();
+        }
+      };
     }
+  }
 
+  /**
+   * Reproduce locking up that happens when we get an inopportune sync during setup for
+   * zigzaglatch wait. See HBASE-14317. If below is broken, we will see this test timeout because
+   * it is locked up.
+   * <p>First I need to set up some mocks for Server and RegionServerServices. I also need to
+   * set up a dodgy WAL that will throw an exception when we go to append to it.
+   */
+  @Test
+  public void testLockupWhenSyncInMiddleOfZigZagSetup() throws IOException {
     // Mocked up server and regionserver services. Needed below.
     Server server = Mockito.mock(Server.class);
     Mockito.when(server.getConfiguration()).thenReturn(CONF);
@@ -222,7 +221,6 @@ public class TestWALLockup {
     // There is no 'stop' once a logRoller is running.. it just dies.
     logRoller.start();
     // Now get a region and start adding in edits.
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
     final HRegion region = initHRegion(tableName, null, null, dodgyWAL);
     byte [] bytes = Bytes.toBytes(getName());
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(
@@ -236,7 +234,7 @@ public class TestWALLockup {
       Put put = new Put(bytes);
       put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
       WALKeyImpl key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
-          htd.getTableName(), System.currentTimeMillis(), mvcc, scopes);
+        TableName.META_TABLE_NAME, System.currentTimeMillis(), mvcc, scopes);
       WALEdit edit = new WALEdit();
       CellScanner CellScanner = put.cellScanner();
       assertTrue(CellScanner.advance());
@@ -281,7 +279,9 @@ public class TestWALLockup {
       t.setDaemon(true);
       t.start();
       // Wait until
-      while (dodgyWAL.latch.getCount() > 0) Threads.sleep(1);
+      while (dodgyWAL.latch.getCount() > 0) {
+        Threads.sleep(1);
+      }
       // Now assert I got a new WAL file put in place even though loads of errors above.
       assertTrue(originalWAL != dodgyWAL.getCurrentFileName());
       // Can I append to it?
@@ -294,203 +294,13 @@ public class TestWALLockup {
     } finally {
       // To stop logRoller, its server has to say it is stopped.
       Mockito.when(server.isStopped()).thenReturn(true);
-      if (logRoller != null) logRoller.close();
-      try {
-        if (region != null) region.close();
-        if (dodgyWAL != null) dodgyWAL.close();
-      } catch (Exception e) {
-        LOG.info("On way out", e);
-      }
-    }
-  }
-
-  /**
-   * Reproduce locking up that happens when there's no further syncs after
-   * append fails, and causing an isolated sync then infinite wait. See
-   * HBASE-16960. If below is broken, we will see this test timeout because it
-   * is locked up.
-   * <p/>
-   * Steps for reproduce:<br/>
-   * 1. Trigger server abort through dodgyWAL1<br/>
-   * 2. Add a {@link DummyWALActionsListener} to dodgyWAL2 to cause ringbuffer
-   * event handler thread sleep for a while thus keeping {@code endOfBatch}
-   * false<br/>
-   * 3. Publish a sync then an append which will throw exception, check whether
-   * the sync could return
-   */
-  @Test(timeout = 20000)
-  public void testLockup16960() throws IOException {
-    // A WAL that we can have throw exceptions when a flag is set.
-    class DodgyFSLog extends FSHLog {
-      // Set this when want the WAL to start throwing exceptions.
-      volatile boolean throwException = false;
-
-      public DodgyFSLog(FileSystem fs, Path root, String logDir,
-          Configuration conf) throws IOException {
-        super(fs, root, logDir, conf);
-      }
-
-      @Override
-      protected Writer createWriterInstance(Path path) throws IOException {
-        final Writer w = super.createWriterInstance(path);
-        return new Writer() {
-          @Override
-          public void close() throws IOException {
-            w.close();
-          }
-
-          @Override
-          public void sync() throws IOException {
-            if (throwException) {
-              throw new IOException(
-                  "FAKE! Failed to replace a bad datanode...SYNC");
-            }
-            w.sync();
-          }
-
-          @Override
-          public void append(Entry entry) throws IOException {
-            if (throwException) {
-              throw new IOException(
-                  "FAKE! Failed to replace a bad datanode...APPEND");
-            }
-            w.append(entry);
-          }
-
-          @Override
-          public long getLength() {
-            return w.getLength();
-          }
-        };
-      }
-
-      @Override
-      protected long doReplaceWriter(Path oldPath, Path newPath,
-          Writer nextWriter) throws IOException {
-        if (throwException) {
-          throw new FailedLogCloseException("oldPath=" + oldPath + ", newPath="
-              + newPath);
-        }
-        long oldFileLen = 0L;
-        oldFileLen = super.doReplaceWriter(oldPath, newPath, nextWriter);
-        return oldFileLen;
-      }
-    }
-
-    // Mocked up server and regionserver services. Needed below.
-    Server server = new DummyServer(CONF, ServerName.valueOf(
-        "hostname1.example.org", 1234, 1L).toString());
-    RegionServerServices services = Mockito.mock(RegionServerServices.class);
-
-    CONF.setLong("hbase.regionserver.hlog.sync.timeout", 10000);
-
-    // OK. Now I have my mocked up Server & RegionServerServices and dodgy WAL,
-    // go ahead with test.
-    FileSystem fs = FileSystem.get(CONF);
-    Path rootDir = new Path(dir + getName());
-    DodgyFSLog dodgyWAL1 = new DodgyFSLog(fs, rootDir, getName(), CONF);
-
-    Path rootDir2 = new Path(dir + getName() + "2");
-    final DodgyFSLog dodgyWAL2 = new DodgyFSLog(fs, rootDir2, getName() + "2",
-        CONF);
-    // Add a listener to force ringbuffer event handler sleep for a while
-    dodgyWAL2.registerWALActionsListener(new DummyWALActionsListener());
-
-    // I need a log roller running.
-    LogRoller logRoller = new LogRoller(server, services);
-    logRoller.addWAL(dodgyWAL1);
-    logRoller.addWAL(dodgyWAL2);
-    // There is no 'stop' once a logRoller is running.. it just dies.
-    logRoller.start();
-    // Now get a region and start adding in edits.
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
-    final HRegion region = initHRegion(tableName, null, null, dodgyWAL1);
-    byte[] bytes = Bytes.toBytes(getName());
-    NavigableMap<byte[], Integer> scopes = new TreeMap<>(
-        Bytes.BYTES_COMPARATOR);
-    scopes.put(COLUMN_FAMILY_BYTES, 0);
-    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
-    try {
-      Put put = new Put(bytes);
-      put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
-      WALKeyImpl key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
-          htd.getTableName(), System.currentTimeMillis(), mvcc, scopes);
-      WALEdit edit = new WALEdit();
-      CellScanner CellScanner = put.cellScanner();
-      assertTrue(CellScanner.advance());
-      edit.add(CellScanner.current());
-
-      LOG.info("SET throwing of exception on append");
-      dodgyWAL1.throwException = true;
-      // This append provokes a WAL roll request
-      dodgyWAL1.append(region.getRegionInfo(), key, edit, true);
-      boolean exception = false;
-      try {
-        dodgyWAL1.sync();
-      } catch (Exception e) {
-        exception = true;
-      }
-      assertTrue("Did not get sync exception", exception);
-
-      // LogRoller call dodgyWAL1.rollWriter get FailedLogCloseException and
-      // cause server abort.
-      try {
-        // wait LogRoller exit.
-        Thread.sleep(50);
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      }
-
-      final CountDownLatch latch = new CountDownLatch(1);
-
-      // make RingBufferEventHandler sleep 1s, so the following sync
-      // endOfBatch=false
-      key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
-          TableName.valueOf("sleep"), System.currentTimeMillis(), mvcc, scopes);
-      dodgyWAL2.append(region.getRegionInfo(), key, edit, true);
-
-      Thread t = new Thread("Sync") {
-        @Override
-        public void run() {
-          try {
-            dodgyWAL2.sync();
-          } catch (IOException e) {
-            LOG.info("In sync", e);
-          }
-          latch.countDown();
-          LOG.info("Sync exiting");
-        }
-      };
-      t.setDaemon(true);
-      t.start();
-      try {
-        // make sure sync have published.
-        Thread.sleep(100);
-      } catch (InterruptedException e1) {
-        e1.printStackTrace();
-      }
-      // make append throw DamagedWALException
-      key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
-          TableName.valueOf("DamagedWALException"), System.currentTimeMillis(), mvcc, scopes);
-      dodgyWAL2.append(region.getRegionInfo(), key, edit, true);
-
-      while (latch.getCount() > 0) {
-        Threads.sleep(100);
-      }
-      assertTrue(server.isAborted());
-    } finally {
-      if (logRoller != null) {
-        logRoller.close();
-      }
+      Closeables.close(logRoller, true);
       try {
         if (region != null) {
           region.close();
         }
-        if (dodgyWAL1 != null) {
-          dodgyWAL1.close();
-        }
-        if (dodgyWAL2 != null) {
-          dodgyWAL2.close();
+        if (dodgyWAL != null) {
+          dodgyWAL.close();
         }
       } catch (Exception e) {
         LOG.info("On way out", e);
@@ -606,11 +416,11 @@ public class TestWALLockup {
   }
 
   /**
-   * @return A region on which you must call
-   *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
+   * @return A region on which you must call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)}
+   *         when done.
    */
-  public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
-  throws IOException {
+  private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
+      throws IOException {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);


[06/18] hbase git commit: HBASE-19900 Region-level exception destroy the result of batch

Posted by zh...@apache.org.
HBASE-19900 Region-level exception destroy the result of batch


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

Branch: refs/heads/HBASE-19064
Commit: d8b999e6950bc01e8aab8ecce437e710e4a98e15
Parents: a5b86dd
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Tue Feb 6 05:33:37 2018 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Wed Feb 7 08:41:56 2018 +0800

----------------------------------------------------------------------
 .../client/AsyncBatchRpcRetryingCaller.java     |  29 +--
 .../hbase/client/AsyncRequestFutureImpl.java    | 243 +++++++++---------
 .../TestAsyncProcessWithRegionException.java    | 252 +++++++++++++++++++
 .../client/TestMalformedCellFromClient.java     | 173 +++++++++++++
 4 files changed, 558 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d8b999e6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
index 62ee0ab..51b89a9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
@@ -251,8 +251,8 @@ class AsyncBatchRpcRetryingCaller<T> {
 
   @SuppressWarnings("unchecked")
   private void onComplete(Action action, RegionRequest regionReq, int tries, ServerName serverName,
-      RegionResult regionResult, List<Action> failedActions) {
-    Object result = regionResult.result.get(action.getOriginalIndex());
+      RegionResult regionResult, List<Action> failedActions, Throwable regionException) {
+    Object result = regionResult.result.getOrDefault(action.getOriginalIndex(), regionException);
     if (result == null) {
       LOG.error("Server " + serverName + " sent us neither result nor exception for row '"
           + Bytes.toStringBinary(action.getAction().getRow()) + "' of "
@@ -279,27 +279,28 @@ class AsyncBatchRpcRetryingCaller<T> {
     List<Action> failedActions = new ArrayList<>();
     actionsByRegion.forEach((rn, regionReq) -> {
       RegionResult regionResult = resp.getResults().get(rn);
+      Throwable regionException = resp.getException(rn);
       if (regionResult != null) {
         regionReq.actions.forEach(
-          action -> onComplete(action, regionReq, tries, serverName, regionResult, failedActions));
+          action -> onComplete(action, regionReq, tries, serverName, regionResult, failedActions,
+            regionException));
       } else {
-        Throwable t = resp.getException(rn);
         Throwable error;
-        if (t == null) {
+        if (regionException == null) {
           LOG.error(
             "Server sent us neither results nor exceptions for " + Bytes.toStringBinary(rn));
           error = new RuntimeException("Invalid response");
         } else {
-          error = translateException(t);
-          logException(tries, () -> Stream.of(regionReq), error, serverName);
-          conn.getLocator().updateCachedLocation(regionReq.loc, error);
-          if (error instanceof DoNotRetryIOException || tries >= maxAttempts) {
-            failAll(regionReq.actions.stream(), tries, error, serverName);
-            return;
-          }
-          addError(regionReq.actions, error, serverName);
-          failedActions.addAll(regionReq.actions);
+          error = translateException(regionException);
+        }
+        logException(tries, () -> Stream.of(regionReq), error, serverName);
+        conn.getLocator().updateCachedLocation(regionReq.loc, error);
+        if (error instanceof DoNotRetryIOException || tries >= maxAttempts) {
+          failAll(regionReq.actions.stream(), tries, error, serverName);
+          return;
         }
+        addError(regionReq.actions, error, serverName);
+        failedActions.addAll(regionReq.actions);
       }
     });
     if (!failedActions.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8b999e6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
index 3ab94c5..ace74f9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
@@ -19,8 +19,6 @@
 
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -36,28 +34,29 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.hbase.CallQueueTooBigException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.RetryImmediatelyException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.trace.TraceUtil;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.htrace.core.Tracer;
+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.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 
 /**
  * The context, and return value, for a single submit/submitAll call.
@@ -152,7 +151,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
         if (results[index] != null) return;
         // We set the number of calls here. After that any path must call setResult/setError.
         // True even for replicas that are not found - if we refuse to send we MUST set error.
-        results[index] = new ReplicaResultState(locs.length);
+        updateResult(index, new ReplicaResultState(locs.length));
       }
       for (int i = 1; i < locs.length; ++i) {
         Action replicaAction = new Action(action, i);
@@ -234,7 +233,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
         } else {
           if (results != null) {
             SingleResponse singleResponse = (SingleResponse) res;
-            results[0] = singleResponse.getEntry();
+            updateResult(0, singleResponse.getEntry());
           }
           decActionCounter(1);
         }
@@ -706,27 +705,17 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
     Retry canRetry = errorsByServer.canTryMore(numAttempt)
         ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED;
 
-    if (tableName == null && ClientExceptionsUtil.isMetaClearingException(t)) {
-      // tableName is null when we made a cross-table RPC call.
-      asyncProcess.connection.clearCaches(server);
-    }
-    int failed = 0, stopped = 0;
+    cleanServerCache(server, t);
+    int failed = 0;
+    int stopped = 0;
     List<Action> toReplay = new ArrayList<>();
     for (Map.Entry<byte[], List<Action>> e : rsActions.actions.entrySet()) {
       byte[] regionName = e.getKey();
-      byte[] row = e.getValue().iterator().next().getAction().getRow();
+      byte[] row = e.getValue().get(0).getAction().getRow();
       // Do not use the exception for updating cache because it might be coming from
       // any of the regions in the MultiAction.
-      try {
-        if (tableName != null) {
-          asyncProcess.connection.updateCachedLocations(tableName, regionName, row,
-              ClientExceptionsUtil.isMetaClearingException(t) ? null : t, server);
-        }
-      } catch (Throwable ex) {
-        // That should never happen, but if it did, we want to make sure
-        // we still process errors
-        LOG.error("Couldn't update cached region locations: " + ex);
-      }
+      updateCachedLocations(server, regionName, row,
+        ClientExceptionsUtil.isMetaClearingException(t) ? null : t);
       for (Action action : e.getValue()) {
         Retry retry = manageError(
             action.getOriginalIndex(), action.getAction(), canRetry, t, server);
@@ -819,6 +808,9 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
                                   ServerName server, MultiResponse responses, int numAttempt) {
     assert responses != null;
 
+    Map<byte[], MultiResponse.RegionResult> results = responses.getResults();
+    updateStats(server, results);
+
     // Success or partial success
     // Analyze detailed results. We can still have individual failures to be redo.
     // two specific throwables are managed:
@@ -826,126 +818,111 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
     //  - RegionMovedException: we update the cache with the new region location
 
     List<Action> toReplay = new ArrayList<>();
-    Throwable throwable = null;
+    Throwable lastException = null;
     int failureCount = 0;
-    boolean canRetry = true;
-
-    Map<byte[], MultiResponse.RegionResult> results = responses.getResults();
-    updateStats(server, results);
-
-    int failed = 0, stopped = 0;
+    int failed = 0;
+    int stopped = 0;
+    Retry retry = null;
     // Go by original action.
     for (Map.Entry<byte[], List<Action>> regionEntry : multiAction.actions.entrySet()) {
       byte[] regionName = regionEntry.getKey();
-      Map<Integer, Object> regionResults = results.get(regionName) == null
-          ?  null : results.get(regionName).result;
-      if (regionResults == null) {
-        if (!responses.getExceptions().containsKey(regionName)) {
-          LOG.error("Server sent us neither results nor exceptions for "
-              + Bytes.toStringBinary(regionName));
-          responses.getExceptions().put(regionName, new RuntimeException("Invalid response"));
-        }
-        continue;
-      }
+
+      Throwable regionException = responses.getExceptions().get(regionName);
+      cleanServerCache(server, regionException);
+
+      Map<Integer, Object> regionResults =
+        results.containsKey(regionName) ? results.get(regionName).result : Collections.emptyMap();
       boolean regionFailureRegistered = false;
       for (Action sentAction : regionEntry.getValue()) {
         Object result = regionResults.get(sentAction.getOriginalIndex());
+        if (result == null) {
+          if (regionException == null) {
+            LOG.error("Server sent us neither results nor exceptions for "
+              + Bytes.toStringBinary(regionName)
+              + ", numAttempt:" + numAttempt);
+            regionException = new RuntimeException("Invalid response");
+          }
+          // If the row operation encounters the region-lever error, the exception of action may be
+          // null.
+          result = regionException;
+        }
         // Failure: retry if it's make sense else update the errors lists
-        if (result == null || result instanceof Throwable) {
+        if (result instanceof Throwable) {
+          Throwable actionException = (Throwable) result;
           Row row = sentAction.getAction();
-          throwable = ClientExceptionsUtil.findException(result);
+          lastException = regionException != null ? regionException
+            : ClientExceptionsUtil.findException(actionException);
           // Register corresponding failures once per server/once per region.
           if (!regionFailureRegistered) {
             regionFailureRegistered = true;
-            try {
-              asyncProcess.connection.updateCachedLocations(
-                  tableName, regionName, row.getRow(), result, server);
-            } catch (Throwable ex) {
-              // That should never happen, but if it did, we want to make sure
-              // we still process errors
-              LOG.error("Couldn't update cached region locations: " + ex);
-            }
+            updateCachedLocations(server, regionName, row.getRow(), actionException);
           }
-          if (failureCount == 0) {
+          if (retry == null) {
             errorsByServer.reportServerError(server);
             // We determine canRetry only once for all calls, after reporting server failure.
-            canRetry = errorsByServer.canTryMore(numAttempt);
+            retry = errorsByServer.canTryMore(numAttempt) ?
+              Retry.YES : Retry.NO_RETRIES_EXHAUSTED;
           }
           ++failureCount;
-          Retry retry = manageError(sentAction.getOriginalIndex(), row,
-              canRetry ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED, (Throwable) result, server);
-          if (retry == Retry.YES) {
-            toReplay.add(sentAction);
-          } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
-            ++stopped;
-          } else {
-            ++failed;
+          switch (manageError(sentAction.getOriginalIndex(), row, retry, actionException,
+            server)) {
+            case YES:
+              toReplay.add(sentAction);
+              break;
+            case NO_OTHER_SUCCEEDED:
+              ++stopped;
+              break;
+            default:
+              ++failed;
+              break;
           }
         } else {
-          if (callback != null) {
-            try {
-              //noinspection unchecked
-              // TODO: would callback expect a replica region name if it gets one?
-              this.callback.update(regionName, sentAction.getAction().getRow(), (CResult) result);
-            } catch (Throwable t) {
-              LOG.error("User callback threw an exception for "
-                  + Bytes.toStringBinary(regionName) + ", ignoring", t);
-            }
-          }
+          invokeCallBack(regionName, sentAction.getAction().getRow(), (CResult) result);
           setResult(sentAction, result);
         }
       }
     }
+    if (toReplay.isEmpty()) {
+      logNoResubmit(server, numAttempt, failureCount, lastException, failed, stopped);
+    } else {
+      resubmit(server, toReplay, numAttempt, failureCount, lastException);
+    }
+  }
 
-    // The failures global to a region. We will use for multiAction we sent previously to find the
-    //   actions to replay.
-    for (Map.Entry<byte[], Throwable> throwableEntry : responses.getExceptions().entrySet()) {
-      throwable = throwableEntry.getValue();
-      byte[] region = throwableEntry.getKey();
-      List<Action> actions = multiAction.actions.get(region);
-      if (actions == null || actions.isEmpty()) {
-        throw new IllegalStateException("Wrong response for the region: " +
-            HRegionInfo.encodeRegionName(region));
-      }
-
-      if (failureCount == 0) {
-        errorsByServer.reportServerError(server);
-        canRetry = errorsByServer.canTryMore(numAttempt);
-      }
-      if (null == tableName && ClientExceptionsUtil.isMetaClearingException(throwable)) {
-        // For multi-actions, we don't have a table name, but we want to make sure to clear the
-        // cache in case there were location-related exceptions. We don't to clear the cache
-        // for every possible exception that comes through, however.
-        asyncProcess.connection.clearCaches(server);
-      } else {
-        try {
-          asyncProcess.connection.updateCachedLocations(
-              tableName, region, actions.get(0).getAction().getRow(), throwable, server);
-        } catch (Throwable ex) {
-          // That should never happen, but if it did, we want to make sure
-          // we still process errors
-          LOG.error("Couldn't update cached region locations: " + ex);
-        }
-      }
-      failureCount += actions.size();
+  private void updateCachedLocations(ServerName server, byte[] regionName, byte[] row,
+    Throwable rowException) {
+    if (tableName == null) {
+      return;
+    }
+    try {
+      asyncProcess.connection
+        .updateCachedLocations(tableName, regionName, row, rowException, server);
+    } catch (Throwable ex) {
+      // That should never happen, but if it did, we want to make sure
+      // we still process errors
+      LOG.error("Couldn't update cached region locations: " + ex);
+    }
+  }
 
-      for (Action action : actions) {
-        Row row = action.getAction();
-        Retry retry = manageError(action.getOriginalIndex(), row,
-            canRetry ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED, throwable, server);
-        if (retry == Retry.YES) {
-          toReplay.add(action);
-        } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
-          ++stopped;
-        } else {
-          ++failed;
-        }
+  private void invokeCallBack(byte[] regionName, byte[] row, CResult result) {
+    if (callback != null) {
+      try {
+        //noinspection unchecked
+        // TODO: would callback expect a replica region name if it gets one?
+        this.callback.update(regionName, row, result);
+      } catch (Throwable t) {
+        LOG.error("User callback threw an exception for "
+          + Bytes.toStringBinary(regionName) + ", ignoring", t);
       }
     }
-    if (toReplay.isEmpty()) {
-      logNoResubmit(server, numAttempt, failureCount, throwable, failed, stopped);
-    } else {
-      resubmit(server, toReplay, numAttempt, failureCount, throwable);
+  }
+
+  private void cleanServerCache(ServerName server, Throwable regionException) {
+    if (tableName == null && ClientExceptionsUtil.isMetaClearingException(regionException)) {
+      // For multi-actions, we don't have a table name, but we want to make sure to clear the
+      // cache in case there were location-related exceptions. We don't to clear the cache
+      // for every possible exception that comes through, however.
+      asyncProcess.connection.clearCaches(server);
     }
   }
 
@@ -1041,7 +1018,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
       if (results[index] != state) {
         throw new AssertionError("We set the callCount but someone else replaced the result");
       }
-      results[index] = result;
+      updateResult(index, result);
     }
 
     decActionCounter(index);
@@ -1099,7 +1076,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
         if (results[index] != state) {
           throw new AssertionError("We set the callCount but someone else replaced the result");
         }
-        results[index] = throwable;
+        updateResult(index, throwable);
       }
       decActionCounter(index);
     }
@@ -1130,7 +1107,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
       if (isFromReplica) {
         throw new AssertionError("Unexpected stale result for " + row);
       }
-      results[index] = result;
+      updateResult(index, result);
     } else {
       synchronized (replicaResultLock) {
         resObj = results[index];
@@ -1138,7 +1115,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
           if (isFromReplica) {
             throw new AssertionError("Unexpected stale result for " + row);
           }
-          results[index] = result;
+          updateResult(index, result);
         }
       }
     }
@@ -1276,4 +1253,20 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
     return new MultiServerCallable(asyncProcess.connection, tableName, server,
         multi, asyncProcess.rpcFactory.newController(), rpcTimeout, tracker, multi.getPriority());
   }
+
+  private void updateResult(int index, Object result) {
+    Object current = results[index];
+    if (current != null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("The result is assigned repeatedly! current:" + current
+          + ", new:" + result);
+      }
+    }
+    results[index] = result;
+  }
+
+  @VisibleForTesting
+  long getNumberOfActionsInProgress() {
+    return actionsInProgress.get();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8b999e6/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
new file mode 100644
index 0000000..c46385e
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
@@ -0,0 +1,252 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * The purpose of this test is to make sure the region exception won't corrupt the results
+ * of batch. The prescription is shown below.
+ * 1) honor the action result rather than region exception. If the action have both of true result
+ * and region exception, the action is fine as the exception is caused by other actions
+ * which are in the same region.
+ * 2) honor the action exception rather than region exception. If the action have both of action
+ * exception and region exception, we deal with the action exception only. If we also
+ * handle the region exception for the same action, it will introduce the negative count of
+ * actions in progress. The AsyncRequestFuture#waitUntilDone will block forever.
+ *
+ * This bug can be reproduced by real use case. see TestMalformedCellFromClient(in branch-1.4+).
+ * It uses the batch of RowMutations to present the bug. Given that the batch of RowMutations is
+ * only supported by branch-1.4+, perhaps the branch-1.3 and branch-1.2 won't encounter this issue.
+ * We still backport the fix to branch-1.3 and branch-1.2 in case we ignore some write paths.
+ */
+@Category({ ClientTests.class, SmallTests.class })
+public class TestAsyncProcessWithRegionException {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAsyncProcessWithRegionException.class);
+
+  private static final Result EMPTY_RESULT = Result.create(null, true);
+  private static final IOException IOE = new IOException("YOU CAN'T PASS");
+  private static final Configuration CONF = new Configuration();
+  private static final TableName DUMMY_TABLE = TableName.valueOf("DUMMY_TABLE");
+  private static final byte[] GOOD_ROW = Bytes.toBytes("GOOD_ROW");
+  private static final byte[] BAD_ROW = Bytes.toBytes("BAD_ROW");
+  private static final byte[] BAD_ROW_WITHOUT_ACTION_EXCEPTION =
+    Bytes.toBytes("BAD_ROW_WITHOUT_ACTION_EXCEPTION");
+  private static final byte[] FAMILY = Bytes.toBytes("FAMILY");
+  private static final ServerName SERVER_NAME = ServerName.valueOf("s1,1,1");
+  private static final RegionInfo REGION_INFO =
+    RegionInfoBuilder.newBuilder(DUMMY_TABLE)
+      .setStartKey(HConstants.EMPTY_START_ROW)
+      .setEndKey(HConstants.EMPTY_END_ROW)
+      .setSplit(false)
+      .setRegionId(1)
+      .build();
+
+  private static final HRegionLocation REGION_LOCATION =
+    new HRegionLocation(REGION_INFO, SERVER_NAME);
+
+  @BeforeClass
+  public static void setUpBeforeClass() {
+    // disable the retry
+    CONF.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
+  }
+
+  @Test
+  public void testSuccessivePut() throws Exception {
+    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
+
+    List<Put> puts = new ArrayList<>(1);
+    puts.add(new Put(GOOD_ROW).addColumn(FAMILY, FAMILY, FAMILY));
+    final int expectedSize = puts.size();
+    AsyncRequestFuture arf = ap.submit(DUMMY_TABLE, puts);
+    arf.waitUntilDone();
+    Object[] result = arf.getResults();
+    assertEquals(expectedSize, result.length);
+    for (Object r : result) {
+      assertEquals(Result.class, r.getClass());
+    }
+    assertTrue(puts.isEmpty());
+    assertActionsInProgress(arf);
+  }
+
+  @Test
+  public void testFailedPut() throws Exception {
+    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
+
+    List<Put> puts = new ArrayList<>(2);
+    puts.add(new Put(GOOD_ROW).addColumn(FAMILY, FAMILY, FAMILY));
+    // this put should fail
+    puts.add(new Put(BAD_ROW).addColumn(FAMILY, FAMILY, FAMILY));
+    final int expectedSize = puts.size();
+
+    AsyncRequestFuture arf = ap.submit(DUMMY_TABLE, puts);
+    arf.waitUntilDone();
+    // There is a failed puts
+    assertError(arf, 1);
+    Object[] result = arf.getResults();
+    assertEquals(expectedSize, result.length);
+    assertEquals(Result.class, result[0].getClass());
+    assertTrue(result[1] instanceof IOException);
+    assertTrue(puts.isEmpty());
+    assertActionsInProgress(arf);
+  }
+
+  @Test
+  public void testFailedPutWithoutActionException() throws Exception {
+    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
+
+    List<Put> puts = new ArrayList<>(3);
+    puts.add(new Put(GOOD_ROW).addColumn(FAMILY, FAMILY, FAMILY));
+    // this put should fail
+    puts.add(new Put(BAD_ROW).addColumn(FAMILY, FAMILY, FAMILY));
+    // this put should fail, and it won't have action exception
+    puts.add(new Put(BAD_ROW_WITHOUT_ACTION_EXCEPTION).addColumn(FAMILY, FAMILY, FAMILY));
+    final int expectedSize = puts.size();
+
+    AsyncRequestFuture arf = ap.submit(DUMMY_TABLE, puts);
+    arf.waitUntilDone();
+    // There are two failed puts
+    assertError(arf, 2);
+    Object[] result = arf.getResults();
+    assertEquals(expectedSize, result.length);
+    assertEquals(Result.class, result[0].getClass());
+    assertTrue(result[1] instanceof IOException);
+    assertTrue(result[2] instanceof IOException);
+    assertTrue(puts.isEmpty());
+    assertActionsInProgress(arf);
+  }
+
+  private static void assertError(AsyncRequestFuture arf, int expectedCountOfFailure) {
+    assertTrue(arf.hasError());
+    RetriesExhaustedWithDetailsException e = arf.getErrors();
+    List<Throwable> errors = e.getCauses();
+    assertEquals(expectedCountOfFailure, errors.size());
+    for (Throwable t : errors) {
+      assertTrue(t instanceof IOException);
+    }
+  }
+
+  private static void assertActionsInProgress(AsyncRequestFuture arf) {
+    if (arf instanceof AsyncRequestFutureImpl) {
+      assertEquals(0, ((AsyncRequestFutureImpl) arf).getNumberOfActionsInProgress());
+    }
+  }
+
+  private static ClusterConnection createHConnection() throws IOException {
+    ClusterConnection hc = Mockito.mock(ClusterConnection.class);
+    NonceGenerator ng = Mockito.mock(NonceGenerator.class);
+    Mockito.when(ng.getNonceGroup()).thenReturn(HConstants.NO_NONCE);
+    Mockito.when(hc.getNonceGenerator()).thenReturn(ng);
+    Mockito.when(hc.getConfiguration()).thenReturn(CONF);
+    Mockito.when(hc.getConnectionConfiguration()).thenReturn(new ConnectionConfiguration(CONF));
+    setMockLocation(hc, GOOD_ROW, new RegionLocations(REGION_LOCATION));
+    setMockLocation(hc, BAD_ROW, new RegionLocations(REGION_LOCATION));
+    Mockito
+      .when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), Mockito.anyBoolean()))
+      .thenReturn(Collections.singletonList(REGION_LOCATION));
+    return hc;
+  }
+
+  private static void setMockLocation(ClusterConnection hc, byte[] row, RegionLocations result)
+    throws IOException {
+    Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(),
+      Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result);
+    Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(),
+      Mockito.anyBoolean())).thenReturn(result);
+  }
+
+  private static class MyAsyncProcess extends AsyncProcess {
+    private final ExecutorService service = Executors.newFixedThreadPool(5);
+
+    MyAsyncProcess(ClusterConnection hc, Configuration conf) {
+      super(hc, conf, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf));
+    }
+
+    public AsyncRequestFuture submit(TableName tableName, List<? extends Row> rows)
+      throws InterruptedIOException {
+      return submit(AsyncProcessTask.newBuilder()
+        .setPool(service)
+        .setTableName(tableName)
+        .setRowAccess(rows)
+        .setSubmittedRows(AsyncProcessTask.SubmittedRows.NORMAL)
+        .setNeedResults(true)
+        .setRpcTimeout(HConstants.DEFAULT_HBASE_RPC_TIMEOUT)
+        .setOperationTimeout(HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT)
+        .build());
+    }
+
+    @Override
+    protected RpcRetryingCaller<AbstractResponse> createCaller(
+      CancellableRegionServerCallable callable, int rpcTimeout) {
+      MultiServerCallable callable1 = (MultiServerCallable) callable;
+      MultiResponse mr = new MultiResponse();
+      callable1.getMulti().actions.forEach((regionName, actions) -> {
+        actions.forEach(action -> {
+          if (Bytes.equals(action.getAction().getRow(), GOOD_ROW)) {
+            mr.add(regionName, action.getOriginalIndex(), EMPTY_RESULT);
+          } else if (Bytes.equals(action.getAction().getRow(), BAD_ROW)) {
+            mr.add(regionName, action.getOriginalIndex(), IOE);
+          }
+        });
+      });
+      mr.addException(REGION_INFO.getRegionName(), IOE);
+      return new RpcRetryingCallerImpl<AbstractResponse>(100, 500, 0, 9) {
+        @Override
+        public AbstractResponse callWithoutRetries(RetryingCallable<AbstractResponse> callable,
+          int callTimeout) {
+          try {
+            // sleep one second in order for threadpool to start another thread instead of reusing
+            // existing one.
+            Thread.sleep(1000);
+          } catch (InterruptedException e) {
+            // pass
+          }
+          return mr;
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8b999e6/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMalformedCellFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMalformedCellFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMalformedCellFromClient.java
new file mode 100644
index 0000000..e44a2e9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMalformedCellFromClient.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import static org.junit.Assert.assertEquals;
+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.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+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;
+
+/**
+ * The purpose of this test is to make sure the region exception won't corrupt the results
+ * of batch. The prescription is shown below.
+ * 1) honor the action result rather than region exception. If the action have both of true result
+ * and region exception, the action is fine as the exception is caused by other actions
+ * which are in the same region.
+ * 2) honor the action exception rather than region exception. If the action have both of action
+ * exception and region exception, we deal with the action exception only. If we also
+ * handle the region exception for the same action, it will introduce the negative count of
+ * actions in progress. The AsyncRequestFuture#waitUntilDone will block forever.
+ *
+ * The no-cluster test is in TestAsyncProcessWithRegionException.
+ */
+@Category({ MediumTests.class, ClientTests.class })
+public class TestMalformedCellFromClient {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMalformedCellFromClient.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final byte[] FAMILY = Bytes.toBytes("testFamily");
+  private static final int CELL_SIZE = 100;
+  private static final TableName TABLE_NAME = TableName.valueOf("TestMalformedCellFromClient");
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // disable the retry
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @Before
+  public void before() throws Exception {
+    TableDescriptor desc = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY))
+      .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, String.valueOf(CELL_SIZE)).build();
+    TEST_UTIL.getConnection().getAdmin().createTable(desc);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    for (TableDescriptor htd : TEST_UTIL.getAdmin().listTableDescriptors()) {
+      TEST_UTIL.deleteTable(htd.getTableName());
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * The purpose of this ut is to check the consistency between the exception and results.
+   * If the RetriesExhaustedWithDetailsException contains the whole batch,
+   * each result should be of IOE. Otherwise, the row operation which is not in the exception
+   * should have a true result.
+   */
+  @Test
+  public void testRegionException() throws InterruptedException, IOException {
+    List<Row> batches = new ArrayList<>();
+    batches.add(new Put(Bytes.toBytes("good")).addColumn(FAMILY, null, new byte[10]));
+    // the rm is used to prompt the region exception.
+    // see RSRpcServices#multi
+    RowMutations rm = new RowMutations(Bytes.toBytes("fail"));
+    rm.add(new Put(rm.getRow()).addColumn(FAMILY, null, new byte[CELL_SIZE]));
+    batches.add(rm);
+    Object[] results = new Object[batches.size()];
+
+    try (Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME)) {
+      Throwable exceptionByCaught = null;
+      try {
+        table.batch(batches, results);
+        fail("Where is the exception? We put the malformed cells!!!");
+      } catch (RetriesExhaustedWithDetailsException e) {
+        for (Throwable throwable : e.getCauses()) {
+          assertNotNull(throwable);
+        }
+        assertEquals(1, e.getNumExceptions());
+        exceptionByCaught = e.getCause(0);
+      }
+      for (Object obj : results) {
+        assertNotNull(obj);
+      }
+      assertEquals(Result.class, results[0].getClass());
+      assertEquals(exceptionByCaught.getClass(), results[1].getClass());
+      Result result = table.get(new Get(Bytes.toBytes("good")));
+      assertEquals(1, result.size());
+      Cell cell = result.getColumnLatestCell(FAMILY, null);
+      assertTrue(Bytes.equals(CellUtil.cloneValue(cell), new byte[10]));
+    }
+  }
+
+  /**
+   * The purpose of this ut is to check the consistency between the exception and results.
+   * If the RetriesExhaustedWithDetailsException contains the whole batch,
+   * each result should be of IOE. Otherwise, the row operation which is not in the exception
+   * should have a true result.
+   */
+  @Test
+  public void testRegionExceptionByAsync() throws Exception {
+    List<Row> batches = new ArrayList<>();
+    batches.add(new Put(Bytes.toBytes("good")).addColumn(FAMILY, null, new byte[10]));
+    // the rm is used to prompt the region exception.
+    // see RSRpcServices#multi
+    RowMutations rm = new RowMutations(Bytes.toBytes("fail"));
+    rm.add(new Put(rm.getRow()).addColumn(FAMILY, null, new byte[CELL_SIZE]));
+    batches.add(rm);
+    try (AsyncConnection asyncConnection = ConnectionFactory
+      .createAsyncConnection(TEST_UTIL.getConfiguration()).get()) {
+      AsyncTable<AdvancedScanResultConsumer> table = asyncConnection.getTable(TABLE_NAME);
+      List<CompletableFuture<AdvancedScanResultConsumer>> results = table.batch(batches);
+      assertEquals(2, results.size());
+      try {
+        results.get(1).get();
+        fail("Where is the exception? We put the malformed cells!!!");
+      } catch (ExecutionException e) {
+        // pass
+      }
+      Result result = table.get(new Get(Bytes.toBytes("good"))).get();
+      assertEquals(1, result.size());
+      Cell cell = result.getColumnLatestCell(FAMILY, null);
+      assertTrue(Bytes.equals(CellUtil.cloneValue(cell), new byte[10]));
+    }
+  }
+}


[11/18] hbase git commit: HBASE-19083 Introduce a new log writer which can write to two HDFSes

Posted by zh...@apache.org.
HBASE-19083 Introduce a new log writer which can write to two HDFSes


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

Branch: refs/heads/HBASE-19064
Commit: 8f260d0694e5b913c50ca786195663a85f469099
Parents: 277ce3d
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 11 21:08:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 8 16:48:23 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  16 ++-
 .../regionserver/wal/CombinedAsyncWriter.java   | 134 ++++++++++++++++++
 .../hbase/regionserver/wal/DualAsyncFSWAL.java  |  67 +++++++++
 .../wal/AbstractTestProtobufLog.java            | 110 +++------------
 .../regionserver/wal/ProtobufLogTestHelper.java |  99 ++++++++++++++
 .../regionserver/wal/TestAsyncProtobufLog.java  |  32 +----
 .../wal/TestCombinedAsyncWriter.java            | 136 +++++++++++++++++++
 .../hbase/regionserver/wal/TestProtobufLog.java |  14 +-
 .../regionserver/wal/WriterOverAsyncWriter.java |  63 +++++++++
 9 files changed, 531 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index 19d89df..8e57441 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -606,12 +606,16 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  @Override
-  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+  protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException {
     return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup,
       channelClass);
   }
 
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    return createAsyncWriter(fs, path);
+  }
+
   private void waitForSafePoint() {
     consumeLock.lock();
     try {
@@ -654,13 +658,13 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     } finally {
       consumeLock.unlock();
     }
-    return executeClose(closeExecutor, oldWriter);
+    return executeClose(oldWriter);
   }
 
   @Override
   protected void doShutdown() throws IOException {
     waitForSafePoint();
-    executeClose(closeExecutor, writer);
+    executeClose(writer);
     closeExecutor.shutdown();
     try {
       if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) {
@@ -698,7 +702,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  private static long executeClose(ExecutorService closeExecutor, AsyncWriter writer) {
+  protected final long executeClose(AsyncWriter writer) {
     long fileLength;
     if (writer != null) {
       fileLength = writer.getLength();
@@ -706,7 +710,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         try {
           writer.close();
         } catch (IOException e) {
-          LOG.warn("close old writer failed", e);
+          LOG.warn("close writer failed", e);
         }
       });
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java
new file mode 100644
index 0000000..8ecfede
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.wal;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+
+/**
+ * An {@link AsyncWriter} wrapper which writes data to a set of {@link AsyncWriter} instances.
+ */
+@InterfaceAudience.Private
+public abstract class CombinedAsyncWriter implements AsyncWriter {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CombinedAsyncWriter.class);
+
+  protected final ImmutableList<AsyncWriter> writers;
+
+  protected CombinedAsyncWriter(ImmutableList<AsyncWriter> writers) {
+    this.writers = writers;
+  }
+
+  @Override
+  public long getLength() {
+    return writers.get(0).getLength();
+  }
+
+  @Override
+  public void close() throws IOException {
+    Exception error = null;
+    for (AsyncWriter writer : writers) {
+      try {
+        writer.close();
+      } catch (Exception e) {
+        LOG.warn("close writer failed", e);
+        if (error == null) {
+          error = e;
+        }
+      }
+    }
+    if (error != null) {
+      throw new IOException("Failed to close at least one writer, please see the warn log above. " +
+        "The cause is the first exception occured", error);
+    }
+  }
+
+  protected abstract void doSync(CompletableFuture<Long> future);
+
+  @Override
+  public CompletableFuture<Long> sync() {
+    CompletableFuture<Long> future = new CompletableFuture<>();
+    doSync(future);
+    return future;
+  }
+
+  @Override
+  public void append(Entry entry) {
+    writers.forEach(w -> w.append(entry));
+  }
+
+  public enum Mode {
+    SEQUENTIAL, PARALLEL
+  }
+
+  public static CombinedAsyncWriter create(Mode mode, AsyncWriter writer, AsyncWriter... writers) {
+    ImmutableList<AsyncWriter> ws =
+        ImmutableList.<AsyncWriter> builder().add(writer).add(writers).build();
+    switch (mode) {
+      case SEQUENTIAL:
+        return new CombinedAsyncWriter(ws) {
+
+          private void doSync(CompletableFuture<Long> future, Long length, int index) {
+            if (index == writers.size()) {
+              future.complete(length);
+              return;
+            }
+            writers.get(index).sync().whenComplete((len, error) -> {
+              if (error != null) {
+                future.completeExceptionally(error);
+                return;
+              }
+              doSync(future, len, index + 1);
+            });
+          }
+
+          @Override
+          protected void doSync(CompletableFuture<Long> future) {
+            doSync(future, null, 0);
+          }
+        };
+      case PARALLEL:
+        return new CombinedAsyncWriter(ws) {
+
+          @Override
+          protected void doSync(CompletableFuture<Long> future) {
+            AtomicInteger remaining = new AtomicInteger(writers.size());
+            writers.forEach(w -> w.sync().whenComplete((length, error) -> {
+              if (error != null) {
+                future.completeExceptionally(error);
+                return;
+              }
+              if (remaining.decrementAndGet() == 0) {
+                future.complete(length);
+              }
+            }));
+          }
+        };
+      default:
+        throw new IllegalArgumentException("Unknown mode: " + mode);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
new file mode 100644
index 0000000..6bf9e02
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.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.regionserver.wal;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
+/**
+ * An AsyncFSWAL which writes data to two filesystems.
+ */
+@InterfaceAudience.Private
+public class DualAsyncFSWAL extends AsyncFSWAL {
+
+  private final FileSystem remoteFs;
+
+  private final Path remoteWalDir;
+
+  public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteRootDir,
+      String logDir, String archiveDir, Configuration conf, List<WALActionsListener> listeners,
+      boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup,
+      Class<? extends Channel> channelClass) throws FailedLogCloseException, IOException {
+    super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix,
+        eventLoopGroup, channelClass);
+    this.remoteFs = remoteFs;
+    this.remoteWalDir = new Path(remoteRootDir, logDir);
+  }
+
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    AsyncWriter localWriter = super.createWriterInstance(path);
+    AsyncWriter remoteWriter;
+    boolean succ = false;
+    try {
+      remoteWriter = createAsyncWriter(remoteFs, new Path(remoteWalDir, path.getName()));
+      succ = true;
+    } finally {
+      if (!succ) {
+        executeClose(localWriter);
+      }
+    }
+    return CombinedAsyncWriter.create(CombinedAsyncWriter.Mode.SEQUENTIAL, remoteWriter,
+      localWriter);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
index c3f3277..5098609 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
@@ -18,33 +18,15 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.Closeable;
 import java.io.IOException;
 import org.apache.hadoop.fs.FileStatus;
 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.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.SampleRegionWALCoprocessor;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hadoop.hbase.wal.WALProvider;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -56,8 +38,8 @@ import org.junit.rules.TestName;
 /**
  * WAL tests that can be reused across providers.
  */
-public abstract class AbstractTestProtobufLog<W extends Closeable> {
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+public abstract class AbstractTestProtobufLog {
+  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   protected FileSystem fs;
   protected Path dir;
@@ -93,14 +75,7 @@ public abstract class AbstractTestProtobufLog<W extends Closeable> {
     TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
 
     // faster failover with cluster.shutdown();fs.close() idiom
-    TEST_UTIL.getConfiguration()
-        .setInt("hbase.ipc.client.connect.max.retries", 1);
-    TEST_UTIL.getConfiguration().setInt(
-        "dfs.client.block.recovery.retries", 1);
-    TEST_UTIL.getConfiguration().setInt(
-      "hbase.ipc.client.connection.maxidletime", 500);
-    TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
-        SampleRegionWALCoprocessor.class.getName());
+    TEST_UTIL.getConfiguration().setInt("dfs.client.block.recovery.retries", 1);
     TEST_UTIL.startMiniDFSCluster(3);
   }
 
@@ -131,77 +106,24 @@ public abstract class AbstractTestProtobufLog<W extends Closeable> {
    * @throws IOException
    */
   private void doRead(boolean withTrailer) throws IOException {
-    final int columnCount = 5;
-    final int recordCount = 5;
-    final TableName tableName =
-        TableName.valueOf("tablename");
-    final byte[] row = Bytes.toBytes("row");
+    int columnCount = 5;
+    int recordCount = 5;
+    TableName tableName = TableName.valueOf("tablename");
+    byte[] row = Bytes.toBytes("row");
     long timestamp = System.currentTimeMillis();
     Path path = new Path(dir, "tempwal");
     // delete the log if already exists, for test only
     fs.delete(path, true);
-    W writer = null;
-    ProtobufLogReader reader = null;
-    try {
-      HRegionInfo hri = new HRegionInfo(tableName,
-          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      fs.mkdirs(dir);
-      // Write log in pb format.
-      writer = createWriter(path);
-      for (int i = 0; i < recordCount; ++i) {
-        WALKeyImpl key = new WALKeyImpl(
-            hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
-        WALEdit edit = new WALEdit();
-        for (int j = 0; j < columnCount; ++j) {
-          if (i == 0) {
-            htd.addFamily(new HColumnDescriptor("column" + j));
-          }
-          String value = i + "" + j;
-          edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
-        }
-        append(writer, new WAL.Entry(key, edit));
-      }
-      sync(writer);
-      if (withTrailer) writer.close();
-
-      // Now read the log using standard means.
-      reader = (ProtobufLogReader) wals.createReader(fs, path);
-      if (withTrailer) {
-        assertNotNull(reader.trailer);
-      } else {
-        assertNull(reader.trailer);
-      }
-      for (int i = 0; i < recordCount; ++i) {
-        WAL.Entry entry = reader.next();
-        assertNotNull(entry);
-        assertEquals(columnCount, entry.getEdit().size());
-        assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
-        assertEquals(tableName, entry.getKey().getTableName());
-        int idx = 0;
-        for (Cell val : entry.getEdit().getCells()) {
-          assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
-            val.getRowLength()));
-          String value = i + "" + idx;
-          assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
-          idx++;
-        }
-      }
-      WAL.Entry entry = reader.next();
-      assertNull(entry);
-    } finally {
-      if (writer != null) {
-        writer.close();
-      }
-      if (reader != null) {
-        reader.close();
+    fs.mkdirs(dir);
+    try (WALProvider.Writer writer = createWriter(path)) {
+      ProtobufLogTestHelper.doWrite(writer, withTrailer, tableName, columnCount, recordCount, row,
+        timestamp);
+      try (ProtobufLogReader reader = (ProtobufLogReader) wals.createReader(fs, path)) {
+        ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row,
+          timestamp);
       }
     }
   }
 
-  protected abstract W createWriter(Path path) throws IOException;
-
-  protected abstract void append(W writer, WAL.Entry entry) throws IOException;
-
-  protected abstract void sync(W writer) throws IOException;
+  protected abstract WALProvider.Writer createWriter(Path path) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
new file mode 100644
index 0000000..ecd8e6c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.stream.IntStream;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hadoop.hbase.wal.WALProvider;
+
+/**
+ * Helper class for testing protobuf log.
+ */
+final class ProtobufLogTestHelper {
+
+  private ProtobufLogTestHelper() {
+  }
+
+  private static byte[] toValue(int prefix, int suffix) {
+    return Bytes.toBytes(prefix + "-" + suffix);
+  }
+
+  private static RegionInfo toRegionInfo(TableName tableName) {
+    return RegionInfoBuilder.newBuilder(tableName).setRegionId(1024).build();
+  }
+
+  public static void doWrite(WALProvider.Writer writer, boolean withTrailer, TableName tableName,
+      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+    RegionInfo hri = toRegionInfo(tableName);
+    for (int i = 0; i < recordCount; i++) {
+      WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, i, timestamp,
+          HConstants.DEFAULT_CLUSTER_ID);
+      WALEdit edit = new WALEdit();
+      int prefix = i;
+      IntStream.range(0, columnCount).mapToObj(j -> toValue(prefix, j))
+          .map(value -> new KeyValue(row, row, row, timestamp, value)).forEachOrdered(edit::add);
+      writer.append(new WAL.Entry(key, edit));
+    }
+    writer.sync();
+    if (withTrailer) {
+      writer.close();
+    }
+  }
+
+  public static void doRead(ProtobufLogReader reader, boolean withTrailer, TableName tableName,
+      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+    if (withTrailer) {
+      assertNotNull(reader.trailer);
+    } else {
+      assertNull(reader.trailer);
+    }
+    RegionInfo hri = toRegionInfo(tableName);
+    for (int i = 0; i < recordCount; ++i) {
+      WAL.Entry entry = reader.next();
+      assertNotNull(entry);
+      assertEquals(columnCount, entry.getEdit().size());
+      assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
+      assertEquals(tableName, entry.getKey().getTableName());
+      int idx = 0;
+      for (Cell val : entry.getEdit().getCells()) {
+        assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
+          val.getRowLength()));
+        assertArrayEquals(toValue(i, idx), CellUtil.cloneValue(val));
+        idx++;
+      }
+    }
+    assertNull(reader.next());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
index 0ea75b6..7626dcf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
@@ -18,29 +18,24 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALProvider;
-import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
 
 @Category({ RegionServerTests.class, MediumTests.class })
-public class TestAsyncProtobufLog extends AbstractTestProtobufLog<WALProvider.AsyncWriter> {
+public class TestAsyncProtobufLog extends AbstractTestProtobufLog {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -64,25 +59,8 @@ public class TestAsyncProtobufLog extends AbstractTestProtobufLog<WALProvider.As
   }
 
   @Override
-  protected AsyncWriter createWriter(Path path) throws IOException {
-    return AsyncFSWALProvider.createAsyncWriter(TEST_UTIL.getConfiguration(), fs, path, false,
-      EVENT_LOOP_GROUP.next(), CHANNEL_CLASS);
-  }
-
-  @Override
-  protected void append(AsyncWriter writer, Entry entry) throws IOException {
-    writer.append(entry);
-  }
-
-  @Override
-  protected void sync(AsyncWriter writer) throws IOException {
-    try {
-      writer.sync().get();
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException();
-    } catch (ExecutionException e) {
-      Throwables.propagateIfPossible(e.getCause());
-      throw new IOException(e.getCause());
-    }
+  protected Writer createWriter(Path path) throws IOException {
+    return new WriterOverAsyncWriter(AsyncFSWALProvider.createAsyncWriter(
+      TEST_UTIL.getConfiguration(), fs, path, false, EVENT_LOOP_GROUP.next(), CHANNEL_CLASS));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
new file mode 100644
index 0000000..d74f9d8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.wal;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider.AsyncWriter;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
+
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestCombinedAsyncWriter {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static EventLoopGroup EVENT_LOOP_GROUP;
+
+  private static Class<? extends Channel> CHANNEL_CLASS;
+
+  private static WALFactory WALS;
+
+  @Rule
+  public final TestName name = new TestName();
+
+  @Parameter
+  public CombinedAsyncWriter.Mode mode;
+
+  @Parameters(name = "{index}: mode={0}")
+  public static List<Object[]> params() {
+    return Arrays.asList(new Object[] { CombinedAsyncWriter.Mode.SEQUENTIAL },
+      new Object[] { CombinedAsyncWriter.Mode.PARALLEL });
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    EVENT_LOOP_GROUP = new NioEventLoopGroup();
+    CHANNEL_CLASS = NioSocketChannel.class;
+    UTIL.startMiniDFSCluster(3);
+    UTIL.getTestFileSystem().mkdirs(UTIL.getDataTestDirOnTestFS());
+    WALS =
+      new WALFactory(UTIL.getConfiguration(), null, TestCombinedAsyncWriter.class.getSimpleName());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (WALS != null) {
+      WALS.close();
+    }
+    EVENT_LOOP_GROUP.shutdownGracefully().syncUninterruptibly();
+    UTIL.shutdownMiniDFSCluster();
+  }
+
+  @Test
+  public void testWithTrailer() throws IOException {
+    doTest(true);
+  }
+
+  @Test
+  public void testWithoutTrailer() throws IOException {
+    doTest(false);
+  }
+
+  private Path getPath(int index) throws IOException {
+    String methodName = name.getMethodName().replaceAll("[^A-Za-z0-9_-]", "_");
+    return new Path(UTIL.getDataTestDirOnTestFS(), methodName + "-" + index);
+  }
+
+  private void doTest(boolean withTrailer) throws IOException {
+    int columnCount = 5;
+    int recordCount = 5;
+    TableName tableName = TableName.valueOf("tablename");
+    byte[] row = Bytes.toBytes("row");
+    long timestamp = System.currentTimeMillis();
+    Path path1 = getPath(1);
+    Path path2 = getPath(2);
+    FileSystem fs = UTIL.getTestFileSystem();
+    Configuration conf = UTIL.getConfiguration();
+    try (
+      AsyncWriter writer1 = AsyncFSWALProvider.createAsyncWriter(conf, fs, path1, false,
+        EVENT_LOOP_GROUP.next(), CHANNEL_CLASS);
+      AsyncWriter writer2 = AsyncFSWALProvider.createAsyncWriter(conf, fs, path2, false,
+        EVENT_LOOP_GROUP.next(), CHANNEL_CLASS);
+      CombinedAsyncWriter writer = CombinedAsyncWriter.create(mode, writer1, writer2)) {
+      ProtobufLogTestHelper.doWrite(new WriterOverAsyncWriter(writer), withTrailer, tableName,
+        columnCount, recordCount, row, timestamp);
+      try (ProtobufLogReader reader = (ProtobufLogReader) WALS.createReader(fs, path1)) {
+        ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row,
+          timestamp);
+      }
+      try (ProtobufLogReader reader = (ProtobufLogReader) WALS.createReader(fs, path2)) {
+        ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row,
+          timestamp);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
index 080b5be..d429a01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
@@ -23,14 +23,12 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.wal.FSHLogProvider;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
 @Category({ RegionServerTests.class, MediumTests.class })
-public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer> {
+public class TestProtobufLog extends AbstractTestProtobufLog {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -40,14 +38,4 @@ public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer>
   protected Writer createWriter(Path path) throws IOException {
     return FSHLogProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false);
   }
-
-  @Override
-  protected void append(Writer writer, Entry entry) throws IOException {
-    writer.append(entry);
-  }
-
-  @Override
-  protected void sync(Writer writer) throws IOException {
-    writer.sync();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f260d06/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java
new file mode 100644
index 0000000..da70ee5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.wal;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALProvider;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+class WriterOverAsyncWriter implements WALProvider.Writer {
+
+  private final WALProvider.AsyncWriter asyncWriter;
+
+  public WriterOverAsyncWriter(AsyncWriter asyncWriter) {
+    this.asyncWriter = asyncWriter;
+  }
+
+  @Override
+  public void close() throws IOException {
+    asyncWriter.close();
+  }
+
+  @Override
+  public long getLength() {
+    return asyncWriter.getLength();
+  }
+
+  @Override
+  public void sync() throws IOException {
+    try {
+      asyncWriter.sync().get();
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    } catch (ExecutionException e) {
+      Throwables.propagateIfPossible(e.getCause(), IOException.class);
+      throw new IOException(e.getCause());
+    }
+  }
+
+  @Override
+  public void append(Entry entry) throws IOException {
+    asyncWriter.append(entry);
+  }
+}


[04/18] hbase git commit: HBASE-19886 Display maintenance mode in shell, web UI

Posted by zh...@apache.org.
HBASE-19886 Display maintenance mode in shell, web UI


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

Branch: refs/heads/HBASE-19064
Commit: 380083e7d3d9864cacd41a4297914d6789618d78
Parents: b5ccfec
Author: Balazs Meszaros <ba...@cloudera.com>
Authored: Mon Jan 29 16:43:20 2018 +0100
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Tue Feb 6 14:53:32 2018 -0800

----------------------------------------------------------------------
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |  6 ++++
 hbase-shell/src/main/ruby/hbase/admin.rb        |  7 ++++
 hbase-shell/src/main/ruby/shell.rb              |  1 +
 .../shell/commands/is_in_maintenance_mode.rb    | 38 ++++++++++++++++++++
 4 files changed, 52 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/380083e7/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index 3253a57..12e0a69b 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -185,6 +185,12 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
           re-enabled from the hbase shell by running the command 'catalogjanitor_switch true'
           </div>
         </%if>
+        <%if master.isInMaintenanceMode() %>
+          <div class="alert alert-warning">
+          Your Master is in maintenance mode. This may be because of HBCK aborting while
+          running in repair mode. Please re-run HBCK in repair mode.
+          </div>
+        </%if>
         <%if !master.isBalancerOn() %>
           <div class="alert alert-warning">
             The Load Balancer is not enabled which will eventually cause performance degradation

http://git-wip-us.apache.org/repos/asf/hbase/blob/380083e7/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 13b6578..0102118 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -224,6 +224,13 @@ module Hbase
     end
 
     #----------------------------------------------------------------------------------------------
+    # Query the current state of master in maintenance mode.
+    # Returns the state of maintenance mode (true is on).
+    def in_maintenance_mode?
+      @admin.isMasterInMaintenanceMode
+    end
+
+    #----------------------------------------------------------------------------------------------
     # Request a scan of the catalog table (for garbage collection)
     # Returns an int signifying the number of entries cleaned
     def catalogjanitor_run

http://git-wip-us.apache.org/repos/asf/hbase/blob/380083e7/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 4a74646..507c0a9 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -336,6 +336,7 @@ Shell.load_command_group(
     normalize
     normalizer_switch
     normalizer_enabled
+    is_in_maintenance_mode
     close_region
     compact
     flush

http://git-wip-us.apache.org/repos/asf/hbase/blob/380083e7/hbase-shell/src/main/ruby/shell/commands/is_in_maintenance_mode.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/is_in_maintenance_mode.rb b/hbase-shell/src/main/ruby/shell/commands/is_in_maintenance_mode.rb
new file mode 100644
index 0000000..8ed244c
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/is_in_maintenance_mode.rb
@@ -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.
+#
+
+module Shell
+  module Commands
+    class IsInMaintenanceMode < Command
+      def help
+        <<-EOF
+Is master in maintenance mode? For example:
+
+  hbase> is_in_maintenance_mode
+EOF
+      end
+
+      def command
+        state = admin.in_maintenance_mode?
+        formatter.row([state.to_s])
+        state
+      end
+    end
+  end
+end


[15/18] hbase git commit: HBASE-19864 Use protobuf instead of enum.ordinal to store SyncReplicationState

Posted by zh...@apache.org.
HBASE-19864 Use protobuf instead of enum.ordinal to store SyncReplicationState

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

Branch: refs/heads/HBASE-19064
Commit: 00d450e5ebe71fb14c82851160d2d039d67aa8f2
Parents: 72954cb
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 26 16:50:48 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 8 16:49:19 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfigUtil.java  | 22 ++++++++++++++---
 .../hbase/replication/SyncReplicationState.java | 17 +++++++++++++
 .../hbase/shaded/protobuf/RequestConverter.java |  7 +++---
 .../src/main/protobuf/Replication.proto         | 13 ++++++----
 .../replication/ZKReplicationPeerStorage.java   | 25 +++++++++-----------
 .../hadoop/hbase/master/MasterRpcServices.java  |  9 ++++---
 ...ransitPeerSyncReplicationStateProcedure.java |  9 ++++---
 .../TestReplicationSourceManager.java           |  2 +-
 8 files changed, 67 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/00d450e5/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 86b49ea..5096824 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
@@ -398,7 +398,7 @@ public final class ReplicationPeerConfigUtil {
         ReplicationProtos.ReplicationState.State.ENABLED == desc.getState().getState();
     ReplicationPeerConfig config = convert(desc.getConfig());
     return new ReplicationPeerDescription(desc.getId(), enabled, config,
-        SyncReplicationState.valueOf(desc.getSyncReplicationState().getNumber()));
+      toSyncReplicationState(desc.getSyncReplicationState()));
   }
 
   public static ReplicationProtos.ReplicationPeerDescription
@@ -406,17 +406,33 @@ public final class ReplicationPeerConfigUtil {
     ReplicationProtos.ReplicationPeerDescription.Builder builder =
         ReplicationProtos.ReplicationPeerDescription.newBuilder();
     builder.setId(desc.getPeerId());
+
     ReplicationProtos.ReplicationState.Builder stateBuilder =
         ReplicationProtos.ReplicationState.newBuilder();
     stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED :
         ReplicationProtos.ReplicationState.State.DISABLED);
     builder.setState(stateBuilder.build());
+
     builder.setConfig(convert(desc.getPeerConfig()));
-    builder.setSyncReplicationState(
-      ReplicationProtos.SyncReplicationState.forNumber(desc.getSyncReplicationState().ordinal()));
+    builder.setSyncReplicationState(toSyncReplicationState(desc.getSyncReplicationState()));
+
     return builder.build();
   }
 
+  public static ReplicationProtos.SyncReplicationState
+      toSyncReplicationState(SyncReplicationState state) {
+    ReplicationProtos.SyncReplicationState.Builder syncReplicationStateBuilder =
+        ReplicationProtos.SyncReplicationState.newBuilder();
+    syncReplicationStateBuilder
+        .setState(ReplicationProtos.SyncReplicationState.State.forNumber(state.ordinal()));
+    return syncReplicationStateBuilder.build();
+  }
+
+  public static SyncReplicationState
+      toSyncReplicationState(ReplicationProtos.SyncReplicationState state) {
+    return SyncReplicationState.valueOf(state.getState().getNumber());
+  }
+
   public static ReplicationPeerConfig appendTableCFsToReplicationPeerConfig(
       Map<TableName, List<String>> tableCfs, ReplicationPeerConfig peerConfig) {
     ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(peerConfig);

http://git-wip-us.apache.org/repos/asf/hbase/blob/00d450e5/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
index bd144e9..a65b144 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
@@ -17,8 +17,15 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.util.Arrays;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
 /**
  * Used by synchronous replication. Indicate the state of the current cluster in a synchronous
  * replication peer. The state may be one of {@link SyncReplicationState#ACTIVE},
@@ -45,4 +52,14 @@ public enum SyncReplicationState {
         throw new IllegalArgumentException("Unknown synchronous replication state " + value);
     }
   }
+
+  public static byte[] toByteArray(SyncReplicationState state) {
+    return ProtobufUtil
+        .prependPBMagic(ReplicationPeerConfigUtil.toSyncReplicationState(state).toByteArray());
+  }
+
+  public static SyncReplicationState parseFrom(byte[] bytes) throws InvalidProtocolBufferException {
+    return ReplicationPeerConfigUtil.toSyncReplicationState(ReplicationProtos.SyncReplicationState
+        .parseFrom(Arrays.copyOfRange(bytes, ProtobufUtil.lengthOfPBMagic(), bytes.length)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/00d450e5/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 659be2a..6ebcc89 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -1878,10 +1878,9 @@ public final class RequestConverter {
   }
 
   public static TransitReplicationPeerSyncReplicationStateRequest
-    buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
-      SyncReplicationState state) {
+      buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
+          SyncReplicationState state) {
     return TransitReplicationPeerSyncReplicationStateRequest.newBuilder().setPeerId(peerId)
-      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
-      .build();
+        .setSyncReplicationState(ReplicationPeerConfigUtil.toSyncReplicationState(state)).build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/00d450e5/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 de7b742..82a242d 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -65,11 +65,14 @@ message ReplicationState {
 /**
  * Indicate the state of the current cluster in a synchronous replication peer.
  */
-enum SyncReplicationState {
-  NONE = 0;
-  ACTIVE = 1;
-  DOWNGRADE_ACTIVE = 2;
-  STANDBY = 3;
+message SyncReplicationState {
+  enum State {
+    NONE = 0;
+    ACTIVE = 1;
+    DOWNGRADE_ACTIVE = 2;
+    STANDBY = 3;
+  }
+  required State state = 1;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/00d450e5/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 338ce3f..909daa0 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
@@ -17,12 +17,12 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.io.IOException;
 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;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
@@ -31,7 +31,6 @@ 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;
 
 /**
@@ -96,7 +95,7 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
       ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
         enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES),
       ZKUtilOp.createAndFailSilent(getSyncReplicationStateNode(peerId),
-        Bytes.toBytes(syncReplicationState.ordinal())));
+        SyncReplicationState.toByteArray(syncReplicationState)));
     try {
       ZKUtil.createWithParents(zookeeper, peersZNode);
       ZKUtil.multiOrSequential(zookeeper, multiOps, false);
@@ -179,29 +178,27 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
   }
 
   @Override
-  public void setPeerSyncReplicationState(String peerId, SyncReplicationState clusterState)
+  public void setPeerSyncReplicationState(String peerId, SyncReplicationState state)
       throws ReplicationException {
-    byte[] clusterStateBytes = Bytes.toBytes(clusterState.ordinal());
     try {
-      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId), clusterStateBytes);
+      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId),
+        SyncReplicationState.toByteArray(state));
     } catch (KeeperException e) {
       throw new ReplicationException(
-          "Unable to change the cluster state for the synchronous replication peer with id=" +
-              peerId,
-          e);
+        "Unable to change the cluster state for the synchronous replication peer with id=" + peerId,
+        e);
     }
   }
 
   @Override
   public SyncReplicationState getPeerSyncReplicationState(String peerId)
       throws ReplicationException {
-    byte[] data;
     try {
-      data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
-    } catch (KeeperException | InterruptedException e) {
+      byte[] data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
+      return SyncReplicationState.parseFrom(data);
+    } catch (KeeperException | InterruptedException | IOException e) {
       throw new ReplicationException(
-          "Error getting cluster state for the synchronous replication peer with id=" + peerId, e);
+        "Error getting cluster state for the synchronous replication peer with id=" + peerId, e);
     }
-    return SyncReplicationState.valueOf(Bytes.toInt(data));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/00d450e5/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 f5bf117..e19d6ee 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
@@ -83,7 +83,6 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
@@ -1962,13 +1961,13 @@ public class MasterRpcServices extends RSRpcServices
 
   @Override
   public TransitReplicationPeerSyncReplicationStateResponse
-    transitReplicationPeerSyncReplicationState(RpcController controller,
-      TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+      transitReplicationPeerSyncReplicationState(RpcController controller,
+          TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
     try {
       long procId = master.transitReplicationPeerSyncReplicationState(request.getPeerId(),
-        SyncReplicationState.valueOf(request.getSyncReplicationState().getNumber()));
+        ReplicationPeerConfigUtil.toSyncReplicationState(request.getSyncReplicationState()));
       return TransitReplicationPeerSyncReplicationStateResponse.newBuilder().setProcId(procId)
-        .build();
+          .build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/00d450e5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
index d26eecc..aad3b06 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
@@ -19,6 +19,7 @@ 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;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@@ -33,7 +34,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TransitPeerSyncReplicationStateStateData;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 
 /**
  * The procedure for transit current cluster state for a synchronous replication peer.
@@ -89,16 +89,15 @@ public class TransitPeerSyncReplicationStateProcedure extends ModifyPeerProcedur
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
     serializer.serialize(TransitPeerSyncReplicationStateStateData.newBuilder()
-      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
-      .build());
+        .setSyncReplicationState(ReplicationPeerConfigUtil.toSyncReplicationState(state)).build());
   }
 
   @Override
   protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.deserializeStateData(serializer);
     TransitPeerSyncReplicationStateStateData data =
-      serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
-    state = SyncReplicationState.valueOf(data.getSyncReplicationState().getNumber());
+        serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
+    state = ReplicationPeerConfigUtil.toSyncReplicationState(data.getSyncReplicationState());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/00d450e5/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 1204c9b..3d7cdaf 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
@@ -174,7 +174,7 @@ public abstract class TestReplicationSourceManager {
       ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/sync-rep-state");
     ZKUtil.setData(zkw, "/hbase/replication/peers/1/sync-rep-state",
-      Bytes.toBytes(SyncReplicationState.NONE.ordinal()));
+      SyncReplicationState.toByteArray(SyncReplicationState.NONE));
     ZKUtil.createWithParents(zkw, "/hbase/replication/state");
     ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
 


[09/18] hbase git commit: HBASE-19947 ITU should overwrite HTU local FS

Posted by zh...@apache.org.
HBASE-19947 ITU should overwrite HTU local FS


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

Branch: refs/heads/HBASE-19064
Commit: f5dbdf0dab731a986d9aea2ad3dfdb400f1ba46c
Parents: d959407
Author: Mike Drob <md...@apache.org>
Authored: Tue Feb 6 14:08:23 2018 -0600
Committer: Mike Drob <md...@apache.org>
Committed: Wed Feb 7 16:53:54 2018 -0600

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/IntegrationTestingUtility.java   | 1 +
 .../test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java    | 3 +++
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f5dbdf0d/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java
index afb21e4..a1a81ed 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestingUtility.java
@@ -132,6 +132,7 @@ public class IntegrationTestingUtility extends HBaseTestingUtility {
 
   public void createDistributedHBaseCluster() throws IOException {
     Configuration conf = getConfiguration();
+    conf.set("fs.defaultFS", conf.get("original.defaultFS"));
     Class<? extends ClusterManager> clusterManagerClass = conf.getClass(HBASE_CLUSTER_MANAGER_CLASS,
       DEFAULT_HBASE_CLUSTER_MANAGER_CLASS, ClusterManager.class);
     ClusterManager clusterManager = ReflectionUtils.newInstance(

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5dbdf0d/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 4f55199..2bdfd2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -346,6 +346,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       }
     }
 
+    // Save this for when setting default file:// breaks things
+    this.conf.set("original.defaultFS", this.conf.get("fs.defaultFS"));
+
     // Every cluster is a local cluster until we start DFS
     // Note that conf could be null, but this.conf will not be
     String dataTestDir = getDataTestDir().toString();


[03/18] hbase git commit: HBASE-19946 More test clean up for local-mode HTU

Posted by zh...@apache.org.
HBASE-19946 More test clean up for local-mode HTU


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

Branch: refs/heads/HBASE-19064
Commit: b5ccfecc96fe13ca1db81b5a3d3640deb176647d
Parents: 001f9cc
Author: Mike Drob <md...@apache.org>
Authored: Tue Feb 6 12:38:46 2018 -0600
Committer: Mike Drob <md...@apache.org>
Committed: Tue Feb 6 15:20:39 2018 -0600

----------------------------------------------------------------------
 .../regionserver/TestPerColumnFamilyFlush.java  |  8 ++---
 .../TestWalAndCompactingMemStoreFlush.java      | 31 +++++++-------------
 2 files changed, 15 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b5ccfecc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
index e9a1a14..8266a88 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
@@ -129,8 +129,8 @@ public class TestPerColumnFamilyFlush {
 
   @Test(timeout = 180000)
   public void testSelectiveFlushWhenEnabled() throws IOException {
-    // Set up the configuration
-    Configuration conf = HBaseConfiguration.create();
+    // Set up the configuration, use new one to not conflict with minicluster in other tests
+    Configuration conf = new HBaseTestingUtility().getConfiguration();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200 * 1024);
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName());
     conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
@@ -271,8 +271,8 @@ public class TestPerColumnFamilyFlush {
 
   @Test(timeout = 180000)
   public void testSelectiveFlushWhenNotEnabled() throws IOException {
-    // Set up the configuration
-    Configuration conf = HBaseConfiguration.create();
+    // Set up the configuration, use new one to not conflict with minicluster in other tests
+    Configuration conf = new HBaseTestingUtility().getConfiguration();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200 * 1024);
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllStoresPolicy.class.getName());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5ccfecc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
index ab56991..8b20b20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -63,6 +64,8 @@ public class TestWalAndCompactingMemStoreFlush {
   public static final byte[] FAMILY2 = FAMILIES[1];
   public static final byte[] FAMILY3 = FAMILIES[2];
 
+  private Configuration conf;
+
   private HRegion initHRegion(String callingMethod, Configuration conf) throws IOException {
     int i=0;
     HTableDescriptor htd = new HTableDescriptor(TABLENAME);
@@ -131,16 +134,19 @@ public class TestWalAndCompactingMemStoreFlush {
       Arrays.equals(r.getFamilyMap(family).get(qf), val));
   }
 
+  @Before
+  public void setup() {
+    conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
+    conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
+        FlushNonSloppyStoresFirstPolicy.class.getName());
+    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
+  }
+
   @Test(timeout = 180000)
   public void testSelectiveFlushWithEager() throws IOException {
-
     // Set up the configuration
-    Configuration conf = HBaseConfiguration.create();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 300 * 1024);
-    conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
-        FlushNonSloppyStoresFirstPolicy.class.getName());
     conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 75 * 1024);
-    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
     // set memstore to do data compaction
     conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
         String.valueOf(MemoryCompactionPolicy.EAGER));
@@ -372,14 +378,10 @@ public class TestWalAndCompactingMemStoreFlush {
   /* Check the same as above but for index-compaction type of compacting memstore */
   @Test(timeout = 180000)
   public void testSelectiveFlushWithIndexCompaction() throws IOException {
-
     /*------------------------------------------------------------------------------*/
     /* SETUP */
     // Set up the configuration
-    Configuration conf = HBaseConfiguration.create();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 300 * 1024);
-    conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
-        FlushNonSloppyStoresFirstPolicy.class.getName());
     conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 75 * 1024);
     conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
     // set memstore to index-compaction
@@ -633,12 +635,8 @@ public class TestWalAndCompactingMemStoreFlush {
   @Test(timeout = 180000)
   public void testSelectiveFlushAndWALinDataCompaction() throws IOException {
     // Set up the configuration
-    Configuration conf = HBaseConfiguration.create();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 300 * 1024);
-    conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
-        FlushNonSloppyStoresFirstPolicy.class.getName());
     conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 75 * 1024);
-    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
     // set memstore to do data compaction and not to use the speculative scan
     conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
         String.valueOf(MemoryCompactionPolicy.EAGER));
@@ -769,10 +767,7 @@ public class TestWalAndCompactingMemStoreFlush {
   @Test(timeout = 180000)
   public void testSelectiveFlushWithBasicAndMerge() throws IOException {
     // Set up the configuration
-    Configuration conf = HBaseConfiguration.create();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 300 * 1024);
-    conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
-        FlushNonSloppyStoresFirstPolicy.class.getName());
     conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 75 * 1024);
     conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.8);
     // set memstore to do index compaction with merge
@@ -915,13 +910,9 @@ public class TestWalAndCompactingMemStoreFlush {
   @Test(timeout = 300000)
   public void testStressFlushAndWALinIndexCompaction() throws IOException {
     // Set up the configuration
-    Configuration conf = HBaseConfiguration.create();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 600 * 1024);
-    conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
-        FlushNonSloppyStoresFirstPolicy.class.getName());
     conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
         200 * 1024);
-    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
     // set memstore to do data compaction and not to use the speculative scan
     conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
         String.valueOf(MemoryCompactionPolicy.BASIC));


[08/18] hbase git commit: HBASE-19949 TestRSGroupsWithACL fails with ExceptionInInitializerError

Posted by zh...@apache.org.
HBASE-19949 TestRSGroupsWithACL fails with ExceptionInInitializerError


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

Branch: refs/heads/HBASE-19064
Commit: d95940776e1411531676fd71e45df056b031683d
Parents: dcbb331
Author: tedyu <yu...@gmail.com>
Authored: Wed Feb 7 04:45:34 2018 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Feb 7 04:45:34 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d9594077/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
index 25bb433..30547ba 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.SecureTestUtil;
 import org.apache.hadoop.hbase.security.access.TableAuthManager;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
@@ -52,7 +53,7 @@ import org.slf4j.LoggerFactory;
  * Performs authorization checks for rsgroup operations, according to different
  * levels of authorized users.
  */
-@Category({SecurityTests.class})
+@Category({SecurityTests.class, MediumTests.class})
 public class TestRSGroupsWithACL extends SecureTestUtil{
 
   @ClassRule


[14/18] hbase git commit: HBASE-19857 Complete the procedure for adding a sync replication peer

Posted by zh...@apache.org.
HBASE-19857 Complete the procedure for adding a sync replication peer


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

Branch: refs/heads/HBASE-19064
Commit: 72954cb10b70ea6bce12d301c515d99b39cc4f07
Parents: 343403d
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 25 20:09:00 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 8 16:49:19 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationPeer.java      |   9 +
 .../hbase/replication/ReplicationPeerImpl.java  |  28 +--
 .../hbase/replication/ReplicationPeers.java     |   3 +-
 .../regionserver/PeerActionListener.java        |  10 +-
 .../SyncReplicationPeerProvider.java            |  35 +++
 .../SynchronousReplicationPeerProvider.java     |  35 ---
 .../hbase/wal/SyncReplicationWALProvider.java   | 234 +++++++++++++++++++
 .../wal/SynchronousReplicationWALProvider.java  | 225 ------------------
 .../org/apache/hadoop/hbase/wal/WALFactory.java |   8 +-
 .../TestReplicationSourceManager.java           |   3 +
 .../wal/TestSyncReplicationWALProvider.java     | 153 ++++++++++++
 .../TestSynchronousReplicationWALProvider.java  | 153 ------------
 12 files changed, 456 insertions(+), 440 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/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 2da3cce..0196a9a 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,15 @@ public interface ReplicationPeer {
   PeerState getPeerState();
 
   /**
+   * Returns the sync replication state of the peer by reading local cache.
+   * <p>
+   * If the peer is not a synchronous replication peer, a {@link SyncReplicationState#NONE} will be
+   * returned.
+   * @return the sync replication state
+   */
+  SyncReplicationState getSyncReplicationState();
+
+  /**
    * Test whether the peer is enabled.
    * @return {@code true} if enabled, otherwise {@code false}.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/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 604e0bb..5ec14cd 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
@@ -36,6 +36,8 @@ public class ReplicationPeerImpl implements ReplicationPeer {
 
   private volatile PeerState peerState;
 
+  private volatile SyncReplicationState syncReplicationState;
+
   private final List<ReplicationPeerConfigListener> peerConfigListeners;
 
   /**
@@ -45,12 +47,13 @@ public class ReplicationPeerImpl implements ReplicationPeer {
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerImpl(Configuration conf, String id, boolean peerState,
-      ReplicationPeerConfig peerConfig) {
+  public ReplicationPeerImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig,
+      boolean peerState, SyncReplicationState syncReplicationState) {
     this.conf = conf;
     this.id = id;
     this.peerState = peerState ? PeerState.ENABLED : PeerState.DISABLED;
     this.peerConfig = peerConfig;
+    this.syncReplicationState = syncReplicationState;
     this.peerConfigListeners = new ArrayList<>();
   }
 
@@ -77,37 +80,26 @@ public class ReplicationPeerImpl implements ReplicationPeer {
     return peerState;
   }
 
-  /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
+  @Override
+  public SyncReplicationState getSyncReplicationState() {
+    return syncReplicationState;
+  }
+
   @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();

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/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 eacb2f4..f120dbc 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
@@ -129,7 +129,8 @@ public class ReplicationPeers {
   private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
     ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
     boolean enabled = peerStorage.isPeerEnabled(peerId);
+    SyncReplicationState syncReplicationState = peerStorage.getPeerSyncReplicationState(peerId);
     return new ReplicationPeerImpl(ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf),
-        peerId, enabled, peerConfig);
+        peerId, peerConfig, enabled, syncReplicationState);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
index 74ad626..6df2af9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
@@ -17,17 +17,19 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Get notification for replication peer events. Mainly used for telling the
- * {@link org.apache.hadoop.hbase.wal.SynchronousReplicationWALProvider} to close some WAL if not
- * used any more.
- * <p>
- * TODO: Also need a synchronous peer state change notification.
+ * {@link org.apache.hadoop.hbase.wal.SyncReplicationWALProvider} to close some WAL if not used any
+ * more.
  */
 @InterfaceAudience.Private
 public interface PeerActionListener {
 
   default void peerRemoved(String peerId) {}
+
+  default void peerSyncReplicationStateChange(String peerId, SyncReplicationState from,
+      SyncReplicationState to) {}
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.java
new file mode 100644
index 0000000..b97bf7e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.Optional;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Get the peer id and remote root dir if the region is synchronously replicated.
+ */
+@InterfaceAudience.Private
+public interface SyncReplicationPeerProvider {
+
+  /**
+   * Return the peer id and remote WAL directory if the region is synchronously replicated.
+   */
+  Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
deleted file mode 100644
index b4e04fb..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication.regionserver;
-
-import java.util.Optional;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Get the peer id and remote root dir if the region is synchronously replicated.
- */
-@InterfaceAudience.Private
-public interface SynchronousReplicationPeerProvider {
-
-  /**
-   * Return the peer id and remote WAL directory if the region is synchronously replicated.
-   */
-  Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
new file mode 100644
index 0000000..bccc842
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.wal;
+
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALArchiveDirectoryName;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALDirectoryName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
+import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerProvider;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.KeyLocker;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Streams;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
+/**
+ * The special {@link WALProvider} for synchronous replication.
+ * <p>
+ * It works like an interceptor, when getting WAL, first it will check if the given region should be
+ * replicated synchronously, if so it will return a special WAL for it, otherwise it will delegate
+ * the request to the normal {@link WALProvider}.
+ */
+@InterfaceAudience.Private
+public class SyncReplicationWALProvider implements WALProvider, PeerActionListener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SyncReplicationWALProvider.class);
+
+  private static final String LOG_SUFFIX = ".syncrep";
+
+  private final WALProvider provider;
+
+  private final SyncReplicationPeerProvider peerProvider;
+
+  private WALFactory factory;
+
+  private Configuration conf;
+
+  private List<WALActionsListener> listeners = new ArrayList<>();
+
+  private EventLoopGroup eventLoopGroup;
+
+  private Class<? extends Channel> channelClass;
+
+  private AtomicBoolean initialized = new AtomicBoolean(false);
+
+  private final ConcurrentMap<String, DualAsyncFSWAL> peerId2WAL = new ConcurrentHashMap<>();
+
+  private final KeyLocker<String> createLock = new KeyLocker<>();
+
+  SyncReplicationWALProvider(WALProvider provider, SyncReplicationPeerProvider peerProvider) {
+    this.provider = provider;
+    this.peerProvider = peerProvider;
+  }
+
+  @Override
+  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
+    if (!initialized.compareAndSet(false, true)) {
+      throw new IllegalStateException("WALProvider.init should only be called once.");
+    }
+    provider.init(factory, conf, providerId);
+    this.conf = conf;
+    this.factory = factory;
+    Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
+        NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
+    eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
+    channelClass = eventLoopGroupAndChannelClass.getSecond();
+  }
+
+  private String getLogPrefix(String peerId) {
+    return factory.factoryId + WAL_FILE_NAME_DELIMITER + peerId;
+  }
+
+  private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException {
+    Path remoteWALDirPath = new Path(remoteWALDir);
+    FileSystem remoteFs = remoteWALDirPath.getFileSystem(conf);
+    return new DualAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), remoteFs,
+        CommonFSUtils.getWALRootDir(conf), new Path(remoteWALDirPath, peerId),
+        getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
+        conf, listeners, true, getLogPrefix(peerId), LOG_SUFFIX, eventLoopGroup, channelClass);
+  }
+
+  private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException {
+    DualAsyncFSWAL wal = peerId2WAL.get(peerId);
+    if (wal != null) {
+      return wal;
+    }
+    Lock lock = createLock.acquireLock(peerId);
+    try {
+      wal = peerId2WAL.get(peerId);
+      if (wal == null) {
+        wal = createWAL(peerId, remoteWALDir);
+        peerId2WAL.put(peerId, wal);
+        wal.init();
+      }
+      return wal;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public WAL getWAL(RegionInfo region) throws IOException {
+    Optional<Pair<String, String>> peerIdAndRemoteWALDir =
+        peerProvider.getPeerIdAndRemoteWALDir(region);
+    if (peerIdAndRemoteWALDir.isPresent()) {
+      Pair<String, String> pair = peerIdAndRemoteWALDir.get();
+      return getWAL(pair.getFirst(), pair.getSecond());
+    } else {
+      return provider.getWAL(region);
+    }
+  }
+
+  private Stream<WAL> getWALStream() {
+    return Streams.concat(peerId2WAL.values().stream(), provider.getWALs().stream());
+  }
+
+  @Override
+  public List<WAL> getWALs() {
+    return getWALStream().collect(Collectors.toList());
+  }
+
+  @Override
+  public void shutdown() throws IOException {
+    // save the last exception and rethrow
+    IOException failure = null;
+    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
+      try {
+        wal.shutdown();
+      } catch (IOException e) {
+        LOG.error("Shutdown WAL failed", e);
+        failure = e;
+      }
+    }
+    provider.shutdown();
+    if (failure != null) {
+      throw failure;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // save the last exception and rethrow
+    IOException failure = null;
+    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
+      try {
+        wal.close();
+      } catch (IOException e) {
+        LOG.error("Close WAL failed", e);
+        failure = e;
+      }
+    }
+    provider.close();
+    if (failure != null) {
+      throw failure;
+    }
+  }
+
+  @Override
+  public long getNumLogFiles() {
+    return peerId2WAL.size() + provider.getNumLogFiles();
+  }
+
+  @Override
+  public long getLogFileSize() {
+    return peerId2WAL.values().stream().mapToLong(DualAsyncFSWAL::getLogFileSize).sum() +
+      provider.getLogFileSize();
+  }
+
+  private void safeClose(WAL wal) {
+    if (wal != null) {
+      try {
+        wal.close();
+      } catch (IOException e) {
+        LOG.error("Close WAL failed", e);
+      }
+    }
+  }
+
+  @Override
+  public void addWALActionsListener(WALActionsListener listener) {
+    listeners.add(listener);
+    provider.addWALActionsListener(listener);
+  }
+
+  @Override
+  public void peerRemoved(String peerId) {
+    safeClose(peerId2WAL.remove(peerId));
+  }
+
+  @Override
+  public void peerSyncReplicationStateChange(String peerId, SyncReplicationState from,
+      SyncReplicationState to) {
+    assert to == SyncReplicationState.DOWNGRADE_ACTIVE;
+    safeClose(peerId2WAL.remove(peerId));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
deleted file mode 100644
index f60599f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.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.wal;
-
-import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
-import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALArchiveDirectoryName;
-import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALDirectoryName;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Lock;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
-import org.apache.hadoop.hbase.replication.regionserver.SynchronousReplicationPeerProvider;
-import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.KeyLocker;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Streams;
-import org.apache.hbase.thirdparty.io.netty.channel.Channel;
-import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-
-/**
- * The special {@link WALProvider} for synchronous replication.
- * <p>
- * It works like an interceptor, when getting WAL, first it will check if the given region should be
- * replicated synchronously, if so it will return a special WAL for it, otherwise it will delegate
- * the request to the normal {@link WALProvider}.
- */
-@InterfaceAudience.Private
-public class SynchronousReplicationWALProvider implements WALProvider, PeerActionListener {
-
-  private static final Logger LOG =
-    LoggerFactory.getLogger(SynchronousReplicationWALProvider.class);
-
-  private static final String LOG_SUFFIX = ".syncrep";
-
-  private final WALProvider provider;
-
-  private final SynchronousReplicationPeerProvider peerProvider;
-
-  private WALFactory factory;
-
-  private Configuration conf;
-
-  private List<WALActionsListener> listeners = new ArrayList<>();
-
-  private EventLoopGroup eventLoopGroup;
-
-  private Class<? extends Channel> channelClass;
-
-  private AtomicBoolean initialized = new AtomicBoolean(false);
-
-  private final ConcurrentMap<String, DualAsyncFSWAL> peerId2WAL = new ConcurrentHashMap<>();
-
-  private final KeyLocker<String> createLock = new KeyLocker<>();
-
-  SynchronousReplicationWALProvider(WALProvider provider,
-      SynchronousReplicationPeerProvider peerProvider) {
-    this.provider = provider;
-    this.peerProvider = peerProvider;
-  }
-
-  @Override
-  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
-    if (!initialized.compareAndSet(false, true)) {
-      throw new IllegalStateException("WALProvider.init should only be called once.");
-    }
-    provider.init(factory, conf, providerId);
-    this.conf = conf;
-    this.factory = factory;
-    Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
-      NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
-    eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
-    channelClass = eventLoopGroupAndChannelClass.getSecond();
-  }
-
-  private String getLogPrefix(String peerId) {
-    return factory.factoryId + WAL_FILE_NAME_DELIMITER + peerId;
-  }
-
-  private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException {
-    Path remoteWALDirPath = new Path(remoteWALDir);
-    FileSystem remoteFs = remoteWALDirPath.getFileSystem(conf);
-    return new DualAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), remoteFs,
-        CommonFSUtils.getWALRootDir(conf), new Path(remoteWALDirPath, peerId),
-        getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
-        conf, listeners, true, getLogPrefix(peerId), LOG_SUFFIX, eventLoopGroup, channelClass);
-  }
-
-  private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException {
-    DualAsyncFSWAL wal = peerId2WAL.get(peerId);
-    if (wal != null) {
-      return wal;
-    }
-    Lock lock = createLock.acquireLock(peerId);
-    try {
-      wal = peerId2WAL.get(peerId);
-      if (wal == null) {
-        wal = createWAL(peerId, remoteWALDir);
-        peerId2WAL.put(peerId, wal);
-        wal.init();
-      }
-      return wal;
-    } finally {
-      lock.unlock();
-    }
-  }
-
-  @Override
-  public WAL getWAL(RegionInfo region) throws IOException {
-    Optional<Pair<String, String>> peerIdAndRemoteWALDir =
-      peerProvider.getPeerIdAndRemoteWALDir(region);
-    if (peerIdAndRemoteWALDir.isPresent()) {
-      Pair<String, String> pair = peerIdAndRemoteWALDir.get();
-      return getWAL(pair.getFirst(), pair.getSecond());
-    } else {
-      return provider.getWAL(region);
-    }
-  }
-
-  private Stream<WAL> getWALStream() {
-    return Streams.concat(peerId2WAL.values().stream(), provider.getWALs().stream());
-  }
-
-  @Override
-  public List<WAL> getWALs() {
-    return getWALStream().collect(Collectors.toList());
-  }
-
-  @Override
-  public void shutdown() throws IOException {
-    // save the last exception and rethrow
-    IOException failure = null;
-    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
-      try {
-        wal.shutdown();
-      } catch (IOException e) {
-        LOG.error("Shutdown WAL failed", e);
-        failure = e;
-      }
-    }
-    provider.shutdown();
-    if (failure != null) {
-      throw failure;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    // save the last exception and rethrow
-    IOException failure = null;
-    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
-      try {
-        wal.close();
-      } catch (IOException e) {
-        LOG.error("Close WAL failed", e);
-        failure = e;
-      }
-    }
-    provider.close();
-    if (failure != null) {
-      throw failure;
-    }
-  }
-
-  @Override
-  public long getNumLogFiles() {
-    return peerId2WAL.size() + provider.getNumLogFiles();
-  }
-
-  @Override
-  public long getLogFileSize() {
-    return peerId2WAL.values().stream().mapToLong(DualAsyncFSWAL::getLogFileSize).sum() +
-      provider.getLogFileSize();
-  }
-
-  @Override
-  public void peerRemoved(String peerId) {
-    WAL wal = peerId2WAL.remove(peerId);
-    if (wal != null) {
-      try {
-        wal.close();
-      } catch (IOException e) {
-        LOG.error("Close WAL failed", e);
-      }
-    }
-  }
-
-  @Override
-  public void addWALActionsListener(WALActionsListener listener) {
-    listeners.add(listener);
-    provider.addWALActionsListener(listener);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 4e519ee..06999ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
-import org.apache.hadoop.hbase.replication.regionserver.SynchronousReplicationPeerProvider;
+import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerProvider;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -186,7 +186,7 @@ public class WALFactory {
    * Remove it once we can integrate the synchronous replication logic in RS.
    */
   @VisibleForTesting
-  WALFactory(Configuration conf, String factoryId, SynchronousReplicationPeerProvider peerProvider)
+  WALFactory(Configuration conf, String factoryId, SyncReplicationPeerProvider peerProvider)
       throws IOException {
     timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000);
     /* TODO Both of these are probably specific to the fs wal provider */
@@ -195,9 +195,9 @@ public class WALFactory {
     this.conf = conf;
     this.factoryId = factoryId;
     WALProvider provider = createProvider(getProviderClass(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
-    this.provider = new SynchronousReplicationWALProvider(provider, peerProvider);
-    this.provider.addWALActionsListener(new MetricsWAL());
+    this.provider = new SyncReplicationWALProvider(provider, peerProvider);
     this.provider.init(this, conf, null);
+    this.provider.addWALActionsListener(new MetricsWAL());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/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 db62b49..1204c9b 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
@@ -172,6 +172,9 @@ public abstract class TestReplicationSourceManager {
     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state");
     ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state",
       ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
+    ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/sync-rep-state");
+    ZKUtil.setData(zkw, "/hbase/replication/peers/1/sync-rep-state",
+      Bytes.toBytes(SyncReplicationState.NONE.ordinal()));
     ZKUtil.createWithParents(zkw, "/hbase/replication/state");
     ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
new file mode 100644
index 0000000..60a9e13
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.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.wal;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogTestHelper;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestSyncReplicationWALProvider {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID = "1";
+
+  private static String REMOTE_WAL_DIR = "/RemoteWAL";
+
+  private static TableName TABLE = TableName.valueOf("table");
+
+  private static TableName TABLE_NO_REP = TableName.valueOf("table-no-rep");
+
+  private static RegionInfo REGION = RegionInfoBuilder.newBuilder(TABLE).build();
+
+  private static RegionInfo REGION_NO_REP = RegionInfoBuilder.newBuilder(TABLE_NO_REP).build();
+
+  private static WALFactory FACTORY;
+
+  private static Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
+    if (info.getTable().equals(TABLE)) {
+      return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR));
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.startMiniDFSCluster(3);
+    FACTORY = new WALFactory(UTIL.getConfiguration(), "test",
+        TestSyncReplicationWALProvider::getPeerIdAndRemoteWALDir);
+    UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID));
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws IOException {
+    FACTORY.close();
+    UTIL.shutdownMiniDFSCluster();
+  }
+
+  private void testReadWrite(DualAsyncFSWAL wal) throws Exception {
+    int recordCount = 100;
+    int columnCount = 10;
+    byte[] row = Bytes.toBytes("testRow");
+    long timestamp = System.currentTimeMillis();
+    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
+    ProtobufLogTestHelper.doWrite(wal, REGION, TABLE, columnCount, recordCount, row, timestamp,
+      mvcc);
+    Path localFile = wal.getCurrentFileName();
+    Path remoteFile = new Path(REMOTE_WAL_DIR + "/" + PEER_ID, localFile.getName());
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
+      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
+      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    wal.rollWriter();
+    DistributedFileSystem dfs = (DistributedFileSystem) UTIL.getDFSCluster().getFileSystem();
+    UTIL.waitFor(5000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return dfs.isFileClosed(localFile) && dfs.isFileClosed(remoteFile);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        StringBuilder sb = new StringBuilder();
+        if (!dfs.isFileClosed(localFile)) {
+          sb.append(localFile + " has not been closed yet.");
+        }
+        if (!dfs.isFileClosed(remoteFile)) {
+          sb.append(remoteFile + " has not been closed yet.");
+        }
+        return sb.toString();
+      }
+    });
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
+      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
+      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    WAL walNoRep = FACTORY.getWAL(REGION_NO_REP);
+    assertThat(walNoRep, not(instanceOf(DualAsyncFSWAL.class)));
+    DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION);
+    assertEquals(2, FACTORY.getWALs().size());
+    testReadWrite(wal);
+    SyncReplicationWALProvider walProvider =
+      (SyncReplicationWALProvider) FACTORY.getWALProvider();
+    walProvider.peerRemoved(PEER_ID);
+    assertEquals(1, FACTORY.getWALs().size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72954cb1/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
deleted file mode 100644
index e6031c6..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.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.wal;
-
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-import java.io.IOException;
-import java.util.Optional;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
-import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
-import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogTestHelper;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ RegionServerTests.class, MediumTests.class })
-public class TestSynchronousReplicationWALProvider {
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static String PEER_ID = "1";
-
-  private static String REMOTE_WAL_DIR = "/RemoteWAL";
-
-  private static TableName TABLE = TableName.valueOf("table");
-
-  private static TableName TABLE_NO_REP = TableName.valueOf("table-no-rep");
-
-  private static RegionInfo REGION = RegionInfoBuilder.newBuilder(TABLE).build();
-
-  private static RegionInfo REGION_NO_REP = RegionInfoBuilder.newBuilder(TABLE_NO_REP).build();
-
-  private static WALFactory FACTORY;
-
-  private static Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
-    if (info.getTable().equals(TABLE)) {
-      return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR));
-    } else {
-      return Optional.empty();
-    }
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    UTIL.startMiniDFSCluster(3);
-    FACTORY = new WALFactory(UTIL.getConfiguration(), "test",
-        TestSynchronousReplicationWALProvider::getPeerIdAndRemoteWALDir);
-    UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID));
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws IOException {
-    FACTORY.close();
-    UTIL.shutdownMiniDFSCluster();
-  }
-
-  private void testReadWrite(DualAsyncFSWAL wal) throws Exception {
-    int recordCount = 100;
-    int columnCount = 10;
-    byte[] row = Bytes.toBytes("testRow");
-    long timestamp = System.currentTimeMillis();
-    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
-    ProtobufLogTestHelper.doWrite(wal, REGION, TABLE, columnCount, recordCount, row, timestamp,
-      mvcc);
-    Path localFile = wal.getCurrentFileName();
-    Path remoteFile = new Path(REMOTE_WAL_DIR + "/" + PEER_ID, localFile.getName());
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
-      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
-      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    wal.rollWriter();
-    DistributedFileSystem dfs = (DistributedFileSystem) UTIL.getDFSCluster().getFileSystem();
-    UTIL.waitFor(5000, new ExplainingPredicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return dfs.isFileClosed(localFile) && dfs.isFileClosed(remoteFile);
-      }
-
-      @Override
-      public String explainFailure() throws Exception {
-        StringBuilder sb = new StringBuilder();
-        if (!dfs.isFileClosed(localFile)) {
-          sb.append(localFile + " has not been closed yet.");
-        }
-        if (!dfs.isFileClosed(remoteFile)) {
-          sb.append(remoteFile + " has not been closed yet.");
-        }
-        return sb.toString();
-      }
-    });
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
-      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
-      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    WAL walNoRep = FACTORY.getWAL(REGION_NO_REP);
-    assertThat(walNoRep, not(instanceOf(DualAsyncFSWAL.class)));
-    DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION);
-    assertEquals(2, FACTORY.getWALs().size());
-    testReadWrite(wal);
-    SynchronousReplicationWALProvider walProvider =
-      (SynchronousReplicationWALProvider) FACTORY.getWALProvider();
-    walProvider.peerRemoved(PEER_ID);
-    assertEquals(1, FACTORY.getWALs().size());
-  }
-}


[17/18] hbase git commit: HBASE-19781 Add a new cluster state flag for synchronous replication

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/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 8911982..f5eca39 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
@@ -28,6 +28,7 @@ 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.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
@@ -67,9 +68,9 @@ public class TestHBaseFsckReplication {
     String peerId1 = "1";
     String peerId2 = "2";
     peerStorage.addPeer(peerId1, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(),
-      true);
+      true, SyncReplicationState.NONE);
     peerStorage.addPeer(peerId2, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(),
-      true);
+      true, SyncReplicationState.NONE);
     for (int i = 0; i < 10; i++) {
       queueStorage.addWAL(ServerName.valueOf("localhost", 10000 + i, 100000 + i), peerId1,
         "file-" + i);

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index ba7d191..d5d4844 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -20,6 +20,7 @@
 include Java
 
 java_import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil
+java_import org.apache.hadoop.hbase.replication.SyncReplicationState
 java_import org.apache.hadoop.hbase.replication.ReplicationPeerConfig
 java_import org.apache.hadoop.hbase.util.Bytes
 java_import org.apache.hadoop.hbase.zookeeper.ZKConfig
@@ -329,6 +330,20 @@ module Hbase
       '!' + ReplicationPeerConfigUtil.convertToString(tableCFs)
     end
 
+    # Transit current cluster to a new state in the specified synchronous
+    # replication peer
+    def transit_peer_sync_replication_state(id, state)
+      if 'ACTIVE'.eql?(state)
+        @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::ACTIVE)
+      elsif 'DOWNGRADE_ACTIVE'.eql?(state)
+        @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::DOWNGRADE_ACTIVE)
+      elsif 'STANDBY'.eql?(state)
+        @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::STANDBY)
+      else
+        raise(ArgumentError, 'synchronous replication state must be ACTIVE, DOWNGRADE_ACTIVE or STANDBY')
+      end
+    end
+
     #----------------------------------------------------------------------------------------------
     # Enables a table's replication switch
     def enable_tablerep(table_name)

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 507c0a9..0ed71ae 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -395,6 +395,7 @@ Shell.load_command_group(
     get_peer_config
     list_peer_configs
     update_peer_config
+    transit_peer_sync_replication_state
   ]
 )
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
index caeab86..aa10fda 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
@@ -39,8 +39,8 @@ EOF
         peers = replication_admin.list_peers
 
         formatter.header(%w[PEER_ID CLUSTER_KEY ENDPOINT_CLASSNAME
-                            REMOTE_ROOT_DIR STATE REPLICATE_ALL
-                            NAMESPACES TABLE_CFS BANDWIDTH])
+                            REMOTE_ROOT_DIR SYNC_REPLICATION_STATE STATE
+                            REPLICATE_ALL NAMESPACES TABLE_CFS BANDWIDTH])
 
         peers.each do |peer|
           id = peer.getPeerId
@@ -66,7 +66,7 @@ EOF
             remote_root_dir = config.getRemoteWALDir
           end
           formatter.row([id, cluster_key, endpoint_classname,
-                         remote_root_dir, state,
+                         remote_root_dir, peer.getSyncReplicationState, state,
                          config.replicateAllUserTables, namespaces, tableCFs,
                          config.getBandwidth])
         end

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb b/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb
new file mode 100644
index 0000000..6d4a963
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb
@@ -0,0 +1,44 @@
+#
+# Copyright The Apache Software Foundation
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+module Shell
+  module Commands
+    class TransitPeerSyncReplicationState < Command
+      def help
+        <<-EOF
+Transit current cluster to new state in the specified synchronous replication peer.
+Examples:
+
+  # Transit cluster state to DOWNGRADE_ACTIVE in a synchronous replication peer
+  hbase> transit_peer_sync_replication_state '1', 'DOWNGRADE_ACTIVE'
+  # Transit cluster state to ACTIVE in a synchronous replication peer
+  hbase> transit_peer_sync_replication_state '1', 'ACTIVE'
+  # Transit cluster state to STANDBY in a synchronous replication peer
+  hbase> transit_peer_sync_replication_state '1', 'STANDBY'
+
+EOF
+      end
+
+      def command(id, state)
+        replication_admin.transit_peer_sync_replication_state(id, state)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/343403d2/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
index 7f2b6ae..a758c96 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -23,6 +23,9 @@ require 'hbase/hbase'
 require 'hbase/table'
 
 include HBaseConstants
+include Java
+
+java_import org.apache.hadoop.hbase.replication.SyncReplicationState
 
 module Hbase
   class ReplicationAdminTest < Test::Unit::TestCase
@@ -490,6 +493,27 @@ module Hbase
       command(:remove_peer, @peer_id)
     end
 
+    define_test "transit_peer_sync_replication_state: test" do
+      cluster_key = "server1.cie.com:2181:/hbase"
+      remote_wal_dir = "hdfs://srv1:9999/hbase"
+      args = { CLUSTER_KEY => cluster_key, REMOTE_WAL_DIR => remote_wal_dir }
+      command(:add_peer, @peer_id, args)
+
+      assert_equal(1, command(:list_peers).length)
+      peer = command(:list_peers).get(0)
+      assert_equal(@peer_id, peer.getPeerId)
+      assert_equal(SyncReplicationState::DOWNGRADE_ACTIVE, peer.getSyncReplicationState)
+
+      command(:transit_peer_sync_replication_state, @peer_id, 'ACTIVE')
+      assert_equal(1, command(:list_peers).length)
+      peer = command(:list_peers).get(0)
+      assert_equal(@peer_id, peer.getPeerId)
+      assert_equal(SyncReplicationState::ACTIVE, peer.getSyncReplicationState)
+
+      # cleanup for future tests
+      command(:remove_peer, @peer_id)
+    end
+
     define_test "get_peer_config: works with simple clusterKey peer" do
       cluster_key = "localhost:2181:/hbase-test"
       args = { CLUSTER_KEY => cluster_key }