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/05/31 07:01:59 UTC

[01/36] hbase git commit: HBASE-20444 Improve comparison logic for HBase version strings [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19064 e43000021 -> 1f3e50fce (forced update)


HBASE-20444 Improve comparison logic for HBase version strings

Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: b77fb77b322a3a4f3419ff7cadc2c8dae256fe09
Parents: 9977470
Author: maoling <ma...@sina.com>
Authored: Mon May 21 17:38:10 2018 +0800
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed May 30 09:18:53 2018 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hbase/util/VersionInfo.java   | 64 ++++++++++----------
 .../hadoop/hbase/util/TestVersionInfo.java      | 20 +++++-
 2 files changed, 51 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b77fb77b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
index 86ac065..0f368f3 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.util;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.Version;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -117,50 +118,49 @@ public class VersionInfo {
     if (v1.equals(v2)) {
       return 0;
     }
-
-    Object[] v1Comps = getVersionComponents(v1); //1.2.3-hotfix -> [1, 2, 3, hotfix]
-    Object[] v2Comps = getVersionComponents(v2);
-    int index = 0;
-    while (index < v1Comps.length && index < v2Comps.length) {
-      int va = v1Comps[index] instanceof Integer ? (Integer)v1Comps[index] : VERY_LARGE_NUMBER;
-      int vb = v2Comps[index] instanceof Integer ? (Integer)v2Comps[index] : VERY_LARGE_NUMBER;
-
-      if (va != vb) {
-        return va - vb;
-      }
-      if (va == VERY_LARGE_NUMBER) {
-        // here, va and vb components must be same and Strings, compare as String
-        int c = ((String)v1Comps[index]).compareTo((String)v2Comps[index]);
-        if (c != 0) {
-          return c;
-        }
+    String[] v1Comps = getVersionComponents(v1);
+    String[] v2Comps = getVersionComponents(v2);
+
+    int length = Math.max(v1Comps.length, v2Comps.length);
+    for (int i = 0; i < length; i++) {
+      Integer va = i < v1Comps.length ? Integer.parseInt(v1Comps[i]) : 0;
+      Integer vb = i < v2Comps.length ? Integer.parseInt(v2Comps[i]) : 0;
+      int compare = va.compareTo(vb);
+      if (compare != 0) {
+        return compare;
       }
-      index++;
-    }
-    if (index < v1Comps.length) {
-      // v1 is longer
-      return 1;
     }
-    //v2 is longer
-    return -1;
+    return 0;
   }
 
   /**
-   * Returns the version components as Integer and String objects
-   * Examples: "1.2.3" returns [1, 2, 3], "4.5.6-SNAPSHOT" returns [4, 5, 6, "SNAPSHOT"]
+   * Returns the version components as String objects
+   * Examples: "1.2.3" returns ["1", "2", "3"], "4.5.6-SNAPSHOT" returns ["4", "5", "6", "-1"]
+   * "4.5.6-beta" returns ["4", "5", "6", "-2"], "4.5.6-alpha" returns ["4", "5", "6", "-3"]
+   * "4.5.6-UNKNOW" returns ["4", "5", "6", "-4"]
    * @return the components of the version string
    */
-  static Object[] getVersionComponents(final String version) {
+  static String[] getVersionComponents(final String version) {
     assert(version != null);
-    Object[] strComps = version.split("[\\.-]");
+    String[] strComps = version.split("[\\.-]");
     assert(strComps.length > 0);
 
-    Object[] comps = new Object[strComps.length];
+    String[] comps = new String[strComps.length];
     for (int i = 0; i < strComps.length; ++i) {
-      try {
-        comps[i] = Integer.parseInt((String) strComps[i]);
-      } catch (NumberFormatException e) {
+      if (StringUtils.isNumeric(strComps[i])) {
         comps[i] = strComps[i];
+      } else if (StringUtils.isEmpty(strComps[i])) {
+        comps[i] = String.valueOf(VERY_LARGE_NUMBER);
+      } else {
+        if("SNAPSHOT".equals(strComps[i])) {
+          comps[i] = "-1";
+        } else if("beta".equals(strComps[i])) {
+          comps[i] = "-2";
+        } else if("alpha".equals(strComps[i])) {
+          comps[i] = "-3";
+        } else {
+          comps[i] = "-4";
+        }
       }
     }
     return comps;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b77fb77b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java
index 325ad08..2e638cf 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestVersionInfo.java
@@ -37,6 +37,24 @@ public class TestVersionInfo {
     assertTrue(VersionInfo.compareVersion("1.0.0", "0.98.11") > 0);
     assertTrue(VersionInfo.compareVersion("0.98.11", "1.0.1") < 0);
     assertTrue(VersionInfo.compareVersion("2.0.0", "1.4.0") > 0);
-    assertTrue(VersionInfo.compareVersion("2.0.0", "2.0.0-SNAPSHOT") < 0);
+    assertTrue(VersionInfo.compareVersion("2.0.0", "2.0.0-SNAPSHOT") > 0);
+    assertTrue(VersionInfo.compareVersion("0.94.6.1", "0.96.1.1") < 0);
+    assertTrue(VersionInfo.compareVersion("0.96.1.1", "0.98.6.1") < 0);
+    assertTrue(VersionInfo.compareVersion("0.98.6.1", "0.98.10.1") < 0);
+    assertTrue(VersionInfo.compareVersion("0.98.10.1", "0.98.12.1") < 0);
+    assertTrue(VersionInfo.compareVersion("0.98.12.1", "0.98.16.1") < 0);
+    assertTrue(VersionInfo.compareVersion("0.98.16.1", "1.0.1.1") < 0);
+    assertTrue(VersionInfo.compareVersion("1.0.1.1", "1.1.0.1") < 0);
+    assertTrue(VersionInfo.compareVersion("2.0..1", "2.0.0") > 0);
+    assertTrue(VersionInfo.compareVersion("2.0.0", "2.0.0") == 0);
+    assertTrue(VersionInfo.compareVersion("1.99.14", "2.0.0-alpha-1") < 0);
+    assertTrue(VersionInfo.compareVersion("2.0.0-alpha-1", "2.0.0-beta-3") < 0);
+    assertTrue(VersionInfo.compareVersion("2.0.0-beta-3", "2.0.0-SNAPSHOT") < 0);
+    assertTrue(VersionInfo.compareVersion("2.0.0-SNAPSHOT", "2.0") < 0);
+    assertTrue(VersionInfo.compareVersion("2.0", "2.0.0.1") < 0);
+    assertTrue(VersionInfo.compareVersion("2.0.0.1", "2.0.1") < 0);
+    assertTrue(VersionInfo.compareVersion("3.0.0-alpha-2", "3.0.0-alpha-11") < 0);
+    assertTrue(VersionInfo.compareVersion("3.0.0-beta-2", "3.0.0-beta-11") < 0);
+    assertTrue(VersionInfo.compareVersion("2.0.0-foobar", "2.0.0.1") < 0);
   }
 }


[02/36] hbase git commit: HBASE-20640 Add missing test category and class rule

Posted by zh...@apache.org.
HBASE-20640 Add missing test category and class rule

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


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

Branch: refs/heads/HBASE-19064
Commit: c17be2e6223eee15ec926eae160bfa2adebab260
Parents: b77fb77
Author: Josh Elser <el...@apache.org>
Authored: Thu May 24 13:21:21 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed May 30 10:34:19 2018 -0400

----------------------------------------------------------------------
 .../hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c17be2e6/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java
index ae326c1..5b716f1 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaGlobalsSettingsBypass.java
@@ -22,12 +22,21 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory.QuotaGlobalsSettingsBypass;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.ClassRule;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category({SmallTests.class})
 public class TestQuotaGlobalsSettingsBypass {
 
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestQuotaGlobalsSettingsBypass.class);
+
   @Test
   public void testMerge() throws IOException {
     QuotaGlobalsSettingsBypass orig = new QuotaGlobalsSettingsBypass("joe", null, null, true);


[34/36] hbase git commit: HBASE-20456 Support removing a ReplicationSourceShipper for a special wal group

Posted by zh...@apache.org.
HBASE-20456 Support removing a ReplicationSourceShipper for a special wal group


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

Branch: refs/heads/HBASE-19064
Commit: 863566a7d5bba3c6b36e5aee2ba3b20472c621a9
Parents: bda4720
Author: zhangduo <zh...@apache.org>
Authored: Tue Apr 24 22:01:21 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  1 +
 .../RecoveredReplicationSource.java             | 13 +---
 .../RecoveredReplicationSourceShipper.java      |  7 --
 .../regionserver/ReplicationSource.java         | 13 +++-
 .../regionserver/ReplicationSourceManager.java  | 19 ++++-
 .../regionserver/ReplicationSourceShipper.java  | 20 +++--
 .../ReplicationSourceWALReader.java             |  9 ++-
 .../regionserver/WALEntryStream.java            |  3 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java | 28 ++++---
 .../hbase/wal/SyncReplicationWALProvider.java   | 10 ++-
 .../TestReplicationSourceManager.java           |  5 +-
 .../TestSyncReplicationShipperQuit.java         | 81 ++++++++++++++++++++
 .../regionserver/TestWALEntryStream.java        |  4 +-
 13 files changed, 163 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/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 d98ab75..9b4ce9c 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
@@ -682,6 +682,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   protected void doShutdown() throws IOException {
     waitForSafePoint();
     closeWriter(this.writer);
+    this.writer = null;
     closeExecutor.shutdown();
     try {
       if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index a21ca44..f1bb538 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -144,15 +143,9 @@ public class RecoveredReplicationSource extends ReplicationSource {
   }
 
   void tryFinish() {
-    // use synchronize to make sure one last thread will clean the queue
-    synchronized (workerThreads) {
-      Threads.sleep(100);// wait a short while for other worker thread to fully exit
-      boolean allTasksDone = workerThreads.values().stream().allMatch(w -> w.isFinished());
-      if (allTasksDone) {
-        this.getSourceMetrics().clear();
-        manager.removeRecoveredSource(this);
-        LOG.info("Finished recovering queue {} with the following stats: {}", queueId, getStats());
-      }
+    if (workerThreads.isEmpty()) {
+      this.getSourceMetrics().clear();
+      manager.finishRecoveredSource(this);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
index 91109cf..b0d4db0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
@@ -48,13 +48,6 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
   }
 
   @Override
-  protected void noMoreData() {
-    LOG.debug("Finished recovering queue for group {} of peer {}", walGroupId, source.getQueueId());
-    source.getSourceMetrics().incrCompletedRecoveryQueue();
-    setWorkerState(WorkerState.FINISHED);
-  }
-
-  @Override
   protected void postFinish() {
     source.tryFinish();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index f25a232..ba665b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -62,6 +62,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
@@ -120,6 +121,7 @@ public class ReplicationSource implements ReplicationSourceInterface {
   private long defaultBandwidth;
   private long currentBandwidth;
   private WALFileLengthProvider walFileLengthProvider;
+  @VisibleForTesting
   protected final ConcurrentHashMap<String, ReplicationSourceShipper> workerThreads =
       new ConcurrentHashMap<>();
 
@@ -190,6 +192,9 @@ public class ReplicationSource implements ReplicationSourceInterface {
     PriorityBlockingQueue<Path> queue = queues.get(logPrefix);
     if (queue == null) {
       queue = new PriorityBlockingQueue<>(queueSizePerGroup, new LogsComparator());
+      // make sure that we do not use an empty queue when setting up a ReplicationSource, otherwise
+      // the shipper may quit immediately
+      queue.put(log);
       queues.put(logPrefix, queue);
       if (this.isSourceActive() && this.walEntryFilter != null) {
         // new wal group observed after source startup, start a new worker thread to track it
@@ -197,8 +202,10 @@ public class ReplicationSource implements ReplicationSourceInterface {
         // still not launched, so it's necessary to check workerThreads before start the worker
         tryStartNewShipper(logPrefix, queue);
       }
+    } else {
+      queue.put(log);
     }
-    queue.put(log);
+
     this.metrics.incrSizeOfLogQueue();
     // This will log a warning for each new log that gets created above the warn threshold
     int queueSize = queue.size();
@@ -612,5 +619,7 @@ public class ReplicationSource implements ReplicationSourceInterface {
     return queueStorage;
   }
 
-
+  void removeWorker(ReplicationSourceShipper worker) {
+    workerThreads.remove(worker.walGroupId, worker);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index cbeba23..2d0d82b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -443,12 +443,25 @@ public class ReplicationSourceManager implements ReplicationListener {
    * Clear the metrics and related replication queue of the specified old source
    * @param src source to clear
    */
-  void removeRecoveredSource(ReplicationSourceInterface src) {
-    LOG.info("Done with the recovered queue " + src.getQueueId());
-    this.oldsources.remove(src);
+  private boolean removeRecoveredSource(ReplicationSourceInterface src) {
+    if (!this.oldsources.remove(src)) {
+      return false;
+    }
+    LOG.info("Done with the recovered queue {}", src.getQueueId());
     // Delete queue from storage and memory
     deleteQueue(src.getQueueId());
     this.walsByIdRecoveredQueues.remove(src.getQueueId());
+    return true;
+  }
+
+  void finishRecoveredSource(ReplicationSourceInterface src) {
+    synchronized (oldsources) {
+      if (!removeRecoveredSource(src)) {
+        return;
+      }
+    }
+    LOG.info("Finished recovering queue {} with the following stats: {}", src.getQueueId(),
+      src.getStats());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
index 3f97b5e..b1361fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
@@ -50,13 +50,13 @@ public class ReplicationSourceShipper extends Thread {
   public enum WorkerState {
     RUNNING,
     STOPPED,
-    FINISHED,  // The worker is done processing a recovered queue
+    FINISHED,  // The worker is done processing a queue
   }
 
   private final Configuration conf;
   protected final String walGroupId;
   protected final PriorityBlockingQueue<Path> queue;
-  private final ReplicationSourceInterface source;
+  private final ReplicationSource source;
 
   // Last position in the log that we sent to ZooKeeper
   // It will be accessed by the stats thread so make it volatile
@@ -73,7 +73,7 @@ public class ReplicationSourceShipper extends Thread {
   protected final int maxRetriesMultiplier;
 
   public ReplicationSourceShipper(Configuration conf, String walGroupId,
-      PriorityBlockingQueue<Path> queue, ReplicationSourceInterface source) {
+      PriorityBlockingQueue<Path> queue, ReplicationSource source) {
     this.conf = conf;
     this.walGroupId = walGroupId;
     this.queue = queue;
@@ -98,7 +98,7 @@ public class ReplicationSourceShipper extends Thread {
       }
       try {
         WALEntryBatch entryBatch = entryReader.take();
-        // the NO_MORE_DATA instance has no path so do not all shipEdits
+        // the NO_MORE_DATA instance has no path so do not call shipEdits
         if (entryBatch == WALEntryBatch.NO_MORE_DATA) {
           noMoreData();
         } else {
@@ -113,12 +113,20 @@ public class ReplicationSourceShipper extends Thread {
     if (!isFinished()) {
       setWorkerState(WorkerState.STOPPED);
     } else {
+      source.removeWorker(this);
       postFinish();
     }
   }
 
-  // To be implemented by recovered shipper
-  protected void noMoreData() {
+  private void noMoreData() {
+    if (source.isRecovered()) {
+      LOG.debug("Finished recovering queue for group {} of peer {}", walGroupId,
+        source.getQueueId());
+      source.getSourceMetrics().incrCompletedRecoveryQueue();
+    } else {
+      LOG.debug("Finished queue for group {} of peer {}", walGroupId, source.getQueueId());
+    }
+    setWorkerState(WorkerState.FINISHED);
   }
 
   // To be implemented by recovered shipper

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index 64fd48d..61ab7c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -142,7 +142,7 @@ class ReplicationSourceWALReader extends Thread {
             entryBatchQueue.put(batch);
             sleepMultiplier = 1;
           } else { // got no entries and didn't advance position in WAL
-            handleEmptyWALEntryBatch(entryStream.getCurrentPath());
+            handleEmptyWALEntryBatch();
             entryStream.reset(); // reuse stream
           }
         }
@@ -224,10 +224,11 @@ class ReplicationSourceWALReader extends Thread {
     return batch;
   }
 
-  private void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
+  private void handleEmptyWALEntryBatch() throws InterruptedException {
     LOG.trace("Didn't read any new entries from WAL");
-    if (source.isRecovered()) {
-      // we're done with queue recovery, shut ourself down
+    if (logQueue.isEmpty()) {
+      // we're done with current queue, either this is a recovered queue, or it is the special group
+      // for a sync replication peer and the peer has been transited to DA or S state.
       setReaderRunning(false);
       // shuts down shipper thread immediately
       entryBatchQueue.put(WALEntryBatch.NO_MORE_DATA);

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
index b2c199e..0393af4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
@@ -304,7 +304,8 @@ class WALEntryStream implements Closeable {
         return true;
       }
     } else {
-      // no more files in queue, this could only happen for recovered queue.
+      // no more files in queue, this could happen for recovered queue, or for a wal group of a sync
+      // replication peer which has already been transited to DA or S.
       setCurrentPath(null);
     }
     return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/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 5a3fba3..e528624 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
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
@@ -247,26 +248,30 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
     if (walName == null) {
       throw new IllegalArgumentException("The WAL path couldn't be null");
     }
-    final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
-    return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2 : 1)]);
+    Matcher matcher = WAL_FILE_NAME_PATTERN.matcher(walName.getName());
+    if (matcher.matches()) {
+      return Long.parseLong(matcher.group(2));
+    } else {
+      throw new IllegalArgumentException(walName.getName() + " is not a valid wal file name");
+    }
   }
 
   /**
    * Pattern used to validate a WAL file name see {@link #validateWALFilename(String)} for
    * description.
    */
-  private static final Pattern pattern =
-    Pattern.compile(".*\\.\\d*(" + META_WAL_PROVIDER_ID + ")*");
+  private static final Pattern WAL_FILE_NAME_PATTERN =
+    Pattern.compile("(.+)\\.(\\d+)(\\.[0-9A-Za-z]+)?");
 
   /**
    * A WAL file name is of the format: &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}
-   * &lt;file-creation-timestamp&gt;[.meta]. provider-name is usually made up of a server-name and a
-   * provider-id
+   * &lt;file-creation-timestamp&gt;[.&lt;suffix&gt;]. provider-name is usually made up of a
+   * server-name and a provider-id
    * @param filename name of the file to validate
    * @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt> otherwise
    */
   public static boolean validateWALFilename(String filename) {
-    return pattern.matcher(filename).matches();
+    return WAL_FILE_NAME_PATTERN.matcher(filename).matches();
   }
 
   /**
@@ -517,10 +522,15 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
    * log_prefix.filenumber.log_suffix
    * @param name Name of the WAL to parse
    * @return prefix of the log
+   * @throws IllegalArgumentException if the name passed in is not a valid wal file name
    * @see AbstractFSWAL#getCurrentFileName()
    */
   public static String getWALPrefixFromWALName(String name) {
-    int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
-    return name.substring(0, endIndex);
+    Matcher matcher = WAL_FILE_NAME_PATTERN.matcher(name);
+    if (matcher.matches()) {
+      return matcher.group(1);
+    } else {
+      throw new IllegalArgumentException(name + " is not a valid wal file name");
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/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
index 3b56aa2..8faccd7 100644
--- 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
@@ -17,7 +17,6 @@
  */
 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;
 
@@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
 import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -113,8 +113,12 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
     channelClass = eventLoopGroupAndChannelClass.getSecond();
   }
 
+  // Use a timestamp to make it identical. That means, after we transit the peer to DA/S and then
+  // back to A, the log prefix will be changed. This is used to simplify the implementation for
+  // replication source, where we do not need to consider that a terminated shipper could be added
+  // back.
   private String getLogPrefix(String peerId) {
-    return factory.factoryId + WAL_FILE_NAME_DELIMITER + peerId;
+    return factory.factoryId + "-" + EnvironmentEdgeManager.currentTime() + "-" + peerId;
   }
 
   private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException {
@@ -250,7 +254,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   @Override
   public void peerSyncReplicationStateChange(String peerId, SyncReplicationState from,
       SyncReplicationState to, int stage) {
-    if (from == SyncReplicationState.ACTIVE && to == SyncReplicationState.DOWNGRADE_ACTIVE) {
+    if (from == SyncReplicationState.ACTIVE) {
       if (stage == 0) {
         Lock lock = createLock.acquireLock(peerId);
         try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/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 5ea3173..cff8ceb 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
@@ -80,6 +80,7 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Pair;
@@ -393,8 +394,8 @@ public abstract class TestReplicationSourceManager {
     // populate some znodes in the peer znode
     SortedSet<String> files = new TreeSet<>();
     String group = "testgroup";
-    String file1 = group + ".log1";
-    String file2 = group + ".log2";
+    String file1 = group + "." + EnvironmentEdgeManager.currentTime() + ".log1";
+    String file2 = group + "." + EnvironmentEdgeManager.currentTime() + ".log2";
     files.add(file1);
     files.add(file2);
     for (String file : files) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSyncReplicationShipperQuit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSyncReplicationShipperQuit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSyncReplicationShipperQuit.java
new file mode 100644
index 0000000..f6dc3d7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSyncReplicationShipperQuit.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.replication.SyncReplicationTestBase;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Testcase for HBASE-20456.
+ */
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSyncReplicationShipperQuit extends SyncReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSyncReplicationShipperQuit.class);
+
+  @Test
+  public void testShipperQuitWhenDA() throws Exception {
+    // set to serial replication
+    UTIL1.getAdmin().updateReplicationPeerConfig(PEER_ID, ReplicationPeerConfig
+      .newBuilder(UTIL1.getAdmin().getReplicationPeerConfig(PEER_ID)).setSerial(true).build());
+    UTIL2.getAdmin().updateReplicationPeerConfig(PEER_ID, ReplicationPeerConfig
+      .newBuilder(UTIL2.getAdmin().getReplicationPeerConfig(PEER_ID)).setSerial(true).build());
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.ACTIVE);
+
+    writeAndVerifyReplication(UTIL1, UTIL2, 0, 100);
+    HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME);
+    DualAsyncFSWAL wal =
+      (DualAsyncFSWAL) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build());
+    String walGroupId =
+      AbstractFSWALProvider.getWALPrefixFromWALName(wal.getCurrentFileName().getName());
+    ReplicationSourceShipper shipper =
+      ((ReplicationSource) ((Replication) rs.getReplicationSourceService()).getReplicationManager()
+        .getSource(PEER_ID)).workerThreads.get(walGroupId);
+    assertFalse(shipper.isFinished());
+
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    writeAndVerifyReplication(UTIL1, UTIL2, 100, 200);
+
+    ReplicationSource source = (ReplicationSource) ((Replication) rs.getReplicationSourceService())
+      .getReplicationManager().getSource(PEER_ID);
+    // the peer is serial so here we can make sure that the previous wals have already been
+    // replicated, and finally the shipper should be removed from the worker pool
+    UTIL1.waitFor(10000, () -> !source.workerThreads.containsKey(walGroupId));
+    assertTrue(shipper.isFinished());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/863566a7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
index 35e4f82..fac6f74 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
@@ -413,9 +413,7 @@ public class TestWALEntryStream {
     batch = reader.take();
     assertEquals(walPath, batch.getLastWalPath());
     assertEquals(5, batch.getNbEntries());
-    // Actually this should be true but we haven't handled this yet since for a normal queue the
-    // last one is always open... Not a big deal for now.
-    assertFalse(batch.isEndOfFile());
+    assertTrue(batch.isEndOfFile());
 
     assertSame(WALEntryBatch.NO_MORE_DATA, reader.take());
   }


[36/36] hbase git commit: HBASE-20637 Polish the WAL switching when transiting from A to S

Posted by zh...@apache.org.
HBASE-20637 Polish the WAL switching when transiting from A to S


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

Branch: refs/heads/HBASE-19064
Commit: 1f3e50fcefd049290e5b102d04b568e40e6dc91e
Parents: 46866d9
Author: zhangduo <zh...@apache.org>
Authored: Tue May 29 20:38:20 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:41:40 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AsyncFSWAL.java      | 52 +++++++++++++-
 .../hbase/regionserver/wal/DualAsyncFSWAL.java  | 71 ++++++++++++++------
 .../apache/hadoop/hbase/util/FSHDFSUtils.java   | 16 +++--
 .../hbase/wal/SyncReplicationWALProvider.java   |  2 +-
 .../replication/DualAsyncFSWALForTest.java      |  4 +-
 .../replication/SyncReplicationTestBase.java    | 26 +++++--
 .../replication/TestSyncReplicationActive.java  | 42 ++++++++++--
 7 files changed, 176 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1f3e50fc/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 9b4ce9c..7f3e30b 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
@@ -52,12 +52,12 @@ import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 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.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
@@ -470,6 +470,44 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     // whether to issue a sync in the caller method.
   }
 
+  private void drainNonMarkerEditsAndFailSyncs() {
+    if (toWriteAppends.isEmpty()) {
+      return;
+    }
+    boolean hasNonMarkerEdits = false;
+    Iterator<FSWALEntry> iter = toWriteAppends.descendingIterator();
+    while (iter.hasNext()) {
+      FSWALEntry entry = iter.next();
+      if (!entry.getEdit().isMetaEdit()) {
+        hasNonMarkerEdits = true;
+        break;
+      }
+    }
+    if (hasNonMarkerEdits) {
+      for (;;) {
+        iter.remove();
+        if (!iter.hasNext()) {
+          break;
+        }
+        iter.next();
+      }
+      unackedAppends.clear();
+      // fail the sync futures which are under the txid of the first remaining edit, if none, fail
+      // all the sync futures.
+      long txid = toWriteAppends.isEmpty() ? Long.MAX_VALUE : toWriteAppends.peek().getTxid();
+      IOException error = new IOException("WAL is closing, only marker edit is allowed");
+      for (Iterator<SyncFuture> syncIter = syncFutures.iterator(); syncIter.hasNext();) {
+        SyncFuture future = syncIter.next();
+        if (future.getTxid() < txid) {
+          future.done(future.getTxid(), error);
+          syncIter.remove();
+        } else {
+          break;
+        }
+      }
+    }
+  }
+
   private void consume() {
     consumeLock.lock();
     try {
@@ -512,6 +550,9 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
       }
       waitingConsumePayloadsGatingSequence.set(nextCursor);
     }
+    if (markerEditOnly()) {
+      drainNonMarkerEditsAndFailSyncs();
+    }
     appendAndSync();
     if (hasConsumerTask.get()) {
       return;
@@ -553,9 +594,18 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     return consumerScheduled.compareAndSet(false, true);
   }
 
+  // This is used by sync replication, where we are going to close the wal soon after we reopen all
+  // the regions. Will be overridden by sub classes.
+  protected boolean markerEditOnly() {
+    return false;
+  }
+
   @Override
   public long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore)
       throws IOException {
+    if (markerEditOnly() && !edits.isMetaEdit()) {
+      throw new IOException("WAL is closing, only marker edit is allowed");
+    }
     long txid =
       stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, waitingConsumePayloads);
     if (shouldScheduleConsumer()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f3e50fc/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 3967e78..bf5b96d 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
@@ -18,14 +18,19 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 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.client.ConnectionUtils;
 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.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
 
@@ -35,20 +40,24 @@ import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
 @InterfaceAudience.Private
 public class DualAsyncFSWAL extends AsyncFSWAL {
 
+  private static final Logger LOG = LoggerFactory.getLogger(DualAsyncFSWAL.class);
+
   private final FileSystem remoteFs;
 
-  private final Path remoteWalDir;
+  private final Path remoteWALDir;
+
+  private volatile boolean skipRemoteWAL = false;
 
-  private volatile boolean skipRemoteWal = false;
+  private volatile boolean markerEditOnly = false;
 
-  public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteWalDir,
+  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);
+      eventLoopGroup, channelClass);
     this.remoteFs = remoteFs;
-    this.remoteWalDir = remoteWalDir;
+    this.remoteWALDir = remoteWALDir;
   }
 
   // will be overridden in testcase
@@ -61,20 +70,37 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
   @Override
   protected AsyncWriter createWriterInstance(Path path) throws IOException {
     AsyncWriter localWriter = super.createWriterInstance(path);
-    if (skipRemoteWal) {
-      return localWriter;
-    }
-    AsyncWriter remoteWriter;
-    boolean succ = false;
-    try {
-      remoteWriter = createAsyncWriter(remoteFs, new Path(remoteWalDir, path.getName()));
-      succ = true;
-    } finally {
-      if (!succ) {
-        closeWriter(localWriter);
+    // retry forever if we can not create the remote writer to prevent aborting the RS due to log
+    // rolling error, unless the skipRemoteWal is set to true.
+    // TODO: since for now we only have one thread doing log rolling, this may block the rolling for
+    // other wals
+    Path remoteWAL = new Path(remoteWALDir, path.getName());
+    for (int retry = 0;; retry++) {
+      if (skipRemoteWAL) {
+        return localWriter;
+      }
+      AsyncWriter remoteWriter;
+      try {
+        remoteWriter = createAsyncWriter(remoteFs, remoteWAL);
+      } catch (IOException e) {
+        LOG.warn("create remote writer {} failed, retry = {}", remoteWAL, retry, e);
+        try {
+          Thread.sleep(ConnectionUtils.getPauseTime(100, retry));
+        } catch (InterruptedException ie) {
+          // restore the interrupt state
+          Thread.currentThread().interrupt();
+          Closeables.close(localWriter, true);
+          throw (IOException) new InterruptedIOException().initCause(ie);
+        }
+        continue;
       }
+      return createCombinedAsyncWriter(localWriter, remoteWriter);
     }
-    return createCombinedAsyncWriter(localWriter, remoteWriter);
+  }
+
+  @Override
+  protected boolean markerEditOnly() {
+    return markerEditOnly;
   }
 
   // Allow temporarily skipping the creation of remote writer. When failing to write to the remote
@@ -82,7 +108,14 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
   // need to write a close marker when closing a region, and if it fails, the whole rs will abort.
   // So here we need to skip the creation of remote writer and make it possible to write the region
   // close marker.
-  public void skipRemoteWal() {
-    this.skipRemoteWal = true;
+  // Setting markerEdit only to true is for transiting from A to S, where we need to give up writing
+  // any pending wal entries as they will be discarded. The remote cluster will replicated the
+  // correct data back later. We still need to allow writing marker edits such as close region event
+  // to allow closing a region.
+  public void skipRemoteWAL(boolean markerEditOnly) {
+    if (markerEditOnly) {
+      this.markerEditOnly = true;
+    }
+    this.skipRemoteWAL = true;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f3e50fc/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
index 301d158..a49ee02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
@@ -28,9 +28,9 @@ import java.util.Collection;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
@@ -155,12 +155,16 @@ public class FSHDFSUtils extends FSUtils {
    * Recover the lease from HDFS, retrying multiple times.
    */
   @Override
-  public void recoverFileLease(final FileSystem fs, final Path p,
-      Configuration conf, CancelableProgressable reporter)
-  throws IOException {
+  public void recoverFileLease(FileSystem fs, Path p, Configuration conf,
+      CancelableProgressable reporter) throws IOException {
+    if (fs instanceof FilterFileSystem) {
+      fs = ((FilterFileSystem) fs).getRawFileSystem();
+    }
     // lease recovery not needed for local file system case.
-    if (!(fs instanceof DistributedFileSystem)) return;
-    recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter);
+    if (!(fs instanceof DistributedFileSystem)) {
+      return;
+    }
+    recoverDFSFileLease((DistributedFileSystem) fs, p, conf, reporter);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f3e50fc/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
index 82f8a89..b9fffcf 100644
--- 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
@@ -291,7 +291,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
         try {
           Optional<DualAsyncFSWAL> opt = peerId2WAL.get(peerId);
           if (opt != null) {
-            opt.ifPresent(DualAsyncFSWAL::skipRemoteWal);
+            opt.ifPresent(w -> w.skipRemoteWAL(to == SyncReplicationState.STANDBY));
           } else {
             // add a place holder to tell the getWAL caller do not use DualAsyncFSWAL any more.
             peerId2WAL.put(peerId, Optional.empty());

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f3e50fc/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java
index fb3daf2..62000b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java
@@ -97,11 +97,11 @@ class DualAsyncFSWALForTest extends DualAsyncFSWAL {
     }
   }
 
-  public DualAsyncFSWALForTest(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteWalDir,
+  public DualAsyncFSWALForTest(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, remoteFs, rootDir, remoteWalDir, logDir, archiveDir, conf, listeners, failIfWALExists,
+    super(fs, remoteFs, rootDir, remoteWALDir, logDir, archiveDir, conf, listeners, failIfWALExists,
       prefix, suffix, eventLoopGroup, channelClass);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f3e50fc/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
index 095be90..a20edd3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,6 +33,7 @@ import org.apache.hadoop.hbase.HBaseZKTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
@@ -127,10 +129,26 @@ public class SyncReplicationTestBase {
         .setRemoteWALDir(REMOTE_WAL_DIR1.toUri().toString()).build());
   }
 
+  private static void shutdown(HBaseTestingUtility util) throws Exception {
+    if (util.getHBaseCluster() == null) {
+      return;
+    }
+    Admin admin = util.getAdmin();
+    if (!admin.listReplicationPeers(Pattern.compile(PEER_ID)).isEmpty()) {
+      if (admin
+        .getReplicationPeerSyncReplicationState(PEER_ID) != SyncReplicationState.DOWNGRADE_ACTIVE) {
+        admin.transitReplicationPeerSyncReplicationState(PEER_ID,
+          SyncReplicationState.DOWNGRADE_ACTIVE);
+      }
+      admin.removeReplicationPeer(PEER_ID);
+    }
+    util.shutdownMiniCluster();
+  }
+
   @AfterClass
   public static void tearDown() throws Exception {
-    UTIL1.shutdownMiniCluster();
-    UTIL2.shutdownMiniCluster();
+    shutdown(UTIL1);
+    shutdown(UTIL2);
     ZK_UTIL.shutdownMiniZKCluster();
   }
 
@@ -207,7 +225,7 @@ public class SyncReplicationTestBase {
   protected void verifyRemovedPeer(String peerId, Path remoteWALDir, HBaseTestingUtility utility)
       throws Exception {
     ReplicationPeerStorage rps = ReplicationStorageFactory
-        .getReplicationPeerStorage(utility.getZooKeeperWatcher(), utility.getConfiguration());
+      .getReplicationPeerStorage(utility.getZooKeeperWatcher(), utility.getConfiguration());
     try {
       rps.getPeerSyncReplicationState(peerId);
       fail("Should throw exception when get the sync replication state of a removed peer.");
@@ -233,7 +251,7 @@ public class SyncReplicationTestBase {
     Entry[] entries = new Entry[10];
     for (int i = 0; i < entries.length; i++) {
       entries[i] =
-          new Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TABLE_NAME, 0), new WALEdit());
+        new Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TABLE_NAME, 0), new WALEdit());
     }
     if (!expectedRejection) {
       ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),

http://git-wip-us.apache.org/repos/asf/hbase/blob/1f3e50fc/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
index fce0cdf..42adab6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
@@ -17,13 +17,28 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.fs.FileStatus;
 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.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WAL.Reader;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -66,8 +81,27 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
     // confirm that the data is there after we convert the peer to DA
     verify(UTIL2, 0, 100);
 
-    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
-      SyncReplicationState.STANDBY);
+    try (AsyncConnection conn =
+      ConnectionFactory.createAsyncConnection(UTIL1.getConfiguration()).get()) {
+      AsyncTable<?> table = conn.getTableBuilder(TABLE_NAME).setMaxAttempts(1).build();
+      CompletableFuture<Void> future =
+        table.put(new Put(Bytes.toBytes(1000)).addColumn(CF, CQ, Bytes.toBytes(1000)));
+      Thread.sleep(2000);
+      // should hang on rolling
+      assertFalse(future.isDone());
+      UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+        SyncReplicationState.STANDBY);
+      try {
+        future.get();
+        fail("should fail because of the wal is closing");
+      } catch (ExecutionException e) {
+        // expected
+        assertThat(e.getCause().getMessage(), containsString("only marker edit is allowed"));
+      }
+    }
+    // confirm that the data has not been persisted
+    HRegion region = UTIL1.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    assertTrue(region.get(new Get(Bytes.toBytes(1000))).isEmpty());
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.ACTIVE);
 
@@ -89,8 +123,8 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
     FileStatus[] files = fs2.listStatus(new Path(remoteDir, peerId));
     Assert.assertTrue(files.length > 0);
     for (FileStatus file : files) {
-      try (Reader reader =
-          WALFactory.createReader(fs2, file.getPath(), utility.getConfiguration())) {
+      try (
+        Reader reader = WALFactory.createReader(fs2, file.getPath(), utility.getConfiguration())) {
         Entry entry = reader.next();
         Assert.assertTrue(entry != null);
         while (entry != null) {


[24/36] hbase git commit: HBASE-19943 Only allow removing sync replication peer which is in DA state

Posted by zh...@apache.org.
HBASE-19943 Only allow removing sync replication peer which is in DA state


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

Branch: refs/heads/HBASE-19064
Commit: 7c326e6c5867ac5505e05a333ff43954ed47a852
Parents: 7f1dbdf
Author: huzheng <op...@gmail.com>
Authored: Thu Mar 1 18:34:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerManager.java     | 14 ++++-
 .../hbase/wal/SyncReplicationWALProvider.java   |  2 +-
 .../replication/TestReplicationAdmin.java       | 63 ++++++++++++++++++++
 .../hbase/replication/TestSyncReplication.java  |  2 +-
 4 files changed, 78 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7c326e6c/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 0dc922d..41dd6e3 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
@@ -120,8 +120,20 @@ public class ReplicationPeerManager {
     return desc;
   }
 
+  private void checkPeerInDAStateIfSyncReplication(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    if (desc != null && desc.getPeerConfig().isSyncReplication()
+        && !SyncReplicationState.DOWNGRADE_ACTIVE.equals(desc.getSyncReplicationState())) {
+      throw new DoNotRetryIOException("Couldn't remove synchronous replication peer with state="
+          + desc.getSyncReplicationState()
+          + ", Transit the synchronous replication state to be DOWNGRADE_ACTIVE firstly.");
+    }
+  }
+
   ReplicationPeerConfig preRemovePeer(String peerId) throws DoNotRetryIOException {
-    return checkPeerExists(peerId).getPeerConfig();
+    ReplicationPeerDescription pd = checkPeerExists(peerId);
+    checkPeerInDAStateIfSyncReplication(peerId);
+    return pd.getPeerConfig();
   }
 
   void preEnablePeer(String peerId) throws DoNotRetryIOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c326e6c/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
index ac4b4cd..282aa21 100644
--- 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
@@ -142,7 +142,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   @Override
   public WAL getWAL(RegionInfo region) throws IOException {
     if (region == null) {
-      return provider.getWAL(region);
+      return provider.getWAL(null);
     }
     Optional<Pair<String, String>> peerIdAndRemoteWALDir =
       peerInfoProvider.getPeerIdAndRemoteWALDir(region);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c326e6c/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 0ad476f..486ab51 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
@@ -254,6 +254,62 @@ public class TestReplicationAdmin {
   }
 
   @Test
+  public void testRemovePeerWithNonDAState() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    TEST_UTIL.createTable(tableName, Bytes.toBytes("family"));
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
+
+    String rootDir = "hdfs://srv1:9999/hbase";
+    builder.setClusterKey(KEY_ONE);
+    builder.setRemoteWALDir(rootDir);
+    builder.setReplicateAllUserTables(false);
+    Map<TableName, List<String>> tableCfs = new HashMap<>();
+    tableCfs.put(tableName, new ArrayList<>());
+    builder.setTableCFsMap(tableCfs);
+    hbaseAdmin.addReplicationPeer(ID_ONE, builder.build());
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE));
+
+    // Transit sync replication state to ACTIVE.
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE, SyncReplicationState.ACTIVE);
+    assertEquals(SyncReplicationState.ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE));
+
+    try {
+      hbaseAdmin.removeReplicationPeer(ID_ONE);
+      fail("Can't remove a synchronous replication peer with state=ACTIVE");
+    } catch (IOException e) {
+      // OK
+    }
+
+    // Transit sync replication state to DA
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE));
+    // Transit sync replication state to STANDBY
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE, SyncReplicationState.STANDBY);
+    assertEquals(SyncReplicationState.STANDBY,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE));
+
+    try {
+      hbaseAdmin.removeReplicationPeer(ID_ONE);
+      fail("Can't remove a synchronous replication peer with state=STANDBY");
+    } catch (IOException e) {
+      // OK
+    }
+
+    // Transit sync replication state to DA
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE));
+
+    hbaseAdmin.removeReplicationPeer(ID_ONE);
+    assertEquals(0, hbaseAdmin.listReplicationPeers().size());
+  }
+
+  @Test
   public void testAddPeerWithState() throws Exception {
     ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
     rpc1.setClusterKey(KEY_ONE);
@@ -1072,5 +1128,12 @@ public class TestReplicationAdmin {
     } catch (Exception e) {
       // OK
     }
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+    hbaseAdmin.removeReplicationPeer(ID_ONE);
+    hbaseAdmin.removeReplicationPeer(ID_SECOND);
+    assertEquals(0, hbaseAdmin.listReplicationPeers().size());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c326e6c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
index 196019d..cc84dab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
@@ -109,7 +109,7 @@ public class TestSyncReplication {
     UTIL1.startMiniCluster(3);
     UTIL2.startMiniCluster(3);
     TableDescriptor td =
-        TableDescriptorBuilder.newBuilder(TABLE_NAME).addColumnFamily(ColumnFamilyDescriptorBuilder
+        TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(ColumnFamilyDescriptorBuilder
           .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build();
     UTIL1.getAdmin().createTable(td);
     UTIL2.getAdmin().createTable(td);


[26/36] hbase git commit: HBASE-20432 Cleanup related resources when remove a sync replication peer

Posted by zh...@apache.org.
HBASE-20432 Cleanup related resources when remove 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/f1e81b9e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f1e81b9e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f1e81b9e

Branch: refs/heads/HBASE-19064
Commit: f1e81b9e35988e1d6f250741abecfbf7a07b3ce6
Parents: 3e2cc73
Author: huzheng <op...@gmail.com>
Authored: Wed Apr 18 20:38:33 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../master/replication/RemovePeerProcedure.java | 10 +++++
 .../ReplaySyncReplicationWALManager.java        |  8 ++++
 .../replication/SyncReplicationTestBase.java    | 45 +++++++++++++++++---
 .../replication/TestSyncReplicationActive.java  |  9 ++--
 .../replication/TestSyncReplicationStandBy.java | 31 ++++++++++++--
 5 files changed, 89 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f1e81b9e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index 82dc07e..7335fe0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -66,9 +66,19 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
     env.getReplicationPeerManager().removePeer(peerId);
   }
 
+  private void removeRemoteWALs(MasterProcedureEnv env) throws IOException {
+    ReplaySyncReplicationWALManager remoteWALManager =
+        env.getMasterServices().getReplaySyncReplicationWALManager();
+    remoteWALManager.removePeerRemoteWALs(peerId);
+    remoteWALManager.removePeerReplayWALDir(peerId);
+  }
+
   @Override
   protected void postPeerModification(MasterProcedureEnv env)
       throws IOException, ReplicationException {
+    if (peerConfig.isSyncReplication()) {
+      removeRemoteWALs(env);
+    }
     env.getReplicationPeerManager().removeAllQueuesAndHFileRefs(peerId);
     if (peerConfig.isSerial()) {
       env.getReplicationPeerManager().removeAllLastPushedSeqIds(peerId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f1e81b9e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
index 72f5c37..eac5aa4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
@@ -115,6 +115,14 @@ public class ReplaySyncReplicationWALManager {
     }
   }
 
+  public void removePeerRemoteWALs(String peerId) throws IOException {
+    Path remoteWALDir = getPeerRemoteWALDir(peerId);
+    if (fs.exists(remoteWALDir) && !fs.delete(remoteWALDir, true)) {
+      throw new IOException(
+          "Failed to remove remote WALs dir " + remoteWALDir + " for peer id=" + peerId);
+    }
+  }
+
   public void initPeerWorkers(String peerId) {
     BlockingQueue<ServerName> servers = new LinkedBlockingQueue<>();
     services.getServerManager().getOnlineServers().keySet()

http://git-wip-us.apache.org/repos/asf/hbase/blob/f1e81b9e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
index 0d5fce8..de679be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -71,6 +72,10 @@ public class SyncReplicationTestBase {
 
   protected static String PEER_ID = "1";
 
+  protected static Path remoteWALDir1;
+
+  protected static Path remoteWALDir2;
+
   private static void initTestingUtility(HBaseTestingUtility util, String zkParent) {
     util.setZkCluster(ZK_UTIL.getZkCluster());
     Configuration conf = util.getConfiguration();
@@ -104,11 +109,11 @@ public class SyncReplicationTestBase {
     UTIL2.getAdmin().createTable(td);
     FileSystem fs1 = UTIL1.getTestFileSystem();
     FileSystem fs2 = UTIL2.getTestFileSystem();
-    Path remoteWALDir1 =
-      new Path(UTIL1.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
+    remoteWALDir1 =
+      new Path(UTIL1.getMiniHBaseCluster().getMaster().getMasterFileSystem().getWALRootDir(),
         "remoteWALs").makeQualified(fs1.getUri(), fs1.getWorkingDirectory());
-    Path remoteWALDir2 =
-      new Path(UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
+    remoteWALDir2 =
+      new Path(UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem().getWALRootDir(),
         "remoteWALs").makeQualified(fs2.getUri(), fs2.getWorkingDirectory());
     UTIL1.getAdmin().addReplicationPeer(PEER_ID,
       ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey())
@@ -188,7 +193,37 @@ public class SyncReplicationTestBase {
 
   protected final Path getRemoteWALDir(MasterFileSystem mfs, String peerId) {
     Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
-    return new Path(remoteWALDir, PEER_ID);
+    return getRemoteWALDir(remoteWALDir, peerId);
+  }
+
+  protected Path getRemoteWALDir(Path remoteWALDir, String peerId) {
+    return new Path(remoteWALDir, peerId);
+  }
+
+  protected Path getReplayRemoteWALs(Path remoteWALDir, String peerId) {
+    return new Path(remoteWALDir, peerId + "-replay");
+  }
+
+  protected void verifyRemovedPeer(String peerId, Path remoteWALDir, HBaseTestingUtility utility)
+      throws Exception {
+    ReplicationPeerStorage rps = ReplicationStorageFactory
+        .getReplicationPeerStorage(utility.getZooKeeperWatcher(), utility.getConfiguration());
+    try {
+      rps.getPeerSyncReplicationState(peerId);
+      fail("Should throw exception when get the sync replication state of a removed peer.");
+    } catch (NullPointerException e) {
+      // ignore.
+    }
+    try {
+      rps.getPeerNewSyncReplicationState(peerId);
+      fail("Should throw exception when get the new sync replication state of a removed peer");
+    } catch (NullPointerException e) {
+      // ignore.
+    }
+    try (FileSystem fs = utility.getTestFileSystem()) {
+      Assert.assertFalse(fs.exists(getRemoteWALDir(remoteWALDir, peerId)));
+      Assert.assertFalse(fs.exists(getReplayRemoteWALs(remoteWALDir, peerId)));
+    }
   }
 
   protected void verifyReplicationRequestRejection(HBaseTestingUtility utility,

http://git-wip-us.apache.org/repos/asf/hbase/blob/f1e81b9e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
index f9020a0..b663c44 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
@@ -58,7 +58,7 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
     verifyNotReplicatedThroughRegion(UTIL2, 0, 100);
 
     // Ensure that there's no cluster id in remote log entries.
-    verifyNoClusterIdInRemoteLog(UTIL2, PEER_ID);
+    verifyNoClusterIdInRemoteLog(UTIL2, remoteWALDir2, PEER_ID);
 
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.DOWNGRADE_ACTIVE);
@@ -84,12 +84,9 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
     write(UTIL2, 200, 300);
   }
 
-  private void verifyNoClusterIdInRemoteLog(HBaseTestingUtility utility, String peerId)
-      throws Exception {
+  private void verifyNoClusterIdInRemoteLog(HBaseTestingUtility utility, Path remoteDir,
+      String peerId) throws Exception {
     FileSystem fs2 = utility.getTestFileSystem();
-    Path remoteDir =
-        new Path(utility.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
-            "remoteWALs").makeQualified(fs2.getUri(), fs2.getWorkingDirectory());
     FileStatus[] files = fs2.listStatus(new Path(remoteDir, peerId));
     Assert.assertTrue(files.length > 0);
     for (FileStatus file : files) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f1e81b9e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
index ed61d2a..8526af8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
@@ -24,6 +24,8 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Arrays;
+
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -39,6 +41,7 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -84,13 +87,35 @@ public class TestSyncReplicationStandBy extends SyncReplicationTestBase {
       assertDisallow(table,
         t -> t.get(Arrays.asList(new Get(Bytes.toBytes("row")), new Get(Bytes.toBytes("row1")))));
       assertDisallow(table,
-        t -> t
-          .put(Arrays.asList(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")),
+        t -> t.put(
+          Arrays.asList(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")),
             new Put(Bytes.toBytes("row1")).addColumn(CF, CQ, Bytes.toBytes("row1")))));
       assertDisallow(table, t -> t.mutateRow(new RowMutations(Bytes.toBytes("row"))
-        .add((Mutation) new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")))));
+          .add((Mutation) new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")))));
     }
     // We should still allow replication writes
     writeAndVerifyReplication(UTIL1, UTIL2, 0, 100);
+
+    // Remove the peers in ACTIVE & STANDBY cluster.
+    FileSystem fs2 = remoteWALDir2.getFileSystem(UTIL2.getConfiguration());
+    Assert.assertTrue(fs2.exists(getRemoteWALDir(remoteWALDir2, PEER_ID)));
+
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    Assert.assertFalse(fs2.exists(getRemoteWALDir(remoteWALDir2, PEER_ID)));
+    Assert.assertFalse(fs2.exists(getReplayRemoteWALs(remoteWALDir2, PEER_ID)));
+
+    UTIL1.getAdmin().removeReplicationPeer(PEER_ID);
+    verifyRemovedPeer(PEER_ID, remoteWALDir1, UTIL1);
+
+    // Peer remoteWAL dir will be renamed to replay WAL dir when transit from S to DA, and the
+    // replay WAL dir will be removed after replaying all WALs, so create a emtpy dir here to test
+    // whether the removeReplicationPeer would remove the remoteWAL dir.
+    fs2.create(getRemoteWALDir(remoteWALDir2, PEER_ID));
+    fs2.create(getReplayRemoteWALs(remoteWALDir2, PEER_ID));
+    Assert.assertTrue(fs2.exists(getRemoteWALDir(remoteWALDir2, PEER_ID)));
+    Assert.assertTrue(fs2.exists(getReplayRemoteWALs(remoteWALDir2, PEER_ID)));
+    UTIL2.getAdmin().removeReplicationPeer(PEER_ID);
+    verifyRemovedPeer(PEER_ID, remoteWALDir2, UTIL2);
   }
 }


[06/36] hbase git commit: HBASE-20501 update minimum supported Hadoop version to 2.7.1.

Posted by zh...@apache.org.
HBASE-20501 update minimum supported Hadoop version to 2.7.1.

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: c19fbf2be0c104174b21dc372e4b4dfae4b61096
Parents: b889c8a
Author: Sean Busbey <bu...@apache.org>
Authored: Wed May 30 21:22:42 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed May 30 21:22:42 2018 -0500

----------------------------------------------------------------------
 dev-support/hbase-personality.sh               | 25 +++++++++++++-------
 src/main/asciidoc/_chapters/configuration.adoc | 26 ++++++++++-----------
 src/main/asciidoc/_chapters/upgrading.adoc     | 12 ++++++++--
 3 files changed, 40 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c19fbf2b/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index b033645..2c6e4a8 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -443,23 +443,32 @@ function hadoopcheck_rebuild
 
   # All supported Hadoop versions that we want to test the compilation with
   # See the Hadoop section on prereqs in the HBase Reference Guide
-  hbase_common_hadoop2_versions="2.6.1 2.6.2 2.6.3 2.6.4 2.6.5 2.7.1 2.7.2 2.7.3 2.7.4"
-  if [[ "${PATCH_BRANCH}" = branch-1* ]]; then
-    yetus_info "Setting Hadoop versions to test based on branch-1-ish rules."
+  hbase_common_hadoop2_versions="2.7.1 2.7.2 2.7.3 2.7.4"
+  if [[ "${PATCH_BRANCH}" = branch-1.* ]] && [[ "${PATCH_BRANCH#branch-1.}" -lt "5" ]]; then
+    yetus_info "Setting Hadoop 2 versions to test based on before-branch-1.5 rules."
     if [[ "${QUICK_HADOOPCHECK}" == "true" ]]; then
       hbase_hadoop2_versions="2.4.1 2.5.2 2.6.5 2.7.4"
     else
-      hbase_hadoop2_versions="2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 ${hbase_common_hadoop2_versions}"
+      hbase_hadoop2_versions="2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.1 2.6.2 2.6.3 2.6.4 2.6.5 ${hbase_common_hadoop2_versions}"
     fi
-    hbase_hadoop3_versions=""
-  else # master or a feature branch
-    yetus_info "Setting Hadoop versions to test based on branch-2/master/feature branch rules."
+  elif [[ "${PATCH_BRANCH}" = branch-2.0 ]]; then
+    yetus_info "Setting Hadoop 2 versions to test based on branch-2.0 rules."
     if [[ "${QUICK_HADOOPCHECK}" == "true" ]]; then
       hbase_hadoop2_versions="2.6.5 2.7.4"
     else
+      hbase_hadoop2_versions="2.6.1 2.6.2 2.6.3 2.6.4 2.6.5 ${hbase_common_hadoop2_versions}"
+    fi
+  else
+    yetus_info "Setting Hadoop 2 versions to test based on branch-1.5+/branch-2.1+/master/feature branch rules."
+    if [[ "${QUICK_HADOOPCHECK}" == "true" ]]; then
+      hbase_hadoop2_versions="2.7.4"
+    else
       hbase_hadoop2_versions="${hbase_common_hadoop2_versions}"
     fi
-    hbase_hadoop3_versions="3.0.0"
+  fi
+  hbase_hadoop3_versions="3.0.0"
+  if [[ "${PATCH_BRANCH}" = branch-1* ]]; then
+    hbase_hadoop3_versions=""
   fi
 
   export MAVEN_OPTS="${MAVEN_OPTS}"

http://git-wip-us.apache.org/repos/asf/hbase/blob/c19fbf2b/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index 1be9e13..31250ff 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -217,20 +217,20 @@ Use the following legend to interpret this table:
 * "X" = not supported
 * "NT" = Not tested
 
-[cols="1,1,1,1", options="header"]
+[cols="1,1,1,1,1,1", options="header"]
 |===
-| | HBase-1.2.x | HBase-1.3.x | HBase-2.0.x
-|Hadoop-2.4.x | S | S | X
-|Hadoop-2.5.x | S | S | X
-|Hadoop-2.6.0 | X | X | X
-|Hadoop-2.6.1+ | S | S | S
-|Hadoop-2.7.0 | X | X | X
-|Hadoop-2.7.1+ | S | S | S
-|Hadoop-2.8.[0-1] | X | X | X
-|Hadoop-2.8.2 | NT | NT | NT
-|Hadoop-2.8.3+ | NT | NT | S
-|Hadoop-2.9.0 | X | X | X
-|Hadoop-3.0.0 | NT | NT | NT
+| | HBase-1.2.x | HBase-1.3.x | HBase-1.5.x | HBase-2.0.x | HBase-2.1.x
+|Hadoop-2.4.x | S | S | X | X | X
+|Hadoop-2.5.x | S | S | X | X | X
+|Hadoop-2.6.0 | X | X | X | X | X
+|Hadoop-2.6.1+ | S | S | X | S | X
+|Hadoop-2.7.0 | X | X | X | X | X
+|Hadoop-2.7.1+ | S | S | S | S | S
+|Hadoop-2.8.[0-1] | X | X | X | X | X
+|Hadoop-2.8.2 | NT | NT | NT | NT | NT
+|Hadoop-2.8.3+ | NT | NT | NT | S | S
+|Hadoop-2.9.0 | X | X | X | X | X
+|Hadoop-3.0.0 | NT | NT | NT | NT | NT
 |===
 
 .Hadoop Pre-2.6.1 and JDK 1.8 Kerberos

http://git-wip-us.apache.org/repos/asf/hbase/blob/c19fbf2b/src/main/asciidoc/_chapters/upgrading.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/upgrading.adoc b/src/main/asciidoc/_chapters/upgrading.adoc
index da4430a..6f0405d 100644
--- a/src/main/asciidoc/_chapters/upgrading.adoc
+++ b/src/main/asciidoc/_chapters/upgrading.adoc
@@ -86,8 +86,16 @@ for warning about incompatible changes). All effort will be made to provide a de
 * Example: Old compiled Coprocessor, Filter, or Plugin code will work unchanged with the new jars.
 
 .Dependency Compatibility
-* An upgrade of HBase will not require an incompatible upgrade of a dependent project, including the Java runtime.
-* Example: An upgrade of Hadoop will not invalidate any of the compatibilities guarantees we made.
+* An upgrade of HBase will not require an incompatible upgrade of a dependent project, except for Apache Hadoop.
+* An upgrade of HBase will not require an incompatible upgrade of the Java runtime.
+* Example: Upgrading HBase to a version that supports _Dependency Compatibility_ won't require that you upgrade your Apache ZooKeeper service.
+* Example: If your current version of HBase supported running on JDK 8, then an upgrade to a version that supports _Dependency Compatibility_ will also run on JDK 8.
+
+.Hadoop Versions
+[TIP]
+====
+Previously, we tried to maintain dependency compatibility for the underly Hadoop service but over the last few years this has proven untenable. While the HBase project attempts to maintain support for older versions of Hadoop, we drop the "supported" designator for minor versions that fail to continue to see releases. Additionally, the Hadoop project has its own set of compatibility guidelines, which means in some cases having to update to a newer supported minor release might break some of our compatibility promises.
+====
 
 .Operational Compatibility
 * Metric changes


[30/36] hbase git commit: HBASE-20425 Do not write the cluster id of the current active cluster when writing remote WAL

Posted by zh...@apache.org.
HBASE-20425 Do not write the cluster id of the current active cluster when writing remote WAL


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

Branch: refs/heads/HBASE-19064
Commit: bda472092332c695c860d4d5f42cf0a42ffecc47
Parents: e4676d9
Author: huzheng <op...@gmail.com>
Authored: Mon Apr 23 17:20:55 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../replication/TestSyncReplicationActive.java  | 32 ++++++++++++++++++++
 1 file changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bda47209/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
index bff4572..f9020a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
@@ -17,9 +17,17 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import org.apache.hadoop.fs.FileStatus;
+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.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -49,6 +57,9 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
     // peer is disabled so no data have been replicated
     verifyNotReplicatedThroughRegion(UTIL2, 0, 100);
 
+    // Ensure that there's no cluster id in remote log entries.
+    verifyNoClusterIdInRemoteLog(UTIL2, PEER_ID);
+
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.DOWNGRADE_ACTIVE);
     // confirm that peer with state DA will reject replication request.
@@ -72,4 +83,25 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
     verifyReplicationRequestRejection(UTIL2, true);
     write(UTIL2, 200, 300);
   }
+
+  private void verifyNoClusterIdInRemoteLog(HBaseTestingUtility utility, String peerId)
+      throws Exception {
+    FileSystem fs2 = utility.getTestFileSystem();
+    Path remoteDir =
+        new Path(utility.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
+            "remoteWALs").makeQualified(fs2.getUri(), fs2.getWorkingDirectory());
+    FileStatus[] files = fs2.listStatus(new Path(remoteDir, peerId));
+    Assert.assertTrue(files.length > 0);
+    for (FileStatus file : files) {
+      try (Reader reader =
+          WALFactory.createReader(fs2, file.getPath(), utility.getConfiguration())) {
+        Entry entry = reader.next();
+        Assert.assertTrue(entry != null);
+        while (entry != null) {
+          Assert.assertEquals(entry.getKey().getClusterIds().size(), 0);
+          entry = reader.next();
+        }
+      }
+    }
+  }
 }


[28/36] hbase git commit: HBASE-20163 Forbid major compaction when standby cluster replay the remote wals

Posted by zh...@apache.org.
HBASE-20163 Forbid major compaction when standby cluster replay the remote wals


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

Branch: refs/heads/HBASE-19064
Commit: 327af3305e1a2fcb1561ddc9709701b9b42f9ad2
Parents: 0c793c4
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Apr 12 14:44:25 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 18 ++++++++
 .../hbase/regionserver/HRegionServer.java       |  2 +-
 .../regionserver/RegionServerServices.java      |  5 +++
 .../ForbidMajorCompactionChecker.java           | 44 ++++++++++++++++++++
 .../hadoop/hbase/MockRegionServerServices.java  |  6 +++
 .../hadoop/hbase/master/MockRegionServer.java   |  6 +++
 6 files changed, 80 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/327af330/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 05fb036..42a86c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -144,6 +144,7 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
+import org.apache.hadoop.hbase.regionserver.compactions.ForbidMajorCompactionChecker;
 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector;
@@ -1986,6 +1987,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return compact(compaction, store, throughputController, null);
   }
 
+  private boolean shouldForbidMajorCompaction() {
+    if (rsServices != null && rsServices.getReplicationSourceService() != null) {
+      return rsServices.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
+          .checkState(getRegionInfo(), ForbidMajorCompactionChecker.get());
+    }
+    return false;
+  }
+
   public boolean compact(CompactionContext compaction, HStore store,
       ThroughputController throughputController, User user) throws IOException {
     assert compaction != null && compaction.hasSelection();
@@ -1995,6 +2004,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       store.cancelRequestedCompaction(compaction);
       return false;
     }
+
+    if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) {
+      LOG.warn("Skipping major compaction on " + this
+          + " because this cluster is transiting sync replication state"
+          + " from STANDBY to DOWNGRADE_ACTIVE");
+      store.cancelRequestedCompaction(compaction);
+      return false;
+    }
+
     MonitoredTask status = null;
     boolean requestNeedsCancellation = true;
     /*

http://git-wip-us.apache.org/repos/asf/hbase/blob/327af330/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 51f9fd0..4605280 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
@@ -2469,7 +2469,7 @@ public class HRegionServer extends HasThread implements
    * @return Return the object that implements the replication
    * source executorService.
    */
-  @VisibleForTesting
+  @Override
   public ReplicationSourceService getReplicationSourceService() {
     return replicationSourceHandler;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/327af330/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 00f4077..7c4362c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -262,4 +262,9 @@ public interface RegionServerServices extends Server, MutableOnlineRegions, Favo
    * @return True if cluster is up; false if cluster is not up (we are shutting down).
    */
   boolean isClusterUp();
+
+  /**
+   * @return Return the object that implements the replication source executorService.
+   */
+  ReplicationSourceService getReplicationSourceService();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/327af330/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ForbidMajorCompactionChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ForbidMajorCompactionChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ForbidMajorCompactionChecker.java
new file mode 100644
index 0000000..eecc780
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ForbidMajorCompactionChecker.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.compactions;
+
+import java.util.function.BiPredicate;
+
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Check whether forbid major compaction for region.
+ */
+@InterfaceAudience.Private
+public class ForbidMajorCompactionChecker
+    implements BiPredicate<SyncReplicationState, SyncReplicationState> {
+
+  private static final ForbidMajorCompactionChecker INST = new ForbidMajorCompactionChecker();
+
+  @Override
+  public boolean test(SyncReplicationState state, SyncReplicationState newState) {
+    // Forbid major compaction when cluster transit sync replication state from S to DA
+    return state == SyncReplicationState.STANDBY
+        || newState == SyncReplicationState.DOWNGRADE_ACTIVE;
+  }
+
+  public static ForbidMajorCompactionChecker get() {
+    return INST;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/327af330/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index b6202fd..a550bf5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
 import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
 import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
@@ -353,4 +354,9 @@ public class MockRegionServerServices implements RegionServerServices {
   public boolean isClusterUp() {
     return true;
   }
+
+  @Override
+  public ReplicationSourceService getReplicationSourceService() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/327af330/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 35ad7eb..1c4e72a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
 import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
 import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
@@ -696,4 +697,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   public boolean isClusterUp() {
     return true;
   }
+
+  @Override
+  public ReplicationSourceService getReplicationSourceService() {
+    return null;
+  }
 }


[03/36] hbase git commit: TestWALProcedureStoreOnHDFS failing on branch-1

Posted by zh...@apache.org.
TestWALProcedureStoreOnHDFS failing on branch-1


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

Branch: refs/heads/HBASE-19064
Commit: dcfa01448c1693308256b41fe43aca6df41fd3b8
Parents: c17be2e
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed May 30 13:45:38 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed May 30 13:45:38 2018 -0700

----------------------------------------------------------------------
 .../hbase/master/procedure/TestWALProcedureStoreOnHDFS.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dcfa0144/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
index 99699ff..6f61916 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -92,7 +91,8 @@ public class TestWALProcedureStoreOnHDFS {
     store.recoverLease();
   }
 
-  @After
+  // No @After
+  @SuppressWarnings("JUnit4TearDownNotRun")
   public void tearDown() throws Exception {
     store.stop(false);
     UTIL.getDFSCluster().getFileSystem().delete(store.getWALDir(), true);


[11/36] hbase git commit: HBASE-19079 Support setting up two clusters with A and S stat

Posted by zh...@apache.org.
HBASE-19079 Support setting up two clusters with A and S stat


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

Branch: refs/heads/HBASE-19064
Commit: 0c793c4931c207d9f2a2d41ced4f3dded1dd47c1
Parents: 679a4ac
Author: zhangduo <zh...@apache.org>
Authored: Tue Apr 10 22:35:19 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerManager.java     |   5 +-
 ...ransitPeerSyncReplicationStateProcedure.java |   2 +-
 .../hbase/regionserver/wal/DualAsyncFSWAL.java  |  14 ++
 .../hadoop/hbase/regionserver/wal/WALUtil.java  |  25 ++-
 .../hbase/replication/ChainWALEntryFilter.java  |  28 +--
 .../ReplaySyncReplicationWALCallable.java       |  27 ++-
 .../SyncReplicationPeerInfoProviderImpl.java    |   6 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |  10 +-
 .../hbase/wal/SyncReplicationWALProvider.java   |  94 ++++++---
 .../org/apache/hadoop/hbase/wal/WALEdit.java    |   8 +-
 .../org/apache/hadoop/hbase/wal/WALFactory.java |   2 +-
 .../replication/TestReplicationAdmin.java       |  33 +--
 .../regionserver/wal/TestWALDurability.java     |   2 +
 .../replication/SyncReplicationTestBase.java    | 185 +++++++++++++++++
 .../hbase/replication/TestSyncReplication.java  | 207 -------------------
 .../replication/TestSyncReplicationActive.java  |  64 ++++++
 .../replication/TestSyncReplicationStandBy.java |  96 +++++++++
 17 files changed, 521 insertions(+), 287 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/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 41dd6e3..229549e 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
@@ -68,8 +68,9 @@ public class ReplicationPeerManager {
 
   private final ImmutableMap<SyncReplicationState, EnumSet<SyncReplicationState>>
     allowedTransition = Maps.immutableEnumMap(ImmutableMap.of(SyncReplicationState.ACTIVE,
-      EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE), SyncReplicationState.STANDBY,
-      EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE), SyncReplicationState.DOWNGRADE_ACTIVE,
+      EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE, SyncReplicationState.STANDBY),
+      SyncReplicationState.STANDBY, EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE),
+      SyncReplicationState.DOWNGRADE_ACTIVE,
       EnumSet.of(SyncReplicationState.STANDBY, SyncReplicationState.ACTIVE)));
 
   ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage,

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/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 cc51890..5da2b0c 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
@@ -171,7 +171,7 @@ public class TransitPeerSyncReplicationStateProcedure
         }
         return Flow.HAS_MORE_STATE;
       case REPLAY_REMOTE_WAL_IN_PEER:
-        // TODO: replay remote wal when transiting from S to DA.
+        addChildProcedure(new RecoverStandbyProcedure(peerId));
         setNextState(PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
         return Flow.HAS_MORE_STATE;
       case REOPEN_ALL_REGIONS_IN_PEER:

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/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 0495337..a98567a 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,6 +38,8 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
 
   private final Path remoteWalDir;
 
+  private volatile boolean skipRemoteWal = false;
+
   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,
@@ -51,6 +53,9 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
   @Override
   protected AsyncWriter createWriterInstance(Path path) throws IOException {
     AsyncWriter localWriter = super.createWriterInstance(path);
+    if (skipRemoteWal) {
+      return localWriter;
+    }
     AsyncWriter remoteWriter;
     boolean succ = false;
     try {
@@ -64,4 +69,13 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
     return CombinedAsyncWriter.create(CombinedAsyncWriter.Mode.SEQUENTIAL, remoteWriter,
       localWriter);
   }
+
+  // Allow temporarily skipping the creation of remote writer. When failing to write to the remote
+  // dfs cluster, we need to reopen the regions and switch to use the original wal writer. But we
+  // need to write a close marker when closing a region, and if it fails, the whole rs will abort.
+  // So here we need to skip the creation of remote writer and make it possible to write the region
+  // close marker.
+  public void skipRemoteWal() {
+    this.skipRemoteWal = true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
index 1b17adc..3b18253 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
@@ -20,11 +20,13 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.NavigableMap;
-
+import java.util.function.Function;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
@@ -34,7 +36,9 @@ import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
@@ -179,4 +183,23 @@ public class WALUtil {
     return conf.getLong("hbase.regionserver.hlog.blocksize",
         CommonFSUtils.getDefaultBlockSize(fs, dir) * 2);
   }
+
+  public static void filterCells(WALEdit edit, Function<Cell, Cell> mapper) {
+    ArrayList<Cell> cells = edit.getCells();
+    int size = cells.size();
+    int newSize = 0;
+    for (int i = 0; i < size; i++) {
+      Cell cell = mapper.apply(cells.get(i));
+      if (cell != null) {
+        cells.set(newSize, cell);
+        newSize++;
+      }
+    }
+    for (int i = size - 1; i >= newSize; i--) {
+      cells.remove(i);
+    }
+    if (newSize < size / 2) {
+      cells.trimToSize();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java
index 6f2c764..2bb9811 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java
@@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.replication;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A {@link WALEntryFilter} which contains multiple filters and applies them
@@ -82,22 +82,16 @@ public class ChainWALEntryFilter implements WALEntryFilter {
     if (entry == null || cellFilters.length == 0) {
       return;
     }
-    ArrayList<Cell> cells = entry.getEdit().getCells();
-    int size = cells.size();
-    for (int i = size - 1; i >= 0; i--) {
-      Cell cell = cells.get(i);
-      for (WALCellFilter filter : cellFilters) {
-        cell = filter.filterCell(entry, cell);
-        if (cell != null) {
-          cells.set(i, cell);
-        } else {
-          cells.remove(i);
-          break;
-        }
+    WALUtil.filterCells(entry.getEdit(), c -> filterCell(entry, c));
+  }
+
+  private Cell filterCell(Entry entry, Cell cell) {
+    for (WALCellFilter filter : cellFilters) {
+      cell = filter.filterCell(entry, cell);
+      if (cell == null) {
+        break;
       }
     }
-    if (cells.size() < size / 2) {
-      cells.trimToSize();
-    }
+    return cell;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
index 8dfe3a2..c9c5ef6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
@@ -21,21 +21,23 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.util.ArrayList;
 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.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -129,20 +131,31 @@ public class ReplaySyncReplicationWALCallable implements RSProcedureCallable {
     }
   }
 
+  // return whether we should include this entry.
+  private boolean filter(Entry entry) {
+    WALEdit edit = entry.getEdit();
+    WALUtil.filterCells(edit, c -> CellUtil.matchingFamily(c, WALEdit.METAFAMILY) ? null : c);
+    return !edit.isEmpty();
+  }
+
   private List<Entry> readWALEntries(Reader reader) throws IOException {
     List<Entry> entries = new ArrayList<>();
     if (reader == null) {
       return entries;
     }
     long size = 0;
-    Entry entry = reader.next();
-    while (entry != null) {
-      entries.add(entry);
-      size += entry.getEdit().heapSize();
-      if (size > batchSize) {
+    for (;;) {
+      Entry entry = reader.next();
+      if (entry == null) {
         break;
       }
-      entry = reader.next();
+      if (filter(entry)) {
+        entries.add(entry);
+        size += entry.getEdit().heapSize();
+        if (size > batchSize) {
+          break;
+        }
+      }
     }
     return entries;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
index e4afc33..cb33dab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
@@ -54,8 +54,10 @@ class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProv
     }
     Pair<SyncReplicationState, SyncReplicationState> states =
         peer.getSyncReplicationStateAndNewState();
-    if (states.getFirst() == SyncReplicationState.ACTIVE &&
-      states.getSecond() == SyncReplicationState.NONE) {
+    if ((states.getFirst() == SyncReplicationState.ACTIVE &&
+      states.getSecond() == SyncReplicationState.NONE) ||
+      (states.getFirst() == SyncReplicationState.DOWNGRADE_ACTIVE &&
+        states.getSecond() == SyncReplicationState.ACTIVE)) {
       return Optional.of(Pair.newPair(peerId, peer.getPeerConfig().getRemoteWALDir()));
     } else {
       return Optional.empty();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/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 3eb8f8f..5a3fba3 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
@@ -136,8 +136,16 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
         walCopy = wal;
         if (walCopy == null) {
           walCopy = createWAL();
+          boolean succ = false;
+          try {
+            walCopy.init();
+            succ = true;
+          } finally {
+            if (!succ) {
+              walCopy.close();
+            }
+          }
           wal = walCopy;
-          walCopy.init();
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/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
index 54287fe..9cbb095 100644
--- 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
@@ -69,7 +69,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   private final WALProvider provider;
 
   private SyncReplicationPeerInfoProvider peerInfoProvider =
-      new DefaultSyncReplicationPeerInfoProvider();
+    new DefaultSyncReplicationPeerInfoProvider();
 
   private WALFactory factory;
 
@@ -83,7 +83,11 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
 
   private AtomicBoolean initialized = new AtomicBoolean(false);
 
-  private final ConcurrentMap<String, DualAsyncFSWAL> peerId2WAL = new ConcurrentHashMap<>();
+  // when switching from A to DA, we will put a Optional.empty into this map if there is no WAL for
+  // the peer yet. When getting WAL from this map the caller should know that it should not use
+  // DualAsyncFSWAL any more.
+  private final ConcurrentMap<String, Optional<DualAsyncFSWAL>> peerId2WAL =
+    new ConcurrentHashMap<>();
 
   private final KeyLocker<String> createLock = new KeyLocker<>();
 
@@ -123,18 +127,27 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   }
 
   private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException {
-    DualAsyncFSWAL wal = peerId2WAL.get(peerId);
-    if (wal != null) {
-      return wal;
+    Optional<DualAsyncFSWAL> opt = peerId2WAL.get(peerId);
+    if (opt != null) {
+      return opt.orElse(null);
     }
     Lock lock = createLock.acquireLock(peerId);
     try {
-      wal = peerId2WAL.get(peerId);
-      if (wal == null) {
-        wal = createWAL(peerId, remoteWALDir);
-        peerId2WAL.put(peerId, wal);
+      opt = peerId2WAL.get(peerId);
+      if (opt != null) {
+        return opt.orElse(null);
+      }
+      DualAsyncFSWAL wal = createWAL(peerId, remoteWALDir);
+      boolean succ = false;
+      try {
         wal.init();
+        succ = true;
+      } finally {
+        if (!succ) {
+          wal.close();
+        }
       }
+      peerId2WAL.put(peerId, Optional.of(wal));
       return wal;
     } finally {
       lock.unlock();
@@ -146,18 +159,20 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
     if (region == null) {
       return provider.getWAL(null);
     }
+    WAL wal = null;
     Optional<Pair<String, String>> peerIdAndRemoteWALDir =
       peerInfoProvider.getPeerIdAndRemoteWALDir(region);
     if (peerIdAndRemoteWALDir.isPresent()) {
       Pair<String, String> pair = peerIdAndRemoteWALDir.get();
-      return getWAL(pair.getFirst(), pair.getSecond());
-    } else {
-      return provider.getWAL(region);
+      wal = getWAL(pair.getFirst(), pair.getSecond());
     }
+    return wal != null ? wal : provider.getWAL(region);
   }
 
   private Stream<WAL> getWALStream() {
-    return Streams.concat(peerId2WAL.values().stream(), provider.getWALs().stream());
+    return Streams.concat(
+      peerId2WAL.values().stream().filter(Optional::isPresent).map(Optional::get),
+      provider.getWALs().stream());
   }
 
   @Override
@@ -169,12 +184,14 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   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;
+    for (Optional<DualAsyncFSWAL> wal : peerId2WAL.values()) {
+      if (wal.isPresent()) {
+        try {
+          wal.get().shutdown();
+        } catch (IOException e) {
+          LOG.error("Shutdown WAL failed", e);
+          failure = e;
+        }
       }
     }
     provider.shutdown();
@@ -187,12 +204,14 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   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;
+    for (Optional<DualAsyncFSWAL> wal : peerId2WAL.values()) {
+      if (wal.isPresent()) {
+        try {
+          wal.get().close();
+        } catch (IOException e) {
+          LOG.error("Close WAL failed", e);
+          failure = e;
+        }
       }
     }
     provider.close();
@@ -208,8 +227,8 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
 
   @Override
   public long getLogFileSize() {
-    return peerId2WAL.values().stream().mapToLong(DualAsyncFSWAL::getLogFileSize).sum() +
-      provider.getLogFileSize();
+    return peerId2WAL.values().stream().filter(Optional::isPresent).map(Optional::get)
+      .mapToLong(DualAsyncFSWAL::getLogFileSize).sum() + provider.getLogFileSize();
   }
 
   private void safeClose(WAL wal) {
@@ -231,10 +250,23 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   @Override
   public void peerSyncReplicationStateChange(String peerId, SyncReplicationState from,
       SyncReplicationState to, int stage) {
-    // TODO: stage 0
-    if (from == SyncReplicationState.ACTIVE && to == SyncReplicationState.DOWNGRADE_ACTIVE &&
-      stage == 1) {
-      safeClose(peerId2WAL.remove(peerId));
+    if (from == SyncReplicationState.ACTIVE && to == SyncReplicationState.DOWNGRADE_ACTIVE) {
+      if (stage == 0) {
+        Lock lock = createLock.acquireLock(peerId);
+        try {
+          Optional<DualAsyncFSWAL> opt = peerId2WAL.get(peerId);
+          if (opt != null) {
+            opt.ifPresent(DualAsyncFSWAL::skipRemoteWal);
+          } else {
+            // add a place holder to tell the getWAL caller do not use DualAsyncFSWAL any more.
+            peerId2WAL.put(peerId, Optional.empty());
+          }
+        } finally {
+          lock.unlock();
+        }
+      } else if (stage == 1) {
+        peerId2WAL.remove(peerId).ifPresent(this::safeClose);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
index 1d4dc1b..cd0e52e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
@@ -20,12 +20,11 @@ package org.apache.hadoop.hbase.wal;
 
 import java.io.IOException;
 import java.util.ArrayList;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.io.HeapSize;
@@ -33,9 +32,9 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
@@ -54,7 +53,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe
 @InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.REPLICATION,
     HBaseInterfaceAudience.COPROC })
 public class WALEdit implements HeapSize {
-  private static final Logger LOG = LoggerFactory.getLogger(WALEdit.class);
 
   // TODO: Get rid of this; see HBASE-8457
   public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/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 afe043f..4f6a898 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
@@ -135,7 +135,7 @@ public class WALFactory {
   static WALProvider createProvider(Class<? extends WALProvider> clazz) throws IOException {
     LOG.info("Instantiating WALProvider of type {}", clazz);
     try {
-      return clazz.newInstance();
+      return clazz.getDeclaredConstructor().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);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/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 486ab51..ac98283 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
@@ -35,6 +35,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -259,9 +260,11 @@ public class TestReplicationAdmin {
     TEST_UTIL.createTable(tableName, Bytes.toBytes("family"));
     ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
 
-    String rootDir = "hdfs://srv1:9999/hbase";
+    Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("remoteWAL");
+    TEST_UTIL.getTestFileSystem().mkdirs(new Path(rootDir, ID_ONE));
     builder.setClusterKey(KEY_ONE);
-    builder.setRemoteWALDir(rootDir);
+    builder.setRemoteWALDir(rootDir.makeQualified(TEST_UTIL.getTestFileSystem().getUri(),
+      TEST_UTIL.getTestFileSystem().getWorkingDirectory()).toString());
     builder.setReplicateAllUserTables(false);
     Map<TableName, List<String>> tableCfs = new HashMap<>();
     tableCfs.put(tableName, new ArrayList<>());
@@ -1081,10 +1084,12 @@ public class TestReplicationAdmin {
       // OK
     }
 
-    String rootDir = "hdfs://srv1:9999/hbase";
+    Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("remoteWAL");
+    TEST_UTIL.getTestFileSystem().mkdirs(new Path(rootDir, ID_SECOND));
     builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_SECOND);
-    builder.setRemoteWALDir(rootDir);
+    builder.setRemoteWALDir(rootDir.makeQualified(TEST_UTIL.getTestFileSystem().getUri(),
+      TEST_UTIL.getTestFileSystem().getWorkingDirectory()).toString());
     builder.setReplicateAllUserTables(false);
     Map<TableName, List<String>> tableCfs = new HashMap<>();
     tableCfs.put(tableName, new ArrayList<>());
@@ -1105,13 +1110,18 @@ public class TestReplicationAdmin {
     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.STANDBY);
+    assertEquals(SyncReplicationState.STANDBY,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
+    assertEquals(SyncReplicationState.ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
 
     hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND,
       SyncReplicationState.DOWNGRADE_ACTIVE);
@@ -1121,7 +1131,6 @@ public class TestReplicationAdmin {
     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");

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
index 17f24e8..c446306 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
@@ -104,6 +104,7 @@ public class TestWALDurability {
     FileSystem fs = FileSystem.get(conf);
     Path rootDir = new Path(dir + getName());
     CustomFSLog customFSLog = new CustomFSLog(fs, rootDir, getName(), conf);
+    customFSLog.init();
     HRegion region = initHRegion(tableName, null, null, customFSLog);
     byte[] bytes = Bytes.toBytes(getName());
     Put put = new Put(bytes);
@@ -118,6 +119,7 @@ public class TestWALDurability {
     conf.set(HRegion.WAL_HSYNC_CONF_KEY, "true");
     fs = FileSystem.get(conf);
     customFSLog = new CustomFSLog(fs, rootDir, getName(), conf);
+    customFSLog.init();
     region = initHRegion(tableName, null, null, customFSLog);
 
     customFSLog.resetSyncFlag();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
new file mode 100644
index 0000000..30dbdb5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
+/**
+ * Base class for testing sync replication.
+ */
+public class SyncReplicationTestBase {
+
+  protected static final HBaseZKTestingUtility ZK_UTIL = new HBaseZKTestingUtility();
+
+  protected static final HBaseTestingUtility UTIL1 = new HBaseTestingUtility();
+
+  protected static final HBaseTestingUtility UTIL2 = new HBaseTestingUtility();
+
+  protected static TableName TABLE_NAME = TableName.valueOf("SyncRep");
+
+  protected static byte[] CF = Bytes.toBytes("cf");
+
+  protected static byte[] CQ = Bytes.toBytes("cq");
+
+  protected static String PEER_ID = "1";
+
+  private static void initTestingUtility(HBaseTestingUtility util, String zkParent) {
+    util.setZkCluster(ZK_UTIL.getZkCluster());
+    Configuration conf = util.getConfiguration();
+    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkParent);
+    conf.setInt("replication.source.size.capacity", 102400);
+    conf.setLong("replication.source.sleepforretries", 100);
+    conf.setInt("hbase.regionserver.maxlogs", 10);
+    conf.setLong("hbase.master.logcleaner.ttl", 10);
+    conf.setInt("zookeeper.recovery.retry", 1);
+    conf.setInt("zookeeper.recovery.retry.intervalmill", 10);
+    conf.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
+    conf.setInt("replication.stats.thread.period.seconds", 5);
+    conf.setBoolean("hbase.tests.use.shortcircuit.reads", false);
+    conf.setLong("replication.sleep.before.failover", 2000);
+    conf.setInt("replication.source.maxretriesmultiplier", 10);
+    conf.setFloat("replication.source.ratio", 1.0f);
+    conf.setBoolean("replication.source.eof.autorecovery", true);
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    ZK_UTIL.startMiniZKCluster();
+    initTestingUtility(UTIL1, "/cluster1");
+    initTestingUtility(UTIL2, "/cluster2");
+    UTIL1.startMiniCluster(3);
+    UTIL2.startMiniCluster(3);
+    TableDescriptor td =
+      TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build();
+    UTIL1.getAdmin().createTable(td);
+    UTIL2.getAdmin().createTable(td);
+    FileSystem fs1 = UTIL1.getTestFileSystem();
+    FileSystem fs2 = UTIL2.getTestFileSystem();
+    Path remoteWALDir1 =
+      new Path(UTIL1.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
+        "remoteWALs").makeQualified(fs1.getUri(), fs1.getWorkingDirectory());
+    Path remoteWALDir2 =
+      new Path(UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
+        "remoteWALs").makeQualified(fs2.getUri(), fs2.getWorkingDirectory());
+    UTIL1.getAdmin().addReplicationPeer(PEER_ID,
+      ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey())
+        .setReplicateAllUserTables(false)
+        .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>()))
+        .setRemoteWALDir(remoteWALDir2.toUri().toString()).build());
+    UTIL2.getAdmin().addReplicationPeer(PEER_ID,
+      ReplicationPeerConfig.newBuilder().setClusterKey(UTIL1.getClusterKey())
+        .setReplicateAllUserTables(false)
+        .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>()))
+        .setRemoteWALDir(remoteWALDir1.toUri().toString()).build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL1.shutdownMiniCluster();
+    UTIL2.shutdownMiniCluster();
+    ZK_UTIL.shutdownMiniZKCluster();
+  }
+
+  protected final void write(HBaseTestingUtility util, int start, int end) throws IOException {
+    try (Table table = util.getConnection().getTable(TABLE_NAME)) {
+      for (int i = start; i < end; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+  }
+
+  protected final void verify(HBaseTestingUtility util, int start, int end) throws IOException {
+    try (Table table = util.getConnection().getTable(TABLE_NAME)) {
+      for (int i = start; i < end; i++) {
+        assertEquals(i, Bytes.toInt(table.get(new Get(Bytes.toBytes(i))).getValue(CF, CQ)));
+      }
+    }
+  }
+
+  protected final void verifyThroughRegion(HBaseTestingUtility util, int start, int end)
+      throws IOException {
+    HRegion region = util.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    for (int i = start; i < end; i++) {
+      assertEquals(i, Bytes.toInt(region.get(new Get(Bytes.toBytes(i))).getValue(CF, CQ)));
+    }
+  }
+
+  protected final void verifyNotReplicatedThroughRegion(HBaseTestingUtility util, int start,
+      int end) throws IOException {
+    HRegion region = util.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    for (int i = start; i < end; i++) {
+      assertTrue(region.get(new Get(Bytes.toBytes(i))).isEmpty());
+    }
+  }
+
+  protected final void waitUntilReplicationDone(HBaseTestingUtility util, int end)
+      throws Exception {
+    // The reject check is in RSRpcService so we can still read through HRegion
+    HRegion region = util.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    util.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return !region.get(new Get(Bytes.toBytes(end - 1))).isEmpty();
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Replication has not been catched up yet";
+      }
+    });
+  }
+
+  protected final void writeAndVerifyReplication(HBaseTestingUtility util1,
+      HBaseTestingUtility util2, int start, int end) throws Exception {
+    write(util1, start, end);
+    waitUntilReplicationDone(util2, end);
+    verifyThroughRegion(util2, start, end);
+  }
+
+  protected final Path getRemoteWALDir(MasterFileSystem mfs, String peerId) {
+    Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
+    return new Path(remoteWALDir, PEER_ID);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
deleted file mode 100644
index 288dcbf..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HBaseZKTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
-
-@Category({ ReplicationTests.class, LargeTests.class })
-public class TestSyncReplication {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestSyncReplication.class);
-
-  private static final HBaseZKTestingUtility ZK_UTIL = new HBaseZKTestingUtility();
-
-  private static final HBaseTestingUtility UTIL1 = new HBaseTestingUtility();
-
-  private static final HBaseTestingUtility UTIL2 = new HBaseTestingUtility();
-
-  private static TableName TABLE_NAME = TableName.valueOf("SyncRep");
-
-  private static byte[] CF = Bytes.toBytes("cf");
-
-  private static byte[] CQ = Bytes.toBytes("cq");
-
-  private static String PEER_ID = "1";
-
-  private static void initTestingUtility(HBaseTestingUtility util, String zkParent) {
-    util.setZkCluster(ZK_UTIL.getZkCluster());
-    Configuration conf = util.getConfiguration();
-    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkParent);
-    conf.setInt("replication.source.size.capacity", 102400);
-    conf.setLong("replication.source.sleepforretries", 100);
-    conf.setInt("hbase.regionserver.maxlogs", 10);
-    conf.setLong("hbase.master.logcleaner.ttl", 10);
-    conf.setInt("zookeeper.recovery.retry", 1);
-    conf.setInt("zookeeper.recovery.retry.intervalmill", 10);
-    conf.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
-    conf.setInt("replication.stats.thread.period.seconds", 5);
-    conf.setBoolean("hbase.tests.use.shortcircuit.reads", false);
-    conf.setLong("replication.sleep.before.failover", 2000);
-    conf.setInt("replication.source.maxretriesmultiplier", 10);
-    conf.setFloat("replication.source.ratio", 1.0f);
-    conf.setBoolean("replication.source.eof.autorecovery", true);
-  }
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    ZK_UTIL.startMiniZKCluster();
-    initTestingUtility(UTIL1, "/cluster1");
-    initTestingUtility(UTIL2, "/cluster2");
-    UTIL1.startMiniCluster(3);
-    UTIL2.startMiniCluster(3);
-    TableDescriptor td =
-        TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(ColumnFamilyDescriptorBuilder
-          .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build();
-    UTIL1.getAdmin().createTable(td);
-    UTIL2.getAdmin().createTable(td);
-    FileSystem fs1 = UTIL1.getTestFileSystem();
-    FileSystem fs2 = UTIL2.getTestFileSystem();
-    Path remoteWALDir1 =
-        new Path(UTIL1.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
-          "remoteWALs").makeQualified(fs1.getUri(), fs1.getWorkingDirectory());
-    Path remoteWALDir2 =
-        new Path(UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
-          "remoteWALs").makeQualified(fs2.getUri(), fs2.getWorkingDirectory());
-    UTIL1.getAdmin().addReplicationPeer(PEER_ID,
-      ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey())
-        .setReplicateAllUserTables(false)
-        .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>()))
-        .setRemoteWALDir(remoteWALDir2.toUri().toString()).build());
-    UTIL2.getAdmin().addReplicationPeer(PEER_ID,
-      ReplicationPeerConfig.newBuilder().setClusterKey(UTIL1.getClusterKey())
-        .setReplicateAllUserTables(false)
-        .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>()))
-        .setRemoteWALDir(remoteWALDir1.toUri().toString()).build());
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    UTIL1.shutdownMiniCluster();
-    UTIL2.shutdownMiniCluster();
-    ZK_UTIL.shutdownMiniZKCluster();
-  }
-
-  @FunctionalInterface
-  private interface TableAction {
-
-    void call(Table table) throws IOException;
-  }
-
-  private void assertDisallow(Table table, TableAction action) throws IOException {
-    try {
-      action.call(table);
-    } catch (DoNotRetryIOException | RetriesExhaustedException e) {
-      // expected
-      assertThat(e.getMessage(), containsString("STANDBY"));
-    }
-  }
-
-  @Test
-  public void testStandby() throws Exception {
-    MasterFileSystem mfs = UTIL2.getHBaseCluster().getMaster().getMasterFileSystem();
-    Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
-    Path remoteWALDirForPeer = new Path(remoteWALDir, PEER_ID);
-    assertFalse(mfs.getWALFileSystem().exists(remoteWALDirForPeer));
-    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
-      SyncReplicationState.STANDBY);
-    assertTrue(mfs.getWALFileSystem().exists(remoteWALDirForPeer));
-    try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
-      assertDisallow(table, t -> t.get(new Get(Bytes.toBytes("row"))));
-      assertDisallow(table,
-        t -> t.put(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row"))));
-      assertDisallow(table, t -> t.delete(new Delete(Bytes.toBytes("row"))));
-      assertDisallow(table, t -> t.incrementColumnValue(Bytes.toBytes("row"), CF, CQ, 1));
-      assertDisallow(table,
-        t -> t.append(new Append(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row"))));
-      assertDisallow(table,
-        t -> t.get(Arrays.asList(new Get(Bytes.toBytes("row")), new Get(Bytes.toBytes("row1")))));
-      assertDisallow(table,
-        t -> t
-          .put(Arrays.asList(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")),
-            new Put(Bytes.toBytes("row1")).addColumn(CF, CQ, Bytes.toBytes("row1")))));
-      assertDisallow(table, t -> t.mutateRow(new RowMutations(Bytes.toBytes("row"))
-        .add((Mutation) new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")))));
-    }
-    // But we should still allow replication writes
-    try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) {
-      for (int i = 0; i < 100; i++) {
-        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
-      }
-    }
-    // The reject check is in RSRpcService so we can still read through HRegion
-    HRegion region = UTIL2.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
-    UTIL2.waitFor(30000, new ExplainingPredicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return !region.get(new Get(Bytes.toBytes(99))).isEmpty();
-      }
-
-      @Override
-      public String explainFailure() throws Exception {
-        return "Replication has not been catched up yet";
-      }
-    });
-    for (int i = 0; i < 100; i++) {
-      assertEquals(i, Bytes.toInt(region.get(new Get(Bytes.toBytes(i))).getValue(CF, CQ)));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
new file mode 100644
index 0000000..f4fb5fe
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSyncReplicationActive extends SyncReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSyncReplicationActive.class);
+
+  @Test
+  public void testActive() throws Exception {
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.ACTIVE);
+    UTIL1.getAdmin().disableReplicationPeer(PEER_ID);
+    write(UTIL1, 0, 100);
+    Thread.sleep(2000);
+    // peer is disabled so no data have been replicated
+    verifyNotReplicatedThroughRegion(UTIL2, 0, 100);
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    // confirm that the data is there after we convert the peer to DA
+    verify(UTIL2, 0, 100);
+
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.ACTIVE);
+
+    writeAndVerifyReplication(UTIL2, UTIL1, 100, 200);
+
+    // shutdown the cluster completely
+    UTIL1.shutdownMiniCluster();
+    // confirm that we can convert to DA even if the remote slave cluster is down
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    write(UTIL2, 200, 300);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0c793c49/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
new file mode 100644
index 0000000..ed61d2a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSyncReplicationStandBy extends SyncReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSyncReplicationStandBy.class);
+
+  @FunctionalInterface
+  private interface TableAction {
+
+    void call(Table table) throws IOException;
+  }
+
+  private void assertDisallow(Table table, TableAction action) throws IOException {
+    try {
+      action.call(table);
+    } catch (DoNotRetryIOException | RetriesExhaustedException e) {
+      // expected
+      assertThat(e.getMessage(), containsString("STANDBY"));
+    }
+  }
+
+  @Test
+  public void testStandby() throws Exception {
+    MasterFileSystem mfs = UTIL2.getHBaseCluster().getMaster().getMasterFileSystem();
+    Path remoteWALDir = getRemoteWALDir(mfs, PEER_ID);
+    assertFalse(mfs.getWALFileSystem().exists(remoteWALDir));
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    assertTrue(mfs.getWALFileSystem().exists(remoteWALDir));
+    try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
+      assertDisallow(table, t -> t.get(new Get(Bytes.toBytes("row"))));
+      assertDisallow(table,
+        t -> t.put(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row"))));
+      assertDisallow(table, t -> t.delete(new Delete(Bytes.toBytes("row"))));
+      assertDisallow(table, t -> t.incrementColumnValue(Bytes.toBytes("row"), CF, CQ, 1));
+      assertDisallow(table,
+        t -> t.append(new Append(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row"))));
+      assertDisallow(table,
+        t -> t.get(Arrays.asList(new Get(Bytes.toBytes("row")), new Get(Bytes.toBytes("row1")))));
+      assertDisallow(table,
+        t -> t
+          .put(Arrays.asList(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")),
+            new Put(Bytes.toBytes("row1")).addColumn(CF, CQ, Bytes.toBytes("row1")))));
+      assertDisallow(table, t -> t.mutateRow(new RowMutations(Bytes.toBytes("row"))
+        .add((Mutation) new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")))));
+    }
+    // We should still allow replication writes
+    writeAndVerifyReplication(UTIL1, UTIL2, 0, 100);
+  }
+}


[25/36] hbase git commit: HBASE-20434 Also remove remote wals when peer is in DA state

Posted by zh...@apache.org.
HBASE-20434 Also remove remote wals when peer is in DA state


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

Branch: refs/heads/HBASE-19064
Commit: 90a0bb25c294d6969f7ccfff73efc35a6bfd9301
Parents: 863566a
Author: zhangduo <zh...@apache.org>
Authored: Wed Apr 25 17:12:23 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationUtils.java     |   4 +
 ...ransitPeerSyncReplicationStateProcedure.java |   2 +-
 .../regionserver/ReplicationSource.java         |   7 +-
 .../regionserver/ReplicationSourceManager.java  |  86 ++++++++++------
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |  19 ++--
 .../hbase/wal/SyncReplicationWALProvider.java   |  30 +++++-
 .../TestSyncReplicationRemoveRemoteWAL.java     | 101 +++++++++++++++++++
 .../TestReplicationSourceManager.java           |  68 ++++++++-----
 8 files changed, 251 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/90a0bb25/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 66e9b01..069db7a 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
@@ -191,6 +191,10 @@ public final class ReplicationUtils {
     return new Path(remoteWALDir, peerId);
   }
 
+  public static Path getRemoteWALDirForPeer(Path remoteWALDir, String peerId) {
+    return new Path(remoteWALDir, peerId);
+  }
+
   /**
    * Do the sleeping logic
    * @param msg Why we sleep

http://git-wip-us.apache.org/repos/asf/hbase/blob/90a0bb25/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 5da2b0c..99fd615 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
@@ -211,7 +211,7 @@ public class TransitPeerSyncReplicationStateProcedure
       case CREATE_DIR_FOR_REMOTE_WAL:
         MasterFileSystem mfs = env.getMasterFileSystem();
         Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
-        Path remoteWALDirForPeer = new Path(remoteWALDir, peerId);
+        Path remoteWALDirForPeer = ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId);
         FileSystem walFs = mfs.getWALFileSystem();
         try {
           if (walFs.exists(remoteWALDirForPeer)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/90a0bb25/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index ba665b6..c669622 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -550,14 +550,17 @@ public class ReplicationSource implements ReplicationSourceInterface {
     }
 
     /**
+     * <p>
      * Split a path to get the start time
+     * </p>
+     * <p>
      * For example: 10.20.20.171%3A60020.1277499063250
+     * </p>
      * @param p path to split
      * @return start time
      */
     private static long getTS(Path p) {
-      int tsIndex = p.getName().lastIndexOf('.') + 1;
-      return Long.parseLong(p.getName().substring(tsIndex));
+      return AbstractFSWALProvider.getWALStartTimeFromWALName(p.getName());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/90a0bb25/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 2d0d82b..5015129 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -61,6 +62,7 @@ import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -542,20 +544,40 @@ public class ReplicationSourceManager implements ReplicationListener {
     if (source.isRecovered()) {
       NavigableSet<String> wals = walsByIdRecoveredQueues.get(source.getQueueId()).get(logPrefix);
       if (wals != null) {
-        cleanOldLogs(wals, log, inclusive, source);
+        NavigableSet<String> walsToRemove = wals.headSet(log, inclusive);
+        if (walsToRemove.isEmpty()) {
+          return;
+        }
+        cleanOldLogs(walsToRemove, source);
+        walsToRemove.clear();
       }
     } else {
+      NavigableSet<String> wals;
+      NavigableSet<String> walsToRemove;
       // synchronized on walsById to avoid race with preLogRoll
       synchronized (this.walsById) {
-        NavigableSet<String> wals = walsById.get(source.getQueueId()).get(logPrefix);
-        if (wals != null) {
-          cleanOldLogs(wals, log, inclusive, source);
+        wals = walsById.get(source.getQueueId()).get(logPrefix);
+        if (wals == null) {
+          return;
+        }
+        walsToRemove = wals.headSet(log, inclusive);
+        if (walsToRemove.isEmpty()) {
+          return;
         }
+        walsToRemove = new TreeSet<>(walsToRemove);
+      }
+      // cleanOldLogs may spend some time, especially for sync replication where we may want to
+      // remove remote wals as the remote cluster may have already been down, so we do it outside
+      // the lock to avoid block preLogRoll
+      cleanOldLogs(walsToRemove, source);
+      // now let's remove the files in the set
+      synchronized (this.walsById) {
+        wals.removeAll(walsToRemove);
       }
     }
   }
 
-  private void removeRemoteWALs(String peerId, String remoteWALDir, Set<String> wals)
+  private void removeRemoteWALs(String peerId, String remoteWALDir, Collection<String> wals)
       throws IOException {
     Path remoteWALDirForPeer = ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId);
     FileSystem fs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
@@ -575,13 +597,8 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
   }
 
-  private void cleanOldLogs(NavigableSet<String> wals, String key, boolean inclusive,
-      ReplicationSourceInterface source) {
-    NavigableSet<String> walSet = wals.headSet(key, inclusive);
-    if (walSet.isEmpty()) {
-      return;
-    }
-    LOG.debug("Removing {} logs in the list: {}", walSet.size(), walSet);
+  private void cleanOldLogs(NavigableSet<String> wals, ReplicationSourceInterface source) {
+    LOG.debug("Removing {} logs in the list: {}", wals.size(), wals);
     // The intention here is that, we want to delete the remote wal files ASAP as it may effect the
     // failover time if you want to transit the remote cluster from S to A. And the infinite retry
     // is not a problem, as if we can not contact with the remote HDFS cluster, then usually we can
@@ -589,31 +606,38 @@ public class ReplicationSourceManager implements ReplicationListener {
     if (source.isSyncReplication()) {
       String peerId = source.getPeerId();
       String remoteWALDir = source.getPeer().getPeerConfig().getRemoteWALDir();
-      LOG.debug("Removing {} logs from remote dir {} in the list: {}", walSet.size(), remoteWALDir,
-        walSet);
-      for (int sleepMultiplier = 0;;) {
-        try {
-          removeRemoteWALs(peerId, remoteWALDir, walSet);
-          break;
-        } catch (IOException e) {
-          LOG.warn("Failed to delete remote wals from remote dir {} for peer {}", remoteWALDir,
-            peerId);
-        }
-        if (!source.isSourceActive()) {
-          // skip the following operations
-          return;
-        }
-        if (ReplicationUtils.sleepForRetries("Failed to delete remote wals", sleepForRetries,
-          sleepMultiplier, maxRetriesMultiplier)) {
-          sleepMultiplier++;
+      // Filter out the wals need to be removed from the remote directory. Its name should be the
+      // special format, and also, the peer id in its name should match the peer id for the
+      // replication source.
+      List<String> remoteWals = wals.stream().filter(w -> SyncReplicationWALProvider
+        .getSyncReplicationPeerIdFromWALName(w).map(peerId::equals).orElse(false))
+        .collect(Collectors.toList());
+      LOG.debug("Removing {} logs from remote dir {} in the list: {}", remoteWals.size(),
+        remoteWALDir, remoteWals);
+      if (!remoteWals.isEmpty()) {
+        for (int sleepMultiplier = 0;;) {
+          try {
+            removeRemoteWALs(peerId, remoteWALDir, remoteWals);
+            break;
+          } catch (IOException e) {
+            LOG.warn("Failed to delete remote wals from remote dir {} for peer {}", remoteWALDir,
+              peerId);
+          }
+          if (!source.isSourceActive()) {
+            // skip the following operations
+            return;
+          }
+          if (ReplicationUtils.sleepForRetries("Failed to delete remote wals", sleepForRetries,
+            sleepMultiplier, maxRetriesMultiplier)) {
+            sleepMultiplier++;
+          }
         }
       }
     }
     String queueId = source.getQueueId();
-    for (String wal : walSet) {
+    for (String wal : wals) {
       abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), queueId, wal));
     }
-    walSet.clear();
   }
 
   // public because of we call it in TestReplicationEmptyWALRecovery

http://git-wip-us.apache.org/repos/asf/hbase/blob/90a0bb25/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 e528624..ccdc95f 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
@@ -517,6 +517,14 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
     listeners.add(listener);
   }
 
+  private static String getWALNameGroupFromWALName(String name, int group) {
+    Matcher matcher = WAL_FILE_NAME_PATTERN.matcher(name);
+    if (matcher.matches()) {
+      return matcher.group(group);
+    } else {
+      throw new IllegalArgumentException(name + " is not a valid wal file name");
+    }
+  }
   /**
    * Get prefix of the log from its name, assuming WAL name in format of
    * log_prefix.filenumber.log_suffix
@@ -526,11 +534,10 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
    * @see AbstractFSWAL#getCurrentFileName()
    */
   public static String getWALPrefixFromWALName(String name) {
-    Matcher matcher = WAL_FILE_NAME_PATTERN.matcher(name);
-    if (matcher.matches()) {
-      return matcher.group(1);
-    } else {
-      throw new IllegalArgumentException(name + " is not a valid wal file name");
-    }
+    return getWALNameGroupFromWALName(name, 1);
+  }
+
+  public static long getWALStartTimeFromWALName(String name) {
+    return Long.parseLong(getWALNameGroupFromWALName(name, 2));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/90a0bb25/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
index 8faccd7..8e82d8b 100644
--- 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
@@ -29,6 +29,8 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Lock;
 import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
@@ -48,6 +50,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Streams;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
@@ -64,7 +67,8 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
 
   private static final Logger LOG = LoggerFactory.getLogger(SyncReplicationWALProvider.class);
 
-  private static final String LOG_SUFFIX = ".syncrep";
+  @VisibleForTesting
+  public static final String LOG_SUFFIX = ".syncrep";
 
   private final WALProvider provider;
 
@@ -288,4 +292,28 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
       return false;
     }
   }
+
+  private static final Pattern LOG_PREFIX_PATTERN = Pattern.compile(".*-\\d+-(.+)");
+
+  /**
+   * <p>
+   * Returns the peer id if the wal file name is in the special group for a sync replication peer.
+   * </p>
+   * <p>
+   * The prefix format is &lt;factoryId&gt;-&lt;ts&gt;-&lt;peerId&gt;.
+   * </p>
+   */
+  public static Optional<String> getSyncReplicationPeerIdFromWALName(String name) {
+    if (!name.endsWith(LOG_SUFFIX)) {
+      // fast path to return earlier if the name is not for a sync replication peer.
+      return Optional.empty();
+    }
+    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(name);
+    Matcher matcher = LOG_PREFIX_PATTERN.matcher(logPrefix);
+    if (matcher.matches()) {
+      return Optional.of(matcher.group(1));
+    } else {
+      return Optional.empty();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/90a0bb25/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java
new file mode 100644
index 0000000..7d380c1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.hamcrest.CoreMatchers.endsWith;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSyncReplicationRemoveRemoteWAL extends SyncReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSyncReplicationRemoveRemoteWAL.class);
+
+  private void waitUntilDeleted(Path remoteWAL) throws Exception {
+    MasterFileSystem mfs = UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem();
+    UTIL1.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return !mfs.getWALFileSystem().exists(remoteWAL);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return remoteWAL + " has not been deleted yet";
+      }
+    });
+  }
+
+  @Test
+  public void testRemoveRemoteWAL() throws Exception {
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.ACTIVE);
+
+    MasterFileSystem mfs = UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem();
+    Path remoteWALDir = ReplicationUtils.getRemoteWALDirForPeer(
+      new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME), PEER_ID);
+    FileStatus[] remoteWALStatus = mfs.getWALFileSystem().listStatus(remoteWALDir);
+    assertEquals(1, remoteWALStatus.length);
+    Path remoteWAL = remoteWALStatus[0].getPath();
+    assertThat(remoteWAL.getName(), endsWith(SyncReplicationWALProvider.LOG_SUFFIX));
+    writeAndVerifyReplication(UTIL1, UTIL2, 0, 100);
+
+    HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME);
+    rs.getWalRoller().requestRollAll();
+    // The replicated wal file should be deleted finally
+    waitUntilDeleted(remoteWAL);
+    remoteWALStatus = mfs.getWALFileSystem().listStatus(remoteWALDir);
+    assertEquals(1, remoteWALStatus.length);
+    remoteWAL = remoteWALStatus[0].getPath();
+    assertThat(remoteWAL.getName(), endsWith(SyncReplicationWALProvider.LOG_SUFFIX));
+
+    UTIL1.getAdmin().disableReplicationPeer(PEER_ID);
+    write(UTIL1, 100, 200);
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+
+    // should still be there since the peer is disabled and we haven't replicated the data yet
+    assertTrue(mfs.getWALFileSystem().exists(remoteWAL));
+
+    UTIL1.getAdmin().enableReplicationPeer(PEER_ID);
+    waitUntilReplicationDone(UTIL2, 200);
+    verifyThroughRegion(UTIL2, 100, 200);
+
+    // Confirm that we will also remove the remote wal files in DA state
+    waitUntilDeleted(remoteWAL);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/90a0bb25/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 cff8ceb..d98b7f85 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
@@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -592,27 +593,10 @@ public abstract class TestReplicationSourceManager {
     }
   }
 
-  @Test
-  public void testRemoveRemoteWALs() throws IOException {
-    // make sure that we can deal with files which does not exist
-    String walNameNotExists = "remoteWAL.0";
-    Path wal = new Path(logDir, walNameNotExists);
-    manager.preLogRoll(wal);
-    manager.postLogRoll(wal);
-
-    Path remoteLogDirForPeer = new Path(remoteLogDir, slaveId);
-    fs.mkdirs(remoteLogDirForPeer);
-    String walName = "remoteWAL.1";
-    Path remoteWAL =
-      new Path(remoteLogDirForPeer, walName).makeQualified(fs.getUri(), fs.getWorkingDirectory());
-    fs.create(remoteWAL).close();
-    wal = new Path(logDir, walName);
-    manager.preLogRoll(wal);
-    manager.postLogRoll(wal);
-
+  private ReplicationSourceInterface mockReplicationSource(String peerId) {
     ReplicationSourceInterface source = mock(ReplicationSourceInterface.class);
-    when(source.getPeerId()).thenReturn(slaveId);
-    when(source.getQueueId()).thenReturn(slaveId);
+    when(source.getPeerId()).thenReturn(peerId);
+    when(source.getQueueId()).thenReturn(peerId);
     when(source.isRecovered()).thenReturn(false);
     when(source.isSyncReplication()).thenReturn(true);
     ReplicationPeerConfig config = mock(ReplicationPeerConfig.class);
@@ -621,17 +605,51 @@ public abstract class TestReplicationSourceManager {
     ReplicationPeer peer = mock(ReplicationPeer.class);
     when(peer.getPeerConfig()).thenReturn(config);
     when(source.getPeer()).thenReturn(peer);
-    manager.cleanOldLogs(walName, true, source);
+    return source;
+  }
 
-    assertFalse(fs.exists(remoteWAL));
+  @Test
+  public void testRemoveRemoteWALs() throws Exception {
+    String peerId2 = slaveId + "_2";
+    addPeerAndWait(peerId2,
+      ReplicationPeerConfig.newBuilder()
+        .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase").build(),
+      true);
+    try {
+      // make sure that we can deal with files which does not exist
+      String walNameNotExists =
+        "remoteWAL-12345-" + slaveId + ".12345" + SyncReplicationWALProvider.LOG_SUFFIX;
+      Path wal = new Path(logDir, walNameNotExists);
+      manager.preLogRoll(wal);
+      manager.postLogRoll(wal);
+
+      Path remoteLogDirForPeer = new Path(remoteLogDir, slaveId);
+      fs.mkdirs(remoteLogDirForPeer);
+      String walName =
+        "remoteWAL-12345-" + slaveId + ".23456" + SyncReplicationWALProvider.LOG_SUFFIX;
+      Path remoteWAL =
+        new Path(remoteLogDirForPeer, walName).makeQualified(fs.getUri(), fs.getWorkingDirectory());
+      fs.create(remoteWAL).close();
+      wal = new Path(logDir, walName);
+      manager.preLogRoll(wal);
+      manager.postLogRoll(wal);
+
+      ReplicationSourceInterface source = mockReplicationSource(peerId2);
+      manager.cleanOldLogs(walName, true, source);
+      // still there if peer id does not match
+      assertTrue(fs.exists(remoteWAL));
+
+      source = mockReplicationSource(slaveId);
+      manager.cleanOldLogs(walName, true, source);
+      assertFalse(fs.exists(remoteWAL));
+    } finally {
+      removePeerAndWait(peerId2);
+    }
   }
 
   /**
    * Add a peer and wait for it to initialize
-   * @param peerId
-   * @param peerConfig
    * @param waitForSource Whether to wait for replication source to initialize
-   * @throws Exception
    */
   private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig,
       final boolean waitForSource) throws Exception {


[20/36] hbase git commit: HBASE-19957 General framework to transit sync replication state

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
new file mode 100644
index 0000000..32159e6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed 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.replication.ReplicationPeer;
+import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProvider {
+
+  private final ReplicationPeers replicationPeers;
+
+  private final SyncReplicationPeerMappingManager mapping;
+
+  SyncReplicationPeerInfoProviderImpl(ReplicationPeers replicationPeers,
+      SyncReplicationPeerMappingManager mapping) {
+    this.replicationPeers = replicationPeers;
+    this.mapping = mapping;
+  }
+
+  @Override
+  public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
+    String peerId = mapping.getPeerId(info);
+    if (peerId == null) {
+      return Optional.empty();
+    }
+    ReplicationPeer peer = replicationPeers.getPeer(peerId);
+    if (peer == null) {
+      return Optional.empty();
+    }
+    if (peer.getSyncReplicationState() == SyncReplicationState.ACTIVE) {
+      return Optional.of(Pair.newPair(peerId, peer.getPeerConfig().getRemoteWALDir()));
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  @Override
+  public boolean isInState(RegionInfo info, SyncReplicationState state) {
+    String peerId = mapping.getPeerId(info);
+    if (peerId == null) {
+      return false;
+    }
+    ReplicationPeer peer = replicationPeers.getPeer(peerId);
+    if (peer == null) {
+      return false;
+    }
+    return peer.getSyncReplicationState() == state;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java
new file mode 100644
index 0000000..64216cb
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.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.regionserver;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Used to map region to sync replication peer id.
+ * <p>
+ * TODO: now only support include table options.
+ */
+@InterfaceAudience.Private
+class SyncReplicationPeerMappingManager {
+
+  private final ConcurrentMap<TableName, String> table2PeerId = new ConcurrentHashMap<>();
+
+  void add(String peerId, ReplicationPeerConfig peerConfig) {
+    peerConfig.getTableCFsMap().keySet().forEach(tn -> table2PeerId.put(tn, peerId));
+  }
+
+  void remove(String peerId, ReplicationPeerConfig peerConfig) {
+    peerConfig.getTableCFsMap().keySet().forEach(table2PeerId::remove);
+  }
+
+  String getPeerId(RegionInfo info) {
+    return table2PeerId.get(info.getTable());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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
deleted file mode 100644
index b97bf7e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.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 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/31f8fe08/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
index bccc842..e3de6b4 100644
--- 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
@@ -39,7 +39,7 @@ 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.replication.regionserver.SyncReplicationPeerInfoProvider;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.hadoop.hbase.util.Pair;
@@ -67,7 +67,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
 
   private final WALProvider provider;
 
-  private final SyncReplicationPeerProvider peerProvider;
+  private SyncReplicationPeerInfoProvider peerInfoProvider;
 
   private WALFactory factory;
 
@@ -85,9 +85,12 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
 
   private final KeyLocker<String> createLock = new KeyLocker<>();
 
-  SyncReplicationWALProvider(WALProvider provider, SyncReplicationPeerProvider peerProvider) {
+  SyncReplicationWALProvider(WALProvider provider) {
     this.provider = provider;
-    this.peerProvider = peerProvider;
+  }
+
+  public void setPeerInfoProvider(SyncReplicationPeerInfoProvider peerInfoProvider) {
+    this.peerInfoProvider = peerInfoProvider;
   }
 
   @Override
@@ -99,7 +102,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
     this.conf = conf;
     this.factory = factory;
     Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
-        NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
+      NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
     eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
     channelClass = eventLoopGroupAndChannelClass.getSecond();
   }
@@ -112,9 +115,9 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
     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);
+      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 {
@@ -139,7 +142,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   @Override
   public WAL getWAL(RegionInfo region) throws IOException {
     Optional<Pair<String, String>> peerIdAndRemoteWALDir =
-        peerProvider.getPeerIdAndRemoteWALDir(region);
+      peerInfoProvider.getPeerIdAndRemoteWALDir(region);
     if (peerIdAndRemoteWALDir.isPresent()) {
       Pair<String, String> pair = peerIdAndRemoteWALDir.get();
       return getWAL(pair.getFirst(), pair.getSecond());
@@ -221,14 +224,12 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   }
 
   @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));
+      SyncReplicationState to, int stage) {
+    // TODO: stage 0
+    if (from == SyncReplicationState.ACTIVE && to == SyncReplicationState.DOWNGRADE_ACTIVE &&
+      stage == 1) {
+      safeClose(peerId2WAL.remove(peerId));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 737ccfb..78355a1 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
@@ -24,10 +24,10 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.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.SyncReplicationPeerProvider;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -145,18 +145,6 @@ 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).
-   */
-  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;
-  }
-
-  /**
    * @param conf must not be null, will keep a reference to read params in later reader/writer
    *          instances.
    * @param factoryId a unique identifier for this factory. used i.e. by filesystem implementations
@@ -173,7 +161,13 @@ public class WALFactory {
     this.factoryId = factoryId;
     // end required early initialization
     if (conf.getBoolean("hbase.regionserver.hlog.enabled", true)) {
-      provider = getProvider(WAL_PROVIDER, DEFAULT_WAL_PROVIDER, null);
+      WALProvider provider = createProvider(getProviderClass(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
+      if (conf.getBoolean(HConstants.SYNC_REPLICATION_ENABLED, false)) {
+        provider = new SyncReplicationWALProvider(provider);
+      }
+      provider.init(this, conf, null);
+      provider.addWALActionsListener(new MetricsWAL());
+      this.provider = provider;
     } else {
       // special handling of existing configuration behavior.
       LOG.warn("Running with WAL disabled.");
@@ -183,26 +177,6 @@ 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, SyncReplicationPeerProvider 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 SyncReplicationWALProvider(provider, peerProvider);
-    this.provider.init(this, conf, null);
-    this.provider.addWALActionsListener(new MetricsWAL());
-  }
-
-  /**
    * 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.
@@ -250,8 +224,9 @@ public class WALFactory {
       if (provider != null) {
         return provider;
       }
-      provider = getProvider(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER,
-        AbstractFSWALProvider.META_WAL_PROVIDER_ID);
+      provider = createProvider(getProviderClass(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER));
+      provider.init(this, conf, AbstractFSWALProvider.META_WAL_PROVIDER_ID);
+      provider.addWALActionsListener(new MetricsWAL());
       if (metaProvider.compareAndSet(null, provider)) {
         return provider;
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 d462dbd..0ad476f 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
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterR
 import org.apache.hadoop.hbase.replication.regionserver.TestReplicator.ReplicationEndpointForTest;
 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.BeforeClass;
@@ -1008,7 +1009,7 @@ public class TestReplicationAdmin {
   @Test
   public void testTransitSyncReplicationPeerState() throws Exception {
     TableName tableName = TableName.valueOf(name.getMethodName());
-
+    TEST_UTIL.createTable(tableName, Bytes.toBytes("family"));
     ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_ONE);
     builder.setReplicateAllUserTables(false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 b058da3..482f49a 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
@@ -175,7 +175,10 @@ 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",
-      SyncReplicationState.toByteArray(SyncReplicationState.NONE));
+      ZKReplicationPeerStorage.NONE_STATE_ZNODE_BYTES);
+    ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/new-sync-rep-state");
+    ZKUtil.setData(zkw, "/hbase/replication/peers/1/new-sync-rep-state",
+      ZKReplicationPeerStorage.NONE_STATE_ZNODE_BYTES);
     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/31f8fe08/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 f09e51e..986228c 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
@@ -27,6 +27,7 @@ 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.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -35,6 +36,8 @@ 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.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -51,7 +54,7 @@ public class TestSyncReplicationWALProvider {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestSyncReplicationWALProvider.class);
+    HBaseClassTestRule.forClass(TestSyncReplicationWALProvider.class);
 
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
@@ -69,19 +72,30 @@ public class TestSyncReplicationWALProvider {
 
   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();
+  public static final class InfoProvider implements SyncReplicationPeerInfoProvider {
+
+    @Override
+    public 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();
+      }
+    }
+
+    @Override
+    public boolean isInState(RegionInfo info, SyncReplicationState state) {
+      // TODO Implement SyncReplicationPeerInfoProvider.isInState
+      return false;
     }
   }
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
+    UTIL.getConfiguration().setBoolean(HConstants.SYNC_REPLICATION_ENABLED, true);
     UTIL.startMiniDFSCluster(3);
-    FACTORY = new WALFactory(UTIL.getConfiguration(), "test",
-        TestSyncReplicationWALProvider::getPeerIdAndRemoteWALDir);
+    FACTORY = new WALFactory(UTIL.getConfiguration(), "test");
+    ((SyncReplicationWALProvider) FACTORY.getWALProvider()).setPeerInfoProvider(new InfoProvider());
     UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID));
   }
 
@@ -151,9 +165,9 @@ public class TestSyncReplicationWALProvider {
     DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION);
     assertEquals(2, FACTORY.getWALs().size());
     testReadWrite(wal);
-    SyncReplicationWALProvider walProvider =
-      (SyncReplicationWALProvider) FACTORY.getWALProvider();
-    walProvider.peerRemoved(PEER_ID);
+    SyncReplicationWALProvider walProvider = (SyncReplicationWALProvider) FACTORY.getWALProvider();
+    walProvider.peerSyncReplicationStateChange(PEER_ID, SyncReplicationState.ACTIVE,
+      SyncReplicationState.DOWNGRADE_ACTIVE, 1);
     assertEquals(1, FACTORY.getWALs().size());
   }
 }


[32/36] hbase git commit: HBASE-19865 Add UT for sync replication peer in DA state

Posted by zh...@apache.org.
HBASE-19865 Add UT for sync replication peer in DA state


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

Branch: refs/heads/HBASE-19064
Commit: 85f229491e636a66a1afaa45a7fa0f69a6c7c0e8
Parents: c87a548
Author: zhangduo <zh...@apache.org>
Authored: Tue May 8 20:33:22 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/TestReplicationBase.java  | 28 +++++++++++---
 ...estReplicationChangingPeerRegionservers.java | 20 ++++++----
 .../TestReplicationSmallTestsSync.java          | 40 ++++++++++++++++++++
 3 files changed, 76 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/85f22949/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index f96dbe5..cd84293 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -28,6 +27,8 @@ import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -58,6 +59,9 @@ import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
 /**
  * This class is only a base for other integration-level replication tests.
  * Do not add tests here.
@@ -99,6 +103,10 @@ public class TestReplicationBase {
     return false;
   }
 
+  protected boolean isSyncPeer() {
+    return false;
+  }
+
   protected final void cleanUp() throws IOException, InterruptedException {
     // Starting and stopping replication can make us miss new logs,
     // rolling like this makes sure the most recent one gets added to the queue
@@ -245,9 +253,19 @@ public class TestReplicationBase {
   @Before
   public void setUpBase() throws Exception {
     if (!peerExist(PEER_ID2)) {
-      ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder()
-          .setClusterKey(utility2.getClusterKey()).setSerial(isSerialPeer()).build();
-      hbaseAdmin.addReplicationPeer(PEER_ID2, rpc);
+      ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder()
+        .setClusterKey(utility2.getClusterKey()).setSerial(isSerialPeer());
+      if (isSyncPeer()) {
+        FileSystem fs2 = utility2.getTestFileSystem();
+        // The remote wal dir is not important as we do not use it in DA state, here we only need to
+        // confirm that a sync peer in DA state can still replicate data to remote cluster
+        // asynchronously.
+        builder.setReplicateAllUserTables(false)
+          .setTableCFsMap(ImmutableMap.of(tableName, ImmutableList.of()))
+          .setRemoteWALDir(new Path("/RemoteWAL")
+            .makeQualified(fs2.getUri(), fs2.getWorkingDirectory()).toUri().toString());
+      }
+      hbaseAdmin.addReplicationPeer(PEER_ID2, builder.build());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/85f22949/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
index b94b443..5c96742 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
@@ -62,22 +62,28 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
   private static final Logger LOG =
       LoggerFactory.getLogger(TestReplicationChangingPeerRegionservers.class);
 
-  @Parameter
+  @Parameter(0)
   public boolean serialPeer;
 
+  @Parameter(1)
+  public boolean syncPeer;
+
   @Override
   protected boolean isSerialPeer() {
     return serialPeer;
   }
 
-  @Parameters(name = "{index}: serialPeer={0}")
-  public static List<Boolean> parameters() {
-    return ImmutableList.of(true, false);
+  @Override
+  protected boolean isSyncPeer() {
+    return syncPeer;
+  }
+
+  @Parameters(name = "{index}: serialPeer={0}, syncPeer={1}")
+  public static List<Object[]> parameters() {
+    return ImmutableList.of(new Object[] { false, false }, new Object[] { false, true },
+      new Object[] { true, false }, new Object[] { true, true });
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     // Starting and stopping replication can make us miss new logs,

http://git-wip-us.apache.org/repos/asf/hbase/blob/85f22949/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTestsSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTestsSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTestsSync.java
new file mode 100644
index 0000000..9ca0044
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTestsSync.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestReplicationSmallTestsSync extends TestReplicationSmallTests {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestReplicationSmallTestsSync.class);
+
+  @Override
+  protected boolean isSyncPeer() {
+    return true;
+  }
+}


[21/36] hbase git commit: HBASE-19957 General framework to transit sync replication state

Posted by zh...@apache.org.
HBASE-19957 General framework to transit sync replication state


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

Branch: refs/heads/HBASE-19064
Commit: 31f8fe0877177ed977160274caf708b5d6d2fda4
Parents: 0d3c667
Author: zhangduo <zh...@apache.org>
Authored: Fri Feb 9 18:33:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfig.java      |   2 -
 .../replication/ReplicationPeerDescription.java |   5 +-
 .../hbase/replication/SyncReplicationState.java |  19 +-
 .../org/apache/hadoop/hbase/HConstants.java     |   3 +
 .../src/main/protobuf/MasterProcedure.proto     |  20 +-
 .../hbase/replication/ReplicationPeerImpl.java  |  45 ++++-
 .../replication/ReplicationPeerStorage.java     |  25 ++-
 .../hbase/replication/ReplicationPeers.java     |  27 ++-
 .../replication/ZKReplicationPeerStorage.java   |  63 +++++--
 .../hbase/coprocessor/MasterObserver.java       |   7 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   4 +-
 .../hbase/master/MasterCoprocessorHost.java     |  12 +-
 .../replication/AbstractPeerProcedure.java      |  14 +-
 .../master/replication/ModifyPeerProcedure.java |  11 --
 .../replication/RefreshPeerProcedure.java       |  18 +-
 .../replication/ReplicationPeerManager.java     |  89 +++++----
 ...ransitPeerSyncReplicationStateProcedure.java | 181 ++++++++++++-------
 .../hbase/regionserver/HRegionServer.java       |  35 ++--
 .../regionserver/ReplicationSourceService.java  |  11 +-
 .../regionserver/PeerActionListener.java        |   4 +-
 .../regionserver/PeerProcedureHandler.java      |  16 +-
 .../regionserver/PeerProcedureHandlerImpl.java  |  52 +++++-
 .../regionserver/RefreshPeerCallable.java       |   7 +
 .../replication/regionserver/Replication.java   |  22 ++-
 .../regionserver/ReplicationSourceManager.java  |  41 +++--
 .../SyncReplicationPeerInfoProvider.java        |  43 +++++
 .../SyncReplicationPeerInfoProviderImpl.java    |  71 ++++++++
 .../SyncReplicationPeerMappingManager.java      |  48 +++++
 .../SyncReplicationPeerProvider.java            |  35 ----
 .../hbase/wal/SyncReplicationWALProvider.java   |  35 ++--
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  47 ++---
 .../replication/TestReplicationAdmin.java       |   3 +-
 .../TestReplicationSourceManager.java           |   5 +-
 .../wal/TestSyncReplicationWALProvider.java     |  36 ++--
 34 files changed, 743 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 997a155..cc7b4bc 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
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.replication;
 
 import java.util.Collection;
@@ -25,7 +24,6 @@ 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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 2d077c5..b0c27bb 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
@@ -20,7 +20,10 @@ package org.apache.hadoop.hbase.replication;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * The POJO equivalent of ReplicationProtos.ReplicationPeerDescription
+ * The POJO equivalent of ReplicationProtos.ReplicationPeerDescription.
+ * <p>
+ * To developer, here we do not store the new sync replication state since it is just an
+ * intermediate state and this class is public.
  */
 @InterfaceAudience.Public
 public class ReplicationPeerDescription {

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 a65b144..de9576c 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
@@ -29,14 +29,19 @@ 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},
- * {@link SyncReplicationState#DOWNGRADE_ACTIVE} or
- * {@link SyncReplicationState#STANDBY}.
+ * {@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;
+  NONE(0), ACTIVE(1), DOWNGRADE_ACTIVE(2), STANDBY(3);
+
+  private final byte value;
+
+  private SyncReplicationState(int value) {
+    this.value = (byte) value;
+  }
 
   public static SyncReplicationState valueOf(int value) {
     switch (value) {
@@ -53,13 +58,17 @@ public enum SyncReplicationState {
     }
   }
 
+  public int value() {
+    return value & 0xFF;
+  }
+
   public static byte[] toByteArray(SyncReplicationState state) {
     return ProtobufUtil
-        .prependPBMagic(ReplicationPeerConfigUtil.toSyncReplicationState(state).toByteArray());
+      .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)));
+      .parseFrom(Arrays.copyOfRange(bytes, ProtobufUtil.lengthOfPBMagic(), bytes.length)));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 9241682..522c2cf 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1355,6 +1355,9 @@ public final class HConstants {
 
   public static final String NOT_IMPLEMENTED = "Not implemented";
 
+  // TODO: need to find a better place to hold it.
+  public static final String SYNC_REPLICATION_ENABLED = "hbase.replication.sync.enabled";
+
   private HConstants() {
     // Can't be instantiated with this ctor.
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 1aea09f..56ac0d0 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -389,6 +389,17 @@ enum PeerModificationState {
   POST_PEER_MODIFICATION = 8;
 }
 
+enum PeerSyncReplicationStateTransitionState {
+  PRE_PEER_SYNC_REPLICATION_STATE_TRANSITION = 1;
+  SET_PEER_NEW_SYNC_REPLICATION_STATE = 2;
+  REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_BEGIN = 3;
+  REPLAY_REMOTE_WAL_IN_PEER = 4;
+  REOPEN_ALL_REGIONS_IN_PEER = 5;
+  TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE = 6;
+  REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END = 7;
+  POST_PEER_SYNC_REPLICATION_STATE_TRANSITION = 8;
+}
+
 message PeerModificationStateData {
   required string peer_id = 1;
 }
@@ -399,18 +410,23 @@ enum PeerModificationType {
   ENABLE_PEER = 3;
   DISABLE_PEER = 4;
   UPDATE_PEER_CONFIG = 5;
+  TRANSIT_SYNC_REPLICATION_STATE = 6;
 }
 
 message RefreshPeerStateData {
   required string peer_id = 1;
   required PeerModificationType type = 2;
   required ServerName target_server = 3;
+    /** We need multiple stages for sync replication state transition **/
+  optional uint32 stage = 4 [default = 0];
 }
 
 message RefreshPeerParameter {
   required string peer_id = 1;
   required PeerModificationType type = 2;
   required ServerName target_server = 3;
+  /** We need multiple stages for sync replication state transition **/
+  optional uint32 stage = 4 [default = 0];;
 }
 
 message PeerProcedureStateData {
@@ -447,5 +463,7 @@ message ReopenTableRegionsStateData {
 }
 
 message TransitPeerSyncReplicationStateStateData {
-  required SyncReplicationState syncReplicationState = 1;
+  /** Could be null if we fail in pre check, so optional */
+  optional SyncReplicationState fromState = 1;
+  required SyncReplicationState toState = 2;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 ff3f662..22026e5 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
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
@@ -36,7 +37,14 @@ public class ReplicationPeerImpl implements ReplicationPeer {
 
   private volatile PeerState peerState;
 
-  private volatile SyncReplicationState syncReplicationState;
+  // The lower 16 bits are the current sync replication state, the higher 16 bits are the new sync
+  // replication state. Embedded in one int so user can not get an inconsistency view of state and
+  // new state.
+  private volatile int syncReplicationStateBits;
+
+  private static final int SHIFT = 16;
+
+  private static final int AND_BITS = 0xFFFF;
 
   private final List<ReplicationPeerConfigListener> peerConfigListeners;
 
@@ -48,12 +56,14 @@ public class ReplicationPeerImpl implements ReplicationPeer {
    * @param peerConfig configuration for the replication peer
    */
   public ReplicationPeerImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig,
-      boolean peerState, SyncReplicationState syncReplicationState) {
+      boolean peerState, SyncReplicationState syncReplicationState,
+      SyncReplicationState newSyncReplicationState) {
     this.conf = conf;
     this.id = id;
     this.peerState = peerState ? PeerState.ENABLED : PeerState.DISABLED;
     this.peerConfig = peerConfig;
-    this.syncReplicationState = syncReplicationState;
+    this.syncReplicationStateBits =
+      syncReplicationState.value() | (newSyncReplicationState.value() << SHIFT);
     this.peerConfigListeners = new ArrayList<>();
   }
 
@@ -66,6 +76,16 @@ public class ReplicationPeerImpl implements ReplicationPeer {
     peerConfigListeners.forEach(listener -> listener.peerConfigUpdated(peerConfig));
   }
 
+  public void setNewSyncReplicationState(SyncReplicationState newState) {
+    this.syncReplicationStateBits =
+      (this.syncReplicationStateBits & AND_BITS) | (newState.value() << SHIFT);
+  }
+
+  public void transitSyncReplicationState() {
+    this.syncReplicationStateBits =
+      (this.syncReplicationStateBits >>> SHIFT) | (SyncReplicationState.NONE.value() << SHIFT);
+  }
+
   /**
    * Get the identifier of this peer
    * @return string representation of the id (short)
@@ -80,9 +100,26 @@ public class ReplicationPeerImpl implements ReplicationPeer {
     return peerState;
   }
 
+  private static SyncReplicationState getSyncReplicationState(int bits) {
+    return SyncReplicationState.valueOf(bits & AND_BITS);
+  }
+
+  private static SyncReplicationState getNewSyncReplicationState(int bits) {
+    return SyncReplicationState.valueOf(bits >>> SHIFT);
+  }
+
+  public Pair<SyncReplicationState, SyncReplicationState> getSyncReplicationStateAndNewState() {
+    int bits = this.syncReplicationStateBits;
+    return Pair.newPair(getSyncReplicationState(bits), getNewSyncReplicationState(bits));
+  }
+
+  public SyncReplicationState getNewSyncReplicationState() {
+    return getNewSyncReplicationState(syncReplicationStateBits);
+  }
+
   @Override
   public SyncReplicationState getSyncReplicationState() {
-    return syncReplicationState;
+    return getSyncReplicationState(syncReplicationStateBits);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 d2538ab..f74ac37 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
-
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -72,16 +71,30 @@ public interface ReplicationPeerStorage {
   ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException;
 
   /**
-   * Set the state of current cluster in a synchronous replication peer.
+   * Set the new sync replication state that we are going to transit to.
    * @throws ReplicationException if there are errors accessing the storage service.
    */
-  void setPeerSyncReplicationState(String peerId, SyncReplicationState state)
+  void setPeerNewSyncReplicationState(String peerId, SyncReplicationState state)
       throws ReplicationException;
 
   /**
-   * Get the state of current cluster in a synchronous replication peer.
+   * Overwrite the sync replication state with the new sync replication state which is set with the
+   * {@link #setPeerNewSyncReplicationState(String, SyncReplicationState)} method above, and clear
+   * the new sync replication state.
    * @throws ReplicationException if there are errors accessing the storage service.
    */
-  SyncReplicationState getPeerSyncReplicationState(String peerId)
-      throws ReplicationException;
+  void transitPeerSyncReplicationState(String peerId) throws ReplicationException;
+
+  /**
+   * Get the sync replication state.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  SyncReplicationState getPeerSyncReplicationState(String peerId) throws ReplicationException;
+
+  /**
+   * Get the new sync replication state. Will return {@link SyncReplicationState#NONE} if we are
+   * not in a transition.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  SyncReplicationState getPeerNewSyncReplicationState(String peerId) throws ReplicationException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 a54f339..ba6da7a 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
@@ -80,8 +80,8 @@ public class ReplicationPeers {
     return true;
   }
 
-  public void removePeer(String peerId) {
-    peerCache.remove(peerId);
+  public ReplicationPeerImpl removePeer(String peerId) {
+    return peerCache.remove(peerId);
   }
 
   /**
@@ -110,22 +110,29 @@ public class ReplicationPeers {
 
   public PeerState refreshPeerState(String peerId) throws ReplicationException {
     ReplicationPeerImpl peer = peerCache.get(peerId);
-    if (peer == null) {
-      throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
-    }
     peer.setPeerState(peerStorage.isPeerEnabled(peerId));
     return peer.getPeerState();
   }
 
   public ReplicationPeerConfig refreshPeerConfig(String peerId) throws ReplicationException {
     ReplicationPeerImpl peer = peerCache.get(peerId);
-    if (peer == null) {
-      throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
-    }
     peer.setPeerConfig(peerStorage.getPeerConfig(peerId));
     return peer.getPeerConfig();
   }
 
+  public SyncReplicationState refreshPeerNewSyncReplicationState(String peerId)
+      throws ReplicationException {
+    ReplicationPeerImpl peer = peerCache.get(peerId);
+    SyncReplicationState newState = peerStorage.getPeerNewSyncReplicationState(peerId);
+    peer.setNewSyncReplicationState(newState);
+    return newState;
+  }
+
+  public void transitPeerSyncReplicationState(String peerId) {
+    ReplicationPeerImpl peer = peerCache.get(peerId);
+    peer.transitSyncReplicationState();
+  }
+
   /**
    * Helper method to connect to a peer
    * @param peerId peer's identifier
@@ -135,7 +142,9 @@ public class ReplicationPeers {
     ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
     boolean enabled = peerStorage.isPeerEnabled(peerId);
     SyncReplicationState syncReplicationState = peerStorage.getPeerSyncReplicationState(peerId);
+    SyncReplicationState newSyncReplicationState =
+      peerStorage.getPeerNewSyncReplicationState(peerId);
     return new ReplicationPeerImpl(ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf),
-        peerId, peerConfig, enabled, syncReplicationState);
+      peerId, peerConfig, enabled, syncReplicationState, newSyncReplicationState);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 9107cf6..a2cdfdf 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
@@ -53,7 +53,12 @@ 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";
+  public static final String SYNC_REPLICATION_STATE_ZNODE = "sync-rep-state";
+
+  public static final String NEW_SYNC_REPLICATION_STATE_ZNODE = "new-sync-rep-state";
+
+  public static final byte[] NONE_STATE_ZNODE_BYTES =
+    SyncReplicationState.toByteArray(SyncReplicationState.NONE);
 
   /**
    * The name of the znode that contains the replication status of a remote slave (i.e. peer)
@@ -85,7 +90,11 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
 
   @VisibleForTesting
   public String getSyncReplicationStateNode(String peerId) {
-    return ZNodePaths.joinZNode(getPeerNode(peerId), SYNCHRONOUS_REPLICATION_STATE_ZNODE);
+    return ZNodePaths.joinZNode(getPeerNode(peerId), SYNC_REPLICATION_STATE_ZNODE);
+  }
+
+  private String getNewSyncReplicationStateNode(String peerId) {
+    return ZNodePaths.joinZNode(getPeerNode(peerId), NEW_SYNC_REPLICATION_STATE_ZNODE);
   }
 
   @Override
@@ -97,14 +106,15 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
       ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
         enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES),
       ZKUtilOp.createAndFailSilent(getSyncReplicationStateNode(peerId),
-        SyncReplicationState.toByteArray(syncReplicationState)));
+        SyncReplicationState.toByteArray(syncReplicationState)),
+      ZKUtilOp.createAndFailSilent(getNewSyncReplicationStateNode(peerId), NONE_STATE_ZNODE_BYTES));
     try {
       ZKUtil.createWithParents(zookeeper, peersZNode);
       ZKUtil.multiOrSequential(zookeeper, multiOps, false);
     } catch (KeeperException e) {
       throw new ReplicationException(
         "Could not add peer with id=" + peerId + ", peerConfig=>" + peerConfig + ", state=" +
-            (enabled ? "ENABLED" : "DISABLED") + ", syncReplicationState=" + syncReplicationState,
+          (enabled ? "ENABLED" : "DISABLED") + ", syncReplicationState=" + syncReplicationState,
         e);
     }
   }
@@ -136,7 +146,7 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
         ReplicationPeerConfigUtil.toByteArray(peerConfig));
     } catch (KeeperException e) {
       throw new ReplicationException(
-          "There was a problem trying to save changes to the " + "replication peer " + peerId, e);
+        "There was a problem trying to save changes to the " + "replication peer " + peerId, e);
     }
   }
 
@@ -170,38 +180,63 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
     }
     if (data == null || data.length == 0) {
       throw new ReplicationException(
-          "Replication peer config data shouldn't be empty, peerId=" + peerId);
+        "Replication peer config data shouldn't be empty, peerId=" + peerId);
     }
     try {
       return ReplicationPeerConfigUtil.parsePeerFrom(data);
     } catch (DeserializationException e) {
       throw new ReplicationException(
-          "Failed to parse replication peer config for peer with id=" + peerId, e);
+        "Failed to parse replication peer config for peer with id=" + peerId, e);
     }
   }
 
   @Override
-  public void setPeerSyncReplicationState(String peerId, SyncReplicationState state)
+  public void setPeerNewSyncReplicationState(String peerId, SyncReplicationState state)
       throws ReplicationException {
     try {
-      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId),
+      ZKUtil.createSetData(zookeeper, getNewSyncReplicationStateNode(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 set the new sync replication state for peer with id=" + peerId, e);
     }
   }
 
   @Override
-  public SyncReplicationState getPeerSyncReplicationState(String peerId)
+  public void transitPeerSyncReplicationState(String peerId) throws ReplicationException {
+    String newStateNode = getNewSyncReplicationStateNode(peerId);
+    try {
+      byte[] data = ZKUtil.getData(zookeeper, newStateNode);
+      ZKUtil.multiOrSequential(zookeeper,
+        Arrays.asList(ZKUtilOp.setData(newStateNode, NONE_STATE_ZNODE_BYTES),
+          ZKUtilOp.setData(getSyncReplicationStateNode(peerId), data)),
+        false);
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException(
+        "Error transiting sync replication state for peer with id=" + peerId, e);
+    }
+  }
+
+  private SyncReplicationState getSyncReplicationState(String peerId, String path)
       throws ReplicationException {
     try {
-      byte[] data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
+      byte[] data = ZKUtil.getData(zookeeper, path);
       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 sync replication state of path " + path + " for peer with id=" + peerId, e);
     }
   }
+
+  @Override
+  public SyncReplicationState getPeerNewSyncReplicationState(String peerId)
+      throws ReplicationException {
+    return getSyncReplicationState(peerId, getNewSyncReplicationStateNode(peerId));
+  }
+
+  @Override
+  public SyncReplicationState getPeerSyncReplicationState(String peerId)
+      throws ReplicationException {
+    return getSyncReplicationState(peerId, getSyncReplicationStateNode(peerId));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 96f1e17..573ac7a 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
@@ -1395,7 +1395,7 @@ public interface MasterObserver {
    * 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
+   * @param state the new state
    */
   default void preTransitReplicationPeerSyncReplicationState(
       final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
@@ -1406,11 +1406,12 @@ public interface MasterObserver {
    * 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
+   * @param from the old state
+   * @param to the new state
    */
   default void postTransitReplicationPeerSyncReplicationState(
       final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
-      SyncReplicationState state) throws IOException {
+      SyncReplicationState from, SyncReplicationState to) throws IOException {
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 8129f9e..e7e585d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -130,10 +130,10 @@ import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.RecoverMetaProcedure;
 import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
+import org.apache.hadoop.hbase.master.replication.AbstractPeerProcedure;
 import org.apache.hadoop.hbase.master.replication.AddPeerProcedure;
 import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
-import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
 import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure;
@@ -3421,7 +3421,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     return favoredNodesManager;
   }
 
-  private long executePeerProcedure(ModifyPeerProcedure procedure) throws IOException {
+  private long executePeerProcedure(AbstractPeerProcedure<?> procedure) throws IOException {
     long procId = procedureExecutor.submitProcedure(procedure);
     procedure.getLatch().await();
     return procId;

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 87ac1b2..019c64f 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
@@ -1607,22 +1607,22 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preTransitReplicationPeerSyncReplicationState(final String peerId,
-      final SyncReplicationState clusterState) throws IOException {
+  public void preTransitReplicationPeerSyncReplicationState(String peerId,
+      SyncReplicationState state) throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
       public void call(MasterObserver observer) throws IOException {
-        observer.preTransitReplicationPeerSyncReplicationState(this, peerId, clusterState);
+        observer.preTransitReplicationPeerSyncReplicationState(this, peerId, state);
       }
     });
   }
 
-  public void postTransitReplicationPeerSyncReplicationState(final String peerId,
-      final SyncReplicationState clusterState) throws IOException {
+  public void postTransitReplicationPeerSyncReplicationState(String peerId,
+      SyncReplicationState from, SyncReplicationState to) throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
       public void call(MasterObserver observer) throws IOException {
-        observer.postTransitReplicationPeerSyncReplicationState(this, peerId, clusterState);
+        observer.postTransitReplicationPeerSyncReplicationState(this, peerId, from, to);
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
index 0ad8a63..6679d78 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
@@ -46,7 +46,7 @@ public abstract class AbstractPeerProcedure<TState>
 
   protected AbstractPeerProcedure(String peerId) {
     this.peerId = peerId;
-    this.latch = ProcedurePrepareLatch.createLatch(2, 0);
+    this.latch = ProcedurePrepareLatch.createLatch(2, 1);
   }
 
   public ProcedurePrepareLatch getLatch() {
@@ -94,4 +94,16 @@ public abstract class AbstractPeerProcedure<TState>
     super.deserializeStateData(serializer);
     peerId = serializer.deserialize(PeerProcedureStateData.class).getPeerId();
   }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, TState state)
+      throws IOException, InterruptedException {
+    if (state == getInitialState()) {
+      // actually the peer related operations has no rollback, but if we haven't done any
+      // modifications on the peer storage yet, we can just return.
+      return;
+    }
+    throw new UnsupportedOperationException();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 6fd8aa3..fc559b0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -328,17 +328,6 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
   }
 
   @Override
-  protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
-      throws IOException, InterruptedException {
-    if (state == PeerModificationState.PRE_PEER_MODIFICATION) {
-      // actually the peer related operations has no rollback, but if we haven't done any
-      // modifications on the peer storage yet, we can just return.
-      return;
-    }
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   protected PeerModificationState getState(int stateId) {
     return PeerModificationState.forNumber(stateId);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
index ba9bcdc..d51ea63 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -54,6 +54,8 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
       justification = "Will never change after construction")
   private ServerName targetServer;
 
+  private int stage;
+
   private boolean dispatched;
 
   private ProcedureEvent<?> event;
@@ -64,9 +66,15 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
   }
 
   public RefreshPeerProcedure(String peerId, PeerOperationType type, ServerName targetServer) {
+    this(peerId, type, targetServer, 0);
+  }
+
+  public RefreshPeerProcedure(String peerId, PeerOperationType type, ServerName targetServer,
+      int stage) {
     this.peerId = peerId;
     this.type = type;
     this.targetServer = targetServer;
+    this.stage = stage;
   }
 
   @Override
@@ -91,6 +99,8 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
         return PeerModificationType.DISABLE_PEER;
       case UPDATE_CONFIG:
         return PeerModificationType.UPDATE_PEER_CONFIG;
+      case TRANSIT_SYNC_REPLICATION_STATE:
+        return PeerModificationType.TRANSIT_SYNC_REPLICATION_STATE;
       default:
         throw new IllegalArgumentException("Unknown type: " + type);
     }
@@ -108,6 +118,8 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
         return PeerOperationType.DISABLE;
       case UPDATE_PEER_CONFIG:
         return PeerOperationType.UPDATE_CONFIG;
+      case TRANSIT_SYNC_REPLICATION_STATE:
+        return PeerOperationType.TRANSIT_SYNC_REPLICATION_STATE;
       default:
         throw new IllegalArgumentException("Unknown type: " + type);
     }
@@ -118,7 +130,8 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
     assert targetServer.equals(remote);
     return new ServerOperation(this, getProcId(), RefreshPeerCallable.class,
         RefreshPeerParameter.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type))
-            .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray());
+            .setTargetServer(ProtobufUtil.toServerName(remote)).setStage(stage).build()
+            .toByteArray());
   }
 
   private void complete(MasterProcedureEnv env, Throwable error) {
@@ -193,7 +206,7 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     serializer.serialize(
       RefreshPeerStateData.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type))
-          .setTargetServer(ProtobufUtil.toServerName(targetServer)).build());
+          .setTargetServer(ProtobufUtil.toServerName(targetServer)).setStage(stage).build());
   }
 
   @Override
@@ -202,5 +215,6 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
     peerId = data.getPeerId();
     type = toPeerOperationType(data.getType());
     targetServer = ProtobufUtil.toServerName(data.getTargetServer());
+    stage = data.getStage();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 ff778a8..0dc922d 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,7 +20,6 @@ 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;
@@ -50,6 +49,9 @@ import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
+
 /**
  * Manages and performs all replication admin operations.
  * <p>
@@ -64,15 +66,11 @@ 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));
-      }
-    };
+  private final ImmutableMap<SyncReplicationState, EnumSet<SyncReplicationState>>
+    allowedTransition = Maps.immutableEnumMap(ImmutableMap.of(SyncReplicationState.ACTIVE,
+      EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE), SyncReplicationState.STANDBY,
+      EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE), SyncReplicationState.DOWNGRADE_ACTIVE,
+      EnumSet.of(SyncReplicationState.STANDBY, SyncReplicationState.ACTIVE)));
 
   ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage,
       ConcurrentMap<String, ReplicationPeerDescription> peers) {
@@ -165,9 +163,9 @@ public class ReplicationPeerManager {
 
     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() + "'");
+        "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.isSyncReplication()) {
@@ -180,15 +178,19 @@ public class ReplicationPeerManager {
     return desc;
   }
 
-  public void preTransitPeerSyncReplicationState(String peerId, SyncReplicationState state)
-      throws DoNotRetryIOException {
+  /**
+   * @return the old state.
+   */
+  public SyncReplicationState 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);
+        " to " + state + " for peer id=" + peerId);
     }
+    return fromState;
   }
 
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
@@ -199,8 +201,8 @@ public class ReplicationPeerManager {
     }
     ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build();
     SyncReplicationState syncReplicationState =
-        copiedPeerConfig.isSyncReplication() ? SyncReplicationState.DOWNGRADE_ACTIVE
-            : SyncReplicationState.NONE;
+      copiedPeerConfig.isSyncReplication() ? SyncReplicationState.DOWNGRADE_ACTIVE
+        : SyncReplicationState.NONE;
     peerStorage.addPeer(peerId, copiedPeerConfig, enabled, syncReplicationState);
     peers.put(peerId,
       new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState));
@@ -240,7 +242,7 @@ public class ReplicationPeerManager {
     ReplicationPeerDescription desc = peers.get(peerId);
     ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
     ReplicationPeerConfigBuilder newPeerConfigBuilder =
-        ReplicationPeerConfig.newBuilder(peerConfig);
+      ReplicationPeerConfig.newBuilder(peerConfig);
     // we need to use the new conf to overwrite the old one.
     newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration());
     newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
@@ -257,7 +259,7 @@ public class ReplicationPeerManager {
       return new ArrayList<>(peers.values());
     }
     return peers.values().stream().filter(r -> pattern.matcher(r.getPeerId()).matches())
-        .collect(Collectors.toList());
+      .collect(Collectors.toList());
   }
 
   public Optional<ReplicationPeerConfig> getPeerConfig(String peerId) {
@@ -269,12 +271,23 @@ public class ReplicationPeerManager {
     queueStorage.removeLastSequenceIds(peerId);
   }
 
-  public void transitPeerSyncReplicationState(String peerId, SyncReplicationState state)
+  public void setPeerNewSyncReplicationState(String peerId, SyncReplicationState state)
       throws ReplicationException {
+    peerStorage.setPeerNewSyncReplicationState(peerId, state);
+  }
+
+  public void transitPeerSyncReplicationState(String peerId, SyncReplicationState newState)
+      throws ReplicationException {
+    if (peerStorage.getPeerNewSyncReplicationState(peerId) != SyncReplicationState.NONE) {
+      // Only transit if this is not a retry
+      peerStorage.transitPeerSyncReplicationState(peerId);
+    }
     ReplicationPeerDescription desc = peers.get(peerId);
-    peerStorage.setPeerSyncReplicationState(peerId, state);
-    peers.put(peerId,
-      new ReplicationPeerDescription(peerId, desc.isEnabled(), desc.getPeerConfig(), state));
+    if (desc.getSyncReplicationState() != newState) {
+      // Only recreate the desc if this is not a retry
+      peers.put(peerId,
+        new ReplicationPeerDescription(peerId, desc.isEnabled(), desc.getPeerConfig(), newState));
+    }
   }
 
   public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
@@ -301,10 +314,10 @@ public class ReplicationPeerManager {
       // If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
       // Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer
       // cluster.
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty())
-          || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly "
-            + "when you want replicate all cluster");
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
+        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " +
+          "when you want replicate all cluster");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
         peerConfig.getExcludeTableCFsMap());
@@ -312,13 +325,13 @@ public class ReplicationPeerManager {
       // If replicate_all flag is false, it means all user tables can't be replicated to peer
       // cluster. Then allow to config namespaces or table-cfs which will be replicated to peer
       // cluster.
-      if ((peerConfig.getExcludeNamespaces() != null
-          && !peerConfig.getExcludeNamespaces().isEmpty())
-          || (peerConfig.getExcludeTableCFsMap() != null
-              && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
+      if ((peerConfig.getExcludeNamespaces() != null &&
+        !peerConfig.getExcludeNamespaces().isEmpty()) ||
+        (peerConfig.getExcludeTableCFsMap() != null &&
+          !peerConfig.getExcludeTableCFsMap().isEmpty())) {
         throw new DoNotRetryIOException(
-            "Need clean exclude-namespaces or exclude-table-cfs config firstly"
-                + " when replicate_all flag is false");
+          "Need clean exclude-namespaces or exclude-table-cfs config firstly" +
+            " when replicate_all flag is false");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
         peerConfig.getTableCFsMap());
@@ -338,11 +351,11 @@ public class ReplicationPeerManager {
     // TODO: Add namespace, replicat_all flag back
     if (peerConfig.replicateAllUserTables()) {
       throw new DoNotRetryIOException(
-          "Only support replicated table config for sync replication peer");
+        "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");
+        "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");
@@ -350,7 +363,7 @@ public class ReplicationPeerManager {
     for (List<String> cfs : peerConfig.getTableCFsMap().values()) {
       if (cfs != null && !cfs.isEmpty()) {
         throw new DoNotRetryIOException(
-            "Only support replicated table config for sync replication peer");
+          "Only support replicated table config for sync replication peer");
       }
     }
   }
@@ -394,7 +407,7 @@ public class ReplicationPeerManager {
   private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
       throws DoNotRetryIOException {
     String filterCSV = peerConfig.getConfiguration()
-        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
+      .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
     if (filterCSV != null && !filterCSV.isEmpty()) {
       String[] filters = filterCSV.split(",");
       for (String filter : filters) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 aad3b06..8fc932f 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
@@ -18,11 +18,12 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.client.RegionInfo;
 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;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
@@ -32,26 +33,29 @@ 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.PeerSyncReplicationStateTransitionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TransitPeerSyncReplicationStateStateData;
 
 /**
- * The procedure for transit current cluster state for a synchronous replication peer.
+ * The procedure for transit current sync replication state for a synchronous replication peer.
  */
 @InterfaceAudience.Private
-public class TransitPeerSyncReplicationStateProcedure extends ModifyPeerProcedure {
+public class TransitPeerSyncReplicationStateProcedure
+    extends AbstractPeerProcedure<PeerSyncReplicationStateTransitionState> {
 
   private static final Logger LOG =
     LoggerFactory.getLogger(TransitPeerSyncReplicationStateProcedure.class);
 
-  private SyncReplicationState state;
+  private SyncReplicationState fromState;
+
+  private SyncReplicationState toState;
 
   public TransitPeerSyncReplicationStateProcedure() {
   }
 
   public TransitPeerSyncReplicationStateProcedure(String peerId, SyncReplicationState state) {
     super(peerId);
-    this.state = state;
+    this.toState = state;
   }
 
   @Override
@@ -60,99 +64,154 @@ public class TransitPeerSyncReplicationStateProcedure extends ModifyPeerProcedur
   }
 
   @Override
-  protected void prePeerModification(MasterProcedureEnv env)
-      throws IOException, ReplicationException {
-    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.preTransitReplicationPeerSyncReplicationState(peerId, state);
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    TransitPeerSyncReplicationStateStateData.Builder builder =
+      TransitPeerSyncReplicationStateStateData.newBuilder()
+        .setToState(ReplicationPeerConfigUtil.toSyncReplicationState(toState));
+    if (fromState != null) {
+      builder.setFromState(ReplicationPeerConfigUtil.toSyncReplicationState(fromState));
     }
-    env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, state);
+    serializer.serialize(builder.build());
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
-    env.getReplicationPeerManager().transitPeerSyncReplicationState(peerId, state);
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    TransitPeerSyncReplicationStateStateData data =
+      serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
+    toState = ReplicationPeerConfigUtil.toSyncReplicationState(data.getToState());
+    if (data.hasFromState()) {
+      fromState = ReplicationPeerConfigUtil.toSyncReplicationState(data.getFromState());
+    }
   }
 
   @Override
-  protected void postPeerModification(MasterProcedureEnv env)
-    throws IOException, ReplicationException {
-    LOG.info("Successfully transit current cluster state to {} in synchronous replication peer {}",
-      state, peerId);
+  protected PeerSyncReplicationStateTransitionState getState(int stateId) {
+    return PeerSyncReplicationStateTransitionState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(PeerSyncReplicationStateTransitionState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected PeerSyncReplicationStateTransitionState getInitialState() {
+    return PeerSyncReplicationStateTransitionState.PRE_PEER_SYNC_REPLICATION_STATE_TRANSITION;
+  }
+
+  private void preTransit(MasterProcedureEnv env) throws IOException {
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
-      env.getMasterCoprocessorHost().postTransitReplicationPeerSyncReplicationState(peerId, state);
+      cpHost.preTransitReplicationPeerSyncReplicationState(peerId, toState);
     }
+    fromState = env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, toState);
   }
 
-  @Override
-  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
-    super.serializeStateData(serializer);
-    serializer.serialize(TransitPeerSyncReplicationStateStateData.newBuilder()
-        .setSyncReplicationState(ReplicationPeerConfigUtil.toSyncReplicationState(state)).build());
+  private void postTransit(MasterProcedureEnv env) throws IOException {
+    LOG.info(
+      "Successfully transit current cluster state from {} to {} for sync replication peer {}",
+      fromState, toState, peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      env.getMasterCoprocessorHost().postTransitReplicationPeerSyncReplicationState(peerId,
+        fromState, toState);
+    }
   }
 
-  @Override
-  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
-    super.deserializeStateData(serializer);
-    TransitPeerSyncReplicationStateStateData data =
-        serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
-    state = ReplicationPeerConfigUtil.toSyncReplicationState(data.getSyncReplicationState());
+  private List<RegionInfo> getRegionsToReopen(MasterProcedureEnv env) {
+    return env.getReplicationPeerManager().getPeerConfig(peerId).get().getTableCFsMap().keySet()
+      .stream()
+      .flatMap(tn -> env.getAssignmentManager().getRegionStates().getRegionsOfTable(tn).stream())
+      .collect(Collectors.toList());
   }
 
   @Override
-  protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
+  protected Flow executeFromState(MasterProcedureEnv env,
+      PeerSyncReplicationStateTransitionState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     switch (state) {
-      case PRE_PEER_MODIFICATION:
+      case PRE_PEER_SYNC_REPLICATION_STATE_TRANSITION:
         try {
-          prePeerModification(env);
+          preTransit(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();
+          LOG.warn("Failed to call pre CP hook or the pre check is failed for peer {} " +
+            "when transiting sync replication peer state to {}, " +
+            "mark the procedure as failure and give up", peerId, toState, e);
+          setFailure("master-transit-peer-sync-replication-state", e);
           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);
+        setNextState(PeerSyncReplicationStateTransitionState.SET_PEER_NEW_SYNC_REPLICATION_STATE);
         return Flow.HAS_MORE_STATE;
-      case UPDATE_PEER_STORAGE:
+      case SET_PEER_NEW_SYNC_REPLICATION_STATE:
         try {
-          updatePeerStorage(env);
+          env.getReplicationPeerManager().setPeerNewSyncReplicationState(peerId, toState);
         } catch (ReplicationException e) {
-          LOG.warn("{} update peer storage for peer {} failed, retry", getClass().getName(), peerId,
-            e);
+          LOG.warn("Failed to update peer storage for peer {} when starting transiting sync " +
+            "replication peer state from {} to {}, retry", peerId, fromState, toState, e);
           throw new ProcedureYieldException();
         }
-        setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
+        setNextState(
+          PeerSyncReplicationStateTransitionState.REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_BEGIN);
+        return Flow.HAS_MORE_STATE;
+      case REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_BEGIN:
+        addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
+          .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 0))
+          .toArray(RefreshPeerProcedure[]::new));
+        if (fromState == SyncReplicationState.STANDBY &&
+          toState == SyncReplicationState.DOWNGRADE_ACTIVE) {
+          setNextState(PeerSyncReplicationStateTransitionState.REPLAY_REMOTE_WAL_IN_PEER);
+        } else {
+          setNextState(PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
+        }
+        return Flow.HAS_MORE_STATE;
+      case REPLAY_REMOTE_WAL_IN_PEER:
+        // TODO: replay remote wal when transiting from S to DA.
+        setNextState(PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
         return Flow.HAS_MORE_STATE;
-      case REFRESH_PEER_ON_RS:
-        // TODO: Need add child procedure for every RegionServer
-        setNextState(PeerModificationState.POST_PEER_MODIFICATION);
+      case REOPEN_ALL_REGIONS_IN_PEER:
+        try {
+          addChildProcedure(
+            env.getAssignmentManager().createReopenProcedures(getRegionsToReopen(env)));
+        } catch (IOException e) {
+          LOG.warn("Failed to schedule region reopen for peer {} when starting transiting sync " +
+            "replication peer state from {} to {}, retry", peerId, fromState, toState, e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(
+          PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE);
         return Flow.HAS_MORE_STATE;
-      case POST_PEER_MODIFICATION:
+      case TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE:
         try {
-          postPeerModification(env);
+          env.getReplicationPeerManager().transitPeerSyncReplicationState(peerId, toState);
         } catch (ReplicationException e) {
-          LOG.warn("{} failed to call postPeerModification for peer {}, retry",
-            getClass().getName(), peerId, e);
+          LOG.warn("Failed to update peer storage for peer {} when ending transiting sync " +
+            "replication peer state from {} to {}, retry", peerId, fromState, toState, e);
           throw new ProcedureYieldException();
+        }
+        setNextState(
+          PeerSyncReplicationStateTransitionState.REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END);
+        return Flow.HAS_MORE_STATE;
+      case REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END:
+        addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
+          .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 1))
+          .toArray(RefreshPeerProcedure[]::new));
+        setNextState(
+          PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION);
+      case POST_PEER_SYNC_REPLICATION_STATE_TRANSITION:
+        try {
+          postTransit(env);
         } catch (IOException e) {
-          LOG.warn("{} failed to call post CP hook for peer {}, " +
-            "ignore since the procedure has already done", getClass().getName(), peerId, e);
+          LOG.warn(
+            "Failed to call post CP hook for peer {} when transiting sync replication " +
+              "peer state from {} to {}, ignore since the procedure has already done",
+            peerId, fromState, toState, 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/31f8fe08/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 a17b402..9c23750 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
@@ -1801,21 +1801,27 @@ public class HRegionServer extends HasThread implements
    * be hooked up to WAL.
    */
   private void setupWALAndReplication() throws IOException {
+    boolean isMasterNoTableOrSystemTableOnly = this instanceof HMaster &&
+      (!LoadBalancer.isTablesOnMaster(conf) || LoadBalancer.isSystemTablesOnlyOnMaster(conf));
+    if (isMasterNoTableOrSystemTableOnly) {
+      conf.setBoolean(HConstants.SYNC_REPLICATION_ENABLED, false);
+    }
     WALFactory factory = new WALFactory(conf, serverName.toString());
+    if (!isMasterNoTableOrSystemTableOnly) {
+      // TODO Replication make assumptions here based on the default filesystem impl
+      Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+      String logName = AbstractFSWALProvider.getWALDirectoryName(this.serverName.toString());
 
-    // TODO Replication make assumptions here based on the default filesystem impl
-    Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    String logName = AbstractFSWALProvider.getWALDirectoryName(this.serverName.toString());
-
-    Path logDir = new Path(walRootDir, logName);
-    LOG.debug("logDir={}", logDir);
-    if (this.walFs.exists(logDir)) {
-      throw new RegionServerRunningException(
-          "Region server has already created directory at " + this.serverName.toString());
+      Path logDir = new Path(walRootDir, logName);
+      LOG.debug("logDir={}", logDir);
+      if (this.walFs.exists(logDir)) {
+        throw new RegionServerRunningException(
+            "Region server has already created directory at " + this.serverName.toString());
+      }
+      // Instantiate replication if replication enabled. Pass it the log directories.
+      createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir,
+        factory.getWALProvider());
     }
-    // Instantiate replication if replication enabled. Pass it the log directories.
-    createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir,
-      factory.getWALProvider());
     this.walFactory = factory;
   }
 
@@ -2937,11 +2943,6 @@ public class HRegionServer extends HasThread implements
    */
   private static void createNewReplicationInstance(Configuration conf, HRegionServer server,
       FileSystem walFs, Path walDir, Path oldWALDir, WALProvider walProvider) throws IOException {
-    if ((server instanceof HMaster) &&
-      (!LoadBalancer.isTablesOnMaster(conf) || LoadBalancer.isSystemTablesOnlyOnMaster(conf))) {
-      return;
-    }
-
     // read in the name of the source replication class from the config file.
     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
       HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
index 23ba773..4529943 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
@@ -18,17 +18,22 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.replication.regionserver.PeerProcedureHandler;
+import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * A source for a replication stream has to expose this service.
- * This service allows an application to hook into the
- * regionserver and watch for new transactions.
+ * A source for a replication stream has to expose this service. This service allows an application
+ * to hook into the regionserver and watch for new transactions.
  */
 @InterfaceAudience.Private
 public interface ReplicationSourceService extends ReplicationService {
 
   /**
+   * Returns an info provider for sync replication peer.
+   */
+  SyncReplicationPeerInfoProvider getSyncReplicationPeerInfoProvider();
+
+  /**
    * Returns a Handler to handle peer procedures.
    */
   PeerProcedureHandler getPeerProcedureHandler();

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/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 6df2af9..efafd09 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
@@ -28,8 +28,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public interface PeerActionListener {
 
-  default void peerRemoved(String peerId) {}
+  static final PeerActionListener DUMMY = new PeerActionListener() {};
 
   default void peerSyncReplicationStateChange(String peerId, SyncReplicationState from,
-      SyncReplicationState to) {}
+      SyncReplicationState to, int stage) {}
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
index 65da9af..52b604b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
@@ -15,11 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -29,13 +28,16 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public interface PeerProcedureHandler {
 
-  public void addPeer(String peerId) throws ReplicationException, IOException;
+  void addPeer(String peerId) throws ReplicationException, IOException;
+
+  void removePeer(String peerId) throws ReplicationException, IOException;
 
-  public void removePeer(String peerId) throws ReplicationException, IOException;
+  void disablePeer(String peerId) throws ReplicationException, IOException;
 
-  public void disablePeer(String peerId) throws ReplicationException, IOException;
+  void enablePeer(String peerId) throws ReplicationException, IOException;
 
-  public void enablePeer(String peerId) throws ReplicationException, IOException;
+  void updatePeerConfig(String peerId) throws ReplicationException, IOException;
 
-  public void updatePeerConfig(String peerId) throws ReplicationException, IOException;
+  void transitSyncReplicationPeerState(String peerId, int stage, HRegionServer rs)
+      throws ReplicationException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
index 78c1977..7fc9f53 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -19,23 +19,32 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
 import java.util.concurrent.locks.Lock;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
+  private static final Logger LOG = LoggerFactory.getLogger(PeerProcedureHandlerImpl.class);
+
   private final ReplicationSourceManager replicationSourceManager;
+  private final PeerActionListener peerActionListener;
   private final KeyLocker<String> peersLock = new KeyLocker<>();
 
-  public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) {
+  public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager,
+      PeerActionListener peerActionListener) {
     this.replicationSourceManager = replicationSourceManager;
+    this.peerActionListener = peerActionListener;
   }
 
   @Override
@@ -61,7 +70,6 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
   }
 
   private void refreshPeerState(String peerId) throws ReplicationException, IOException {
-    PeerState newState;
     Lock peerLock = peersLock.acquireLock(peerId);
     ReplicationPeerImpl peer = null;
     PeerState oldState = null;
@@ -72,7 +80,7 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
         throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
       }
       oldState = peer.getPeerState();
-      newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+      PeerState newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
       // RS need to start work with the new replication state change
       if (oldState.equals(PeerState.ENABLED) && newState.equals(PeerState.DISABLED)) {
         replicationSourceManager.refreshSources(peerId);
@@ -132,4 +140,42 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
       peerLock.unlock();
     }
   }
+
+  @Override
+  public void transitSyncReplicationPeerState(String peerId, int stage, HRegionServer rs)
+      throws ReplicationException, IOException {
+    ReplicationPeers replicationPeers = replicationSourceManager.getReplicationPeers();
+    Lock peerLock = peersLock.acquireLock(peerId);
+    try {
+      ReplicationPeerImpl peer = replicationPeers.getPeer(peerId);
+      if (peer == null) {
+        throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
+      }
+      if (!peer.getPeerConfig().isSyncReplication()) {
+        throw new ReplicationException("Peer with id=" + peerId + " is not synchronous.");
+      }
+      SyncReplicationState newState = peer.getNewSyncReplicationState();
+      if (stage == 0) {
+        if (newState != SyncReplicationState.NONE) {
+          LOG.warn("The new sync replication state for peer {} has already been set to {}, " +
+            "this should be a retry, give up", peerId, newState);
+          return;
+        }
+        newState = replicationPeers.refreshPeerNewSyncReplicationState(peerId);
+        SyncReplicationState oldState = peer.getSyncReplicationState();
+        peerActionListener.peerSyncReplicationStateChange(peerId, oldState, newState, stage);
+      } else {
+        if (newState == SyncReplicationState.NONE) {
+          LOG.warn("The new sync replication state for peer {} has already been clear, and the " +
+            "current state is {}, this should be a retry, give up", peerId, newState);
+          return;
+        }
+        SyncReplicationState oldState = peer.getSyncReplicationState();
+        peerActionListener.peerSyncReplicationStateChange(peerId, oldState, newState, stage);
+        peer.transitSyncReplicationState();
+      }
+    } finally {
+      peerLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
index 7ada24b..8fe16bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
@@ -35,12 +35,15 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.R
 public class RefreshPeerCallable implements RSProcedureCallable {
 
   private static final Logger LOG = Logger.getLogger(RefreshPeerCallable.class);
+
   private HRegionServer rs;
 
   private String peerId;
 
   private PeerModificationType type;
 
+  private int stage;
+
   private Exception initError;
 
   @Override
@@ -67,6 +70,9 @@ public class RefreshPeerCallable implements RSProcedureCallable {
       case UPDATE_PEER_CONFIG:
         handler.updatePeerConfig(this.peerId);
         break;
+      case TRANSIT_SYNC_REPLICATION_STATE:
+        handler.transitSyncReplicationPeerState(peerId, stage, rs);
+        break;
       default:
         throw new IllegalArgumentException("Unknown peer modification type: " + type);
     }
@@ -80,6 +86,7 @@ public class RefreshPeerCallable implements RSProcedureCallable {
       RefreshPeerParameter param = RefreshPeerParameter.parseFrom(parameter);
       this.peerId = param.getPeerId();
       this.type = param.getType();
+      this.stage = param.getStage();
     } catch (InvalidProtocolBufferException e) {
       initError = e;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 8290ac3..2846d2c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
 import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -66,6 +67,7 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
   private ReplicationTracker replicationTracker;
   private Configuration conf;
   private ReplicationSink replicationSink;
+  private SyncReplicationPeerInfoProvider syncReplicationPeerInfoProvider;
   // Hosting server
   private Server server;
   /** Statistics thread schedule pool */
@@ -120,19 +122,30 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
     } catch (KeeperException ke) {
       throw new IOException("Could not read cluster id", ke);
     }
+    SyncReplicationPeerMappingManager mapping = new SyncReplicationPeerMappingManager();
     this.replicationManager = new ReplicationSourceManager(queueStorage, replicationPeers,
         replicationTracker, conf, this.server, fs, logDir, oldLogDir, clusterId,
-        walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty());
+        walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty(),
+        mapping);
+    this.syncReplicationPeerInfoProvider =
+        new SyncReplicationPeerInfoProviderImpl(replicationPeers, mapping);
+    PeerActionListener peerActionListener = PeerActionListener.DUMMY;
     if (walProvider != null) {
       walProvider
         .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager));
+      if (walProvider instanceof SyncReplicationWALProvider) {
+        SyncReplicationWALProvider syncWALProvider = (SyncReplicationWALProvider) walProvider;
+        peerActionListener = syncWALProvider;
+        syncWALProvider.setPeerInfoProvider(syncReplicationPeerInfoProvider);
+      }
     }
     this.statsThreadPeriod =
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
     LOG.debug("Replication stats-in-log period={} seconds",  this.statsThreadPeriod);
     this.replicationLoad = new ReplicationLoad();
 
-    this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager);
+    this.peerProcedureHandler =
+      new PeerProcedureHandlerImpl(replicationManager, peerActionListener);
   }
 
   @Override
@@ -270,4 +283,9 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
     MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics();
     this.replicationLoad.buildReplicationLoad(sourceMetricsList, sinkMetrics);
   }
+
+  @Override
+  public SyncReplicationPeerInfoProvider getSyncReplicationPeerInfoProvider() {
+    return syncReplicationPeerInfoProvider;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/31f8fe08/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 9b4a22c..4212597 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationListener;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
@@ -135,6 +136,8 @@ public class ReplicationSourceManager implements ReplicationListener {
   // For recovered source, the queue id's format is peer_id-servername-*
   private final ConcurrentMap<String, Map<String, NavigableSet<String>>> walsByIdRecoveredQueues;
 
+  private final SyncReplicationPeerMappingManager syncReplicationPeerMappingManager;
+
   private final Configuration conf;
   private final FileSystem fs;
   // The paths to the latest log of each wal group, for new coming peers
@@ -169,9 +172,8 @@ public class ReplicationSourceManager implements ReplicationListener {
   public ReplicationSourceManager(ReplicationQueueStorage queueStorage,
       ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf,
       Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId,
-      WALFileLengthProvider walFileLengthProvider) throws IOException {
-    // CopyOnWriteArrayList is thread-safe.
-    // Generally, reading is more than modifying.
+      WALFileLengthProvider walFileLengthProvider,
+      SyncReplicationPeerMappingManager syncReplicationPeerMappingManager) throws IOException {
     this.sources = new ConcurrentHashMap<>();
     this.queueStorage = queueStorage;
     this.replicationPeers = replicationPeers;
@@ -184,10 +186,11 @@ public class ReplicationSourceManager implements ReplicationListener {
     this.fs = fs;
     this.logDir = logDir;
     this.oldLogDir = oldLogDir;
-    this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 30000); // 30
-                                                                                         // seconds
+    // 30 seconds
+    this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 30000);
     this.clusterId = clusterId;
     this.walFileLengthProvider = walFileLengthProvider;
+    this.syncReplicationPeerMappingManager = syncReplicationPeerMappingManager;
     this.replicationTracker.registerListener(this);
     // It's preferable to failover 1 RS at a time, but with good zk servers
     // more could be processed at the same time.
@@ -248,8 +251,11 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
-   * 1. Add peer to replicationPeers 2. Add the normal source and related replication queue 3. Add
-   * HFile Refs
+   * <ol>
+   * <li>Add peer to replicationPeers</li>
+   * <li>Add the normal source and related replication queue</li>
+   * <li>Add HFile Refs</li>
+   * </ol>
    * @param peerId the id of replication peer
    */
   public void addPeer(String peerId) throws IOException {
@@ -268,13 +274,16 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
-   * 1. Remove peer for replicationPeers 2. Remove all the recovered sources for the specified id
-   * and related replication queues 3. Remove the normal source and related replication queue 4.
-   * Remove HFile Refs
+   * <ol>
+   * <li>Remove peer for replicationPeers</li>
+   * <li>Remove all the recovered sources for the specified id and related replication queues</li>
+   * <li>Remove the normal source and related replication queue</li>
+   * <li>Remove HFile Refs</li>
+   * </ol>
    * @param peerId the id of the replication peer
    */
   public void removePeer(String peerId) {
-    replicationPeers.removePeer(peerId);
+    ReplicationPeer peer = replicationPeers.removePeer(peerId);
     String terminateMessage = "Replication stream was removed by a user";
     List<ReplicationSourceInterface> oldSourcesToDelete = new ArrayList<>();
     // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
@@ -305,7 +314,10 @@ public class ReplicationSourceManager implements ReplicationListener {
       deleteQueue(peerId);
       this.walsById.remove(peerId);
     }
-
+    ReplicationPeerConfig peerConfig = peer.getPeerConfig();
+    if (peerConfig.isSyncReplication()) {
+      syncReplicationPeerMappingManager.remove(peerId, peerConfig);
+    }
     // Remove HFile Refs
     abortWhenFail(() -> this.queueStorage.removePeerFromHFileRefs(peerId));
   }
@@ -357,6 +369,10 @@ public class ReplicationSourceManager implements ReplicationListener {
         }
       }
     }
+    ReplicationPeerConfig peerConfig = peer.getPeerConfig();
+    if (peerConfig.isSyncReplication()) {
+      syncReplicationPeerMappingManager.add(peer.getId(), peerConfig);
+    }
     src.startup();
     return src;
   }
@@ -434,6 +450,7 @@ public class ReplicationSourceManager implements ReplicationListener {
     // Delete queue from storage and memory
     deleteQueue(src.getQueueId());
     this.walsById.remove(src.getQueueId());
+
   }
 
   /**


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

Branch: refs/heads/HBASE-19064
Commit: 0aba887d2be4a0edc598acd9bd3a7b4f60ab17df
Parents: 2fb1022
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Jan 22 11:44:49 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 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     |   4 +
 .../src/main/protobuf/Replication.proto         |  20 +++
 .../replication/ReplicationPeerStorage.java     |  18 ++-
 .../hbase/replication/ReplicationUtils.java     |   1 +
 .../replication/ZKReplicationPeerStorage.java   |  61 +++++--
 .../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    |   8 +-
 .../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, 816 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0aba887d/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 331f2d1..39542e4 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
@@ -51,6 +51,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;
@@ -2657,6 +2658,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/0aba887d/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 8141e74..65cccf7 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/0aba887d/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 5b22668..08952cb 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/0aba887d/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 53e4b7f..0d1cfd6 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
@@ -124,6 +124,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;
 
@@ -1714,6 +1716,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/0aba887d/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 172db5b..9fd6546 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/0aba887d/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 0fd0e59..963cca7 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/0aba887d/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/0aba887d/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 474ded3..6cbe05b 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;
@@ -396,25 +397,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/0aba887d/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/0aba887d/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/0aba887d/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 fc037a8..1269cc7 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.io.TimeRange;
 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;
@@ -147,6 +148,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;
 
 /**
@@ -1867,4 +1869,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/0aba887d/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/0aba887d/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 b65551f..1aea09f 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -445,3 +445,7 @@ enum ReopenTableRegionsState {
 message ReopenTableRegionsStateData {
   required TableName table_name = 1;
 }
+
+message TransitPeerSyncReplicationStateStateData {
+  required SyncReplicationState syncReplicationState = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0aba887d/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 20dd049..3564ae4 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -64,12 +64,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;
 }
 
 /**
@@ -138,3 +149,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/0aba887d/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/0aba887d/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 c7568bb..e4dea83 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/0aba887d/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 bbe6549..ad3c435 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
@@ -23,12 +23,14 @@ 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.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
@@ -51,6 +53,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 +83,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);
     }
   }
 
@@ -167,4 +179,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/0aba887d/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 437804c..4a2c3cd 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
@@ -166,7 +166,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());
@@ -189,10 +190,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);
@@ -241,9 +244,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
@@ -253,7 +260,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());
@@ -365,7 +374,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/0aba887d/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/0aba887d/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 3175af3..96f1e17 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;
 
@@ -1391,6 +1392,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/0aba887d/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 68b3ce8..8129f9e 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
@@ -136,6 +136,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.master.zksyncer.MasterAddressSyncer;
@@ -172,6 +173,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;
@@ -3490,6 +3492,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/0aba887d/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 e563cd4..87ac1b2 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
@@ -56,6 +56,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;
@@ -1606,6 +1607,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/0aba887d/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 fa4ddf5..5de5681 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
@@ -85,6 +85,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;
@@ -291,6 +292,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;
@@ -1943,6 +1946,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/0aba887d/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 52046c5..76aa2d6 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/0aba887d/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/0aba887d/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 05ecd61..f07a0d8 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;
@@ -167,6 +180,17 @@ public class ReplicationPeerManager {
     return desc;
   }
 
+  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)) {
@@ -174,8 +198,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 {
@@ -194,7 +222,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 {
@@ -219,7 +248,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) {
@@ -239,7 +269,15 @@ public class ReplicationPeerManager {
     queueStorage.removeLastSequenceIds(peerId);
   }
 
-  void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
+  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
     // scanned claims the queue of a RS which has not been scanned yet, we will miss that queue in
@@ -368,7 +406,8 @@ 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);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0aba887d/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/0aba887d/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 2758c7e..52b7a92 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;
@@ -2503,6 +2504,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/0aba887d/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/0aba887d/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 0256660..dce062c 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;
@@ -486,4 +485,9 @@ public class MockNoopMasterServices implements MasterServices {
   public boolean isClusterUp() {
     return true;
   }
+
+  public long transitReplicationPeerSyncReplicationState(String peerId,
+    SyncReplicationState clusterState) throws ReplicationException, IOException {
+    return 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0aba887d/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/0aba887d/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 77b4c1b..cbdee7f 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/0aba887d/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 eb46cd7..8170893 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
@@ -71,6 +71,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;
@@ -585,7 +586,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/0aba887d/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 870fa19..d2aa682 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;
@@ -2941,6 +2942,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


[09/36] hbase git commit: HBASE-20424 Allow writing WAL to local and remote cluster concurrently

Posted by zh...@apache.org.
HBASE-20424 Allow writing WAL to local and remote cluster concurrently


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

Branch: refs/heads/HBASE-19064
Commit: 46866d9f6f3372202c5e4f172ac9c3f46258aa22
Parents: 25cac93
Author: zhangduo <zh...@apache.org>
Authored: Thu May 24 16:20:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/MasterProcedure.proto     |   2 +-
 .../hbase/replication/ReplicationUtils.java     |  26 ++-
 .../asyncfs/FanOutOneBlockAsyncDFSOutput.java   |   3 +-
 .../replication/RecoverStandbyProcedure.java    |  10 +-
 .../master/replication/RemovePeerProcedure.java |   5 +-
 .../ReplaySyncReplicationWALManager.java        | 110 ++++++-----
 ...ransitPeerSyncReplicationStateProcedure.java |   4 +-
 .../hbase/regionserver/HRegionServer.java       |   3 +-
 .../regionserver/ReplicationSourceService.java  |   6 +
 .../hbase/regionserver/SplitLogWorker.java      | 188 +++++++++++++------
 .../regionserver/wal/CombinedAsyncWriter.java   |  80 ++------
 .../hbase/regionserver/wal/DualAsyncFSWAL.java  |  11 +-
 .../replication/regionserver/Replication.java   |   5 +
 .../regionserver/ReplicationSourceManager.java  |   2 +-
 .../SyncReplicationPeerInfoProviderImpl.java    |   3 +-
 .../org/apache/hadoop/hbase/util/FSUtils.java   |   9 +
 .../hbase/wal/SyncReplicationWALProvider.java   |  43 ++++-
 .../replication/TestReplicationAdmin.java       |   2 +-
 .../wal/TestCombinedAsyncWriter.java            |  20 +-
 .../replication/DualAsyncFSWALForTest.java      | 149 +++++++++++++++
 .../replication/SyncReplicationTestBase.java    |  12 +-
 .../replication/TestSyncReplicationActive.java  |   5 +-
 ...cReplicationMoreLogsInLocalCopyToRemote.java | 108 +++++++++++
 ...plicationMoreLogsInLocalGiveUpSplitting.java | 128 +++++++++++++
 .../TestSyncReplicationRemoveRemoteWAL.java     |   7 +-
 .../replication/TestSyncReplicationStandBy.java |  20 +-
 .../master/TestRecoverStandbyProcedure.java     |   4 +-
 .../TestReplicationSourceManager.java           |   5 +-
 .../wal/TestSyncReplicationWALProvider.java     |   1 -
 29 files changed, 733 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 f58ad2e..5764a21 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -476,7 +476,7 @@ enum RecoverStandbyState {
   RENAME_SYNC_REPLICATION_WALS_DIR = 1;
   INIT_WORKERS = 2;
   DISPATCH_TASKS = 3;
-  REMOVE_SYNC_REPLICATION_WALS_DIR = 4;
+  SNAPSHOT_SYNC_REPLICATION_WALS_DIR = 4;
 }
 
 message RecoverStandbyStateData {

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 069db7a..dc4217c 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
@@ -46,6 +46,16 @@ public final class ReplicationUtils {
 
   public static final String REMOTE_WAL_DIR_NAME = "remoteWALs";
 
+  public static final String SYNC_WAL_SUFFIX = ".syncrep";
+
+  public static final String REMOTE_WAL_REPLAY_SUFFIX = "-replay";
+
+  public static final String REMOTE_WAL_SNAPSHOT_SUFFIX = "-snapshot";
+
+  // This is used for copying sync replication log from local to remote and overwrite the old one
+  // since some FileSystem implementation may not support atomic rename.
+  public static final String RENAME_WAL_SUFFIX = ".ren";
+
   private ReplicationUtils() {
   }
 
@@ -187,14 +197,26 @@ public final class ReplicationUtils {
     return new Path(remoteWALDir).getFileSystem(conf);
   }
 
-  public static Path getRemoteWALDirForPeer(String remoteWALDir, String peerId) {
+  public static Path getPeerRemoteWALDir(String remoteWALDir, String peerId) {
     return new Path(remoteWALDir, peerId);
   }
 
-  public static Path getRemoteWALDirForPeer(Path remoteWALDir, String peerId) {
+  public static Path getPeerRemoteWALDir(Path remoteWALDir, String peerId) {
     return new Path(remoteWALDir, peerId);
   }
 
+  public static Path getPeerReplayWALDir(Path remoteWALDir, String peerId) {
+    return getPeerRemoteWALDir(remoteWALDir, peerId).suffix(REMOTE_WAL_REPLAY_SUFFIX);
+  }
+
+  public static Path getPeerSnapshotWALDir(String remoteWALDir, String peerId) {
+    return getPeerRemoteWALDir(remoteWALDir, peerId).suffix(REMOTE_WAL_SNAPSHOT_SUFFIX);
+  }
+
+  public static Path getPeerSnapshotWALDir(Path remoteWALDir, String peerId) {
+    return getPeerRemoteWALDir(remoteWALDir, peerId).suffix(REMOTE_WAL_SNAPSHOT_SUFFIX);
+  }
+
   /**
    * Do the sleeping logic
    * @param msg Why we sleep

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
index 1645d68..7ffd3da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
@@ -22,9 +22,9 @@ import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHel
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.completeFile;
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.endFileLease;
 import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.getStatus;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
 import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.READER_IDLE;
 import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.WRITER_IDLE;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -40,7 +40,6 @@ import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.Encryptor;
 import org.apache.hadoop.fs.Path;

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java
index e9e3a97..9860774 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java
@@ -50,7 +50,7 @@ public class RecoverStandbyProcedure extends AbstractPeerProcedure<RecoverStandb
     switch (state) {
       case RENAME_SYNC_REPLICATION_WALS_DIR:
         try {
-          replaySyncReplicationWALManager.renamePeerRemoteWALDir(peerId);
+          replaySyncReplicationWALManager.renameToPeerReplayWALDir(peerId);
         } catch (IOException e) {
           LOG.warn("Failed to rename remote wal dir for peer id={}", peerId, e);
           setFailure("master-recover-standby", e);
@@ -67,11 +67,11 @@ public class RecoverStandbyProcedure extends AbstractPeerProcedure<RecoverStandb
             .map(wal -> new ReplaySyncReplicationWALProcedure(peerId,
                 replaySyncReplicationWALManager.removeWALRootPath(wal)))
             .toArray(ReplaySyncReplicationWALProcedure[]::new));
-        setNextState(RecoverStandbyState.REMOVE_SYNC_REPLICATION_WALS_DIR);
+        setNextState(RecoverStandbyState.SNAPSHOT_SYNC_REPLICATION_WALS_DIR);
         return Flow.HAS_MORE_STATE;
-      case REMOVE_SYNC_REPLICATION_WALS_DIR:
+      case SNAPSHOT_SYNC_REPLICATION_WALS_DIR:
         try {
-          replaySyncReplicationWALManager.removePeerReplayWALDir(peerId);
+          replaySyncReplicationWALManager.renameToPeerSnapshotWALDir(peerId);
         } catch (IOException e) {
           LOG.warn("Failed to cleanup replay wals dir for peer id={}, , retry", peerId, e);
           throw new ProcedureYieldException();
@@ -85,7 +85,7 @@ public class RecoverStandbyProcedure extends AbstractPeerProcedure<RecoverStandb
   private List<Path> getReplayWALs(ReplaySyncReplicationWALManager replaySyncReplicationWALManager)
       throws ProcedureYieldException {
     try {
-      return replaySyncReplicationWALManager.getReplayWALs(peerId);
+      return replaySyncReplicationWALManager.getReplayWALsAndCleanUpUnusedFiles(peerId);
     } catch (IOException e) {
       LOG.warn("Failed to get replay wals for peer id={}, , retry", peerId, e);
       throw new ProcedureYieldException();

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index 7335fe0..254448a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -67,10 +67,7 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
   }
 
   private void removeRemoteWALs(MasterProcedureEnv env) throws IOException {
-    ReplaySyncReplicationWALManager remoteWALManager =
-        env.getMasterServices().getReplaySyncReplicationWALManager();
-    remoteWALManager.removePeerRemoteWALs(peerId);
-    remoteWALManager.removePeerReplayWALDir(peerId);
+    env.getMasterServices().getReplaySyncReplicationWALManager().removePeerRemoteWALs(peerId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
index eac5aa4..348c134 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hbase.master.replication;
 
+import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerRemoteWALDir;
+import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerReplayWALDir;
+import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerSnapshotWALDir;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -25,31 +29,27 @@ import java.util.Map;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 @InterfaceAudience.Private
 public class ReplaySyncReplicationWALManager {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(ReplaySyncReplicationWALManager.class);
 
-  private static final String REPLAY_SUFFIX = "-replay";
-
   private final MasterServices services;
 
-  private final Configuration conf;
-
   private final FileSystem fs;
 
   private final Path walRootDir;
@@ -60,69 +60,86 @@ public class ReplaySyncReplicationWALManager {
 
   public ReplaySyncReplicationWALManager(MasterServices services) {
     this.services = services;
-    this.conf = services.getConfiguration();
     this.fs = services.getMasterFileSystem().getWALFileSystem();
     this.walRootDir = services.getMasterFileSystem().getWALRootDir();
     this.remoteWALDir = new Path(this.walRootDir, ReplicationUtils.REMOTE_WAL_DIR_NAME);
   }
 
-  public Path getPeerRemoteWALDir(String peerId) {
-    return new Path(this.remoteWALDir, peerId);
-  }
-
-  private Path getPeerReplayWALDir(String peerId) {
-    return getPeerRemoteWALDir(peerId).suffix(REPLAY_SUFFIX);
-  }
-
   public void createPeerRemoteWALDir(String peerId) throws IOException {
-    Path peerRemoteWALDir = getPeerRemoteWALDir(peerId);
+    Path peerRemoteWALDir = getPeerRemoteWALDir(remoteWALDir, peerId);
     if (!fs.exists(peerRemoteWALDir) && !fs.mkdirs(peerRemoteWALDir)) {
       throw new IOException("Unable to mkdir " + peerRemoteWALDir);
     }
   }
 
-  public void renamePeerRemoteWALDir(String peerId) throws IOException {
-    Path peerRemoteWALDir = getPeerRemoteWALDir(peerId);
-    Path peerReplayWALDir = peerRemoteWALDir.suffix(REPLAY_SUFFIX);
-    if (fs.exists(peerRemoteWALDir)) {
-      if (!fs.rename(peerRemoteWALDir, peerReplayWALDir)) {
-        throw new IOException("Failed rename remote wal dir from " + peerRemoteWALDir + " to "
-            + peerReplayWALDir + " for peer id=" + peerId);
+  private void rename(Path src, Path dst, String peerId) throws IOException {
+    if (fs.exists(src)) {
+      deleteDir(dst, peerId);
+      if (!fs.rename(src, dst)) {
+        throw new IOException(
+          "Failed to rename dir from " + src + " to " + dst + " for peer id=" + peerId);
       }
-      LOG.info("Rename remote wal dir from {} to {} for peer id={}", remoteWALDir, peerReplayWALDir,
-        peerId);
-    } else if (!fs.exists(peerReplayWALDir)) {
-      throw new IOException("Remote wal dir " + peerRemoteWALDir + " and replay wal dir "
-          + peerReplayWALDir + " not exist for peer id=" + peerId);
+      LOG.info("Renamed dir from {} to {} for peer id={}", src, dst, peerId);
+    } else if (!fs.exists(dst)) {
+      throw new IOException(
+        "Want to rename from " + src + " to " + dst + ", but they both do not exist");
     }
   }
 
-  public List<Path> getReplayWALs(String peerId) throws IOException {
-    Path peerReplayWALDir = getPeerReplayWALDir(peerId);
-    List<Path> replayWals = new ArrayList<>();
-    RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(peerReplayWALDir, false);
-    while (iterator.hasNext()) {
-      replayWals.add(iterator.next().getPath());
+  public void renameToPeerReplayWALDir(String peerId) throws IOException {
+    rename(getPeerRemoteWALDir(remoteWALDir, peerId), getPeerReplayWALDir(remoteWALDir, peerId),
+      peerId);
+  }
+
+  public void renameToPeerSnapshotWALDir(String peerId) throws IOException {
+    rename(getPeerReplayWALDir(remoteWALDir, peerId), getPeerSnapshotWALDir(remoteWALDir, peerId),
+      peerId);
+  }
+
+  public List<Path> getReplayWALsAndCleanUpUnusedFiles(String peerId) throws IOException {
+    Path peerReplayWALDir = getPeerReplayWALDir(remoteWALDir, peerId);
+    for (FileStatus status : fs.listStatus(peerReplayWALDir,
+      p -> p.getName().endsWith(ReplicationUtils.RENAME_WAL_SUFFIX))) {
+      Path src = status.getPath();
+      String srcName = src.getName();
+      String dstName =
+        srcName.substring(0, srcName.length() - ReplicationUtils.RENAME_WAL_SUFFIX.length());
+      FSUtils.renameFile(fs, src, new Path(src.getParent(), dstName));
+    }
+    List<Path> wals = new ArrayList<>();
+    for (FileStatus status : fs.listStatus(peerReplayWALDir)) {
+      Path path = status.getPath();
+      if (path.getName().endsWith(ReplicationUtils.SYNC_WAL_SUFFIX)) {
+        wals.add(path);
+      } else {
+        if (!fs.delete(path, true)) {
+          LOG.warn("Can not delete unused file: " + path);
+        }
+      }
     }
-    return replayWals;
+    return wals;
   }
 
-  public void removePeerReplayWALDir(String peerId) throws IOException {
-    Path peerReplayWALDir = getPeerReplayWALDir(peerId);
+  public void snapshotPeerReplayWALDir(String peerId) throws IOException {
+    Path peerReplayWALDir = getPeerReplayWALDir(remoteWALDir, peerId);
     if (fs.exists(peerReplayWALDir) && !fs.delete(peerReplayWALDir, true)) {
       throw new IOException(
           "Failed to remove replay wals dir " + peerReplayWALDir + " for peer id=" + peerId);
     }
   }
 
-  public void removePeerRemoteWALs(String peerId) throws IOException {
-    Path remoteWALDir = getPeerRemoteWALDir(peerId);
-    if (fs.exists(remoteWALDir) && !fs.delete(remoteWALDir, true)) {
-      throw new IOException(
-          "Failed to remove remote WALs dir " + remoteWALDir + " for peer id=" + peerId);
+  private void deleteDir(Path dir, String peerId) throws IOException {
+    if (!fs.delete(dir, true) && fs.exists(dir)) {
+      throw new IOException("Failed to remove dir " + dir + " for peer id=" + peerId);
     }
   }
 
+  public void removePeerRemoteWALs(String peerId) throws IOException {
+    deleteDir(getPeerRemoteWALDir(remoteWALDir, peerId), peerId);
+    deleteDir(getPeerReplayWALDir(remoteWALDir, peerId), peerId);
+    deleteDir(getPeerSnapshotWALDir(remoteWALDir, peerId), peerId);
+  }
+
   public void initPeerWorkers(String peerId) {
     BlockingQueue<ServerName> servers = new LinkedBlockingQueue<>();
     services.getServerManager().getOnlineServers().keySet()
@@ -144,4 +161,9 @@ public class ReplaySyncReplicationWALManager {
     // remove the "/" too.
     return pathStr.substring(walRootDir.toString().length() + 1);
   }
+
+  @VisibleForTesting
+  public Path getRemoteWALDir() {
+    return remoteWALDir;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 ebe7a93..81ee6b6 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
@@ -118,7 +118,7 @@ public class TransitPeerSyncReplicationStateProcedure
       env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, toState);
     if (toState == SyncReplicationState.ACTIVE) {
       Path remoteWALDirForPeer =
-        ReplicationUtils.getRemoteWALDirForPeer(desc.getPeerConfig().getRemoteWALDir(), peerId);
+        ReplicationUtils.getPeerRemoteWALDir(desc.getPeerConfig().getRemoteWALDir(), peerId);
       // check whether the remote wal directory is present
       if (!remoteWALDirForPeer.getFileSystem(env.getMasterConfiguration())
         .exists(remoteWALDirForPeer)) {
@@ -152,7 +152,7 @@ public class TransitPeerSyncReplicationStateProcedure
       throws ProcedureYieldException, IOException {
     MasterFileSystem mfs = env.getMasterFileSystem();
     Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
-    Path remoteWALDirForPeer = ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId);
+    Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId);
     FileSystem walFs = mfs.getWALFileSystem();
     if (walFs.exists(remoteWALDirForPeer)) {
       LOG.warn("Wal dir {} already exists, usually this should not happen, continue anyway",

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 5052a0b..90f3099 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
@@ -1957,8 +1957,7 @@ public class HRegionServer extends HasThread implements
     sinkConf.setInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER, 1);
     if (this.csm != null) {
       // SplitLogWorker needs csm. If none, don't start this.
-      this.splitLogWorker = new SplitLogWorker(this, sinkConf, this,
-          this, walFactory);
+      this.splitLogWorker = new SplitLogWorker(sinkConf, this, this, walFactory);
       splitLogWorker.start();
     } else {
       LOG.warn("SplitLogWorker Service NOT started; CoordinatedStateManager is null");

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
index 4529943..09ec477 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.regionserver.PeerProcedureHandler;
 import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -37,4 +38,9 @@ public interface ReplicationSourceService extends ReplicationService {
    * Returns a Handler to handle peer procedures.
    */
   PeerProcedureHandler getPeerProcedureHandler();
+
+  /**
+   * Return the replication peers.
+   */
+  ReplicationPeers getReplicationPeers();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
index a1c2030..4a9712c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
@@ -23,22 +23,31 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.ConnectException;
 import java.net.SocketTimeoutException;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.Optional;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
+import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
@@ -67,67 +76,133 @@ public class SplitLogWorker implements Runnable {
   Thread worker;
   // thread pool which executes recovery work
   private SplitLogWorkerCoordination coordination;
-  private Configuration conf;
   private RegionServerServices server;
 
   public SplitLogWorker(Server hserver, Configuration conf, RegionServerServices server,
       TaskExecutor splitTaskExecutor) {
     this.server = server;
-    this.conf = conf;
     this.coordination = hserver.getCoordinatedStateManager().getSplitLogWorkerCoordination();
     coordination.init(server, conf, splitTaskExecutor, this);
   }
 
-  public SplitLogWorker(final Server hserver, final Configuration conf,
-      final RegionServerServices server, final LastSequenceId sequenceIdChecker,
-      final WALFactory factory) {
-    this(hserver, conf, server, new TaskExecutor() {
-      @Override
-      public Status exec(String filename, CancelableProgressable p) {
-        Path walDir;
-        FileSystem fs;
-        try {
-          walDir = FSUtils.getWALRootDir(conf);
-          fs = walDir.getFileSystem(conf);
-        } catch (IOException e) {
-          LOG.warn("could not find root dir or fs", e);
-          return Status.RESIGNED;
-        }
-        // TODO have to correctly figure out when log splitting has been
-        // interrupted or has encountered a transient error and when it has
-        // encountered a bad non-retry-able persistent error.
-        try {
-          if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, filename)),
-            fs, conf, p, sequenceIdChecker,
-              server.getCoordinatedStateManager().getSplitLogWorkerCoordination(), factory)) {
-            return Status.PREEMPTED;
-          }
-        } catch (InterruptedIOException iioe) {
-          LOG.warn("log splitting of " + filename + " interrupted, resigning", iioe);
-          return Status.RESIGNED;
-        } catch (IOException e) {
-          if (e instanceof FileNotFoundException) {
-            // A wal file may not exist anymore. Nothing can be recovered so move on
-            LOG.warn("WAL {} does not exist anymore", filename, e);
-            return Status.DONE;
-          }
-          Throwable cause = e.getCause();
-          if (e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException
-                  || cause instanceof ConnectException
-                  || cause instanceof SocketTimeoutException)) {
-            LOG.warn("log replaying of " + filename + " can't connect to the target regionserver, "
-                + "resigning", e);
-            return Status.RESIGNED;
-          } else if (cause instanceof InterruptedException) {
-            LOG.warn("log splitting of " + filename + " interrupted, resigning", e);
-            return Status.RESIGNED;
-          }
-          LOG.warn("log splitting of " + filename + " failed, returning error", e);
-          return Status.ERR;
-        }
+  public SplitLogWorker(Configuration conf, RegionServerServices server,
+      LastSequenceId sequenceIdChecker, WALFactory factory) {
+    this(server, conf, server, (f, p) -> splitLog(f, p, conf, server, sequenceIdChecker, factory));
+  }
+
+  // returns whether we need to continue the split work
+  private static boolean processSyncReplicationWAL(String name, Configuration conf,
+      RegionServerServices server, FileSystem fs, Path walDir) throws IOException {
+    Path walFile = new Path(walDir, name);
+    String filename = walFile.getName();
+    Optional<String> optSyncPeerId =
+      SyncReplicationWALProvider.getSyncReplicationPeerIdFromWALName(filename);
+    if (!optSyncPeerId.isPresent()) {
+      return true;
+    }
+    String peerId = optSyncPeerId.get();
+    ReplicationPeerImpl peer =
+      server.getReplicationSourceService().getReplicationPeers().getPeer(peerId);
+    if (peer == null || !peer.getPeerConfig().isSyncReplication()) {
+      return true;
+    }
+    Pair<SyncReplicationState, SyncReplicationState> stateAndNewState =
+      peer.getSyncReplicationStateAndNewState();
+    if (stateAndNewState.getFirst().equals(SyncReplicationState.ACTIVE) &&
+      stateAndNewState.getSecond().equals(SyncReplicationState.NONE)) {
+      // copy the file to remote and overwrite the previous one
+      String remoteWALDir = peer.getPeerConfig().getRemoteWALDir();
+      Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId);
+      Path tmpRemoteWAL = new Path(remoteWALDirForPeer, filename + ".tmp");
+      FileSystem remoteFs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
+      try (FSDataInputStream in = fs.open(walFile); @SuppressWarnings("deprecation")
+      FSDataOutputStream out = remoteFs.createNonRecursive(tmpRemoteWAL, true,
+        FSUtils.getDefaultBufferSize(remoteFs), remoteFs.getDefaultReplication(tmpRemoteWAL),
+        remoteFs.getDefaultBlockSize(tmpRemoteWAL), null)) {
+        IOUtils.copy(in, out);
+      }
+      Path toCommitRemoteWAL =
+        new Path(remoteWALDirForPeer, filename + ReplicationUtils.RENAME_WAL_SUFFIX);
+      // Some FileSystem implementations may not support atomic rename so we need to do it in two
+      // phases
+      FSUtils.renameFile(remoteFs, tmpRemoteWAL, toCommitRemoteWAL);
+      FSUtils.renameFile(remoteFs, toCommitRemoteWAL, new Path(remoteWALDirForPeer, filename));
+    } else if ((stateAndNewState.getFirst().equals(SyncReplicationState.ACTIVE) &&
+      stateAndNewState.getSecond().equals(SyncReplicationState.STANDBY)) ||
+      stateAndNewState.getFirst().equals(SyncReplicationState.STANDBY)) {
+      // check whether we still need to process this file
+      // actually we only write wal file which name is ended with .syncrep in A state, and after
+      // transiting to a state other than A, we will reopen all the regions so the data in the wal
+      // will be flushed so the wal file will be archived soon. But it is still possible that there
+      // is a server crash when we are transiting from A to S, to simplify the logic of the transit
+      // procedure, here we will also check the remote snapshot directory in state S, so that we do
+      // not need wait until all the wal files with .syncrep suffix to be archived before finishing
+      // the procedure.
+      String remoteWALDir = peer.getPeerConfig().getRemoteWALDir();
+      Path remoteSnapshotDirForPeer = ReplicationUtils.getPeerSnapshotWALDir(remoteWALDir, peerId);
+      FileSystem remoteFs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
+      if (remoteFs.exists(new Path(remoteSnapshotDirForPeer, filename))) {
+        // the file has been replayed when the remote cluster was transited from S to DA, the
+        // content will be replicated back to us so give up split it.
+        LOG.warn("Giveup splitting {} since it has been replayed in the remote cluster and " +
+          "the content will be replicated back", filename);
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static Status splitLog(String name, CancelableProgressable p, Configuration conf,
+      RegionServerServices server, LastSequenceId sequenceIdChecker, WALFactory factory) {
+    Path walDir;
+    FileSystem fs;
+    try {
+      walDir = FSUtils.getWALRootDir(conf);
+      fs = walDir.getFileSystem(conf);
+    } catch (IOException e) {
+      LOG.warn("could not find root dir or fs", e);
+      return Status.RESIGNED;
+    }
+    try {
+      if (!processSyncReplicationWAL(name, conf, server, fs, walDir)) {
         return Status.DONE;
       }
-    });
+    } catch (IOException e) {
+      LOG.warn("failed to process sync replication wal {}", name, e);
+      return Status.RESIGNED;
+    }
+    // TODO have to correctly figure out when log splitting has been
+    // interrupted or has encountered a transient error and when it has
+    // encountered a bad non-retry-able persistent error.
+    try {
+      if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, name)), fs, conf,
+        p, sequenceIdChecker, server.getCoordinatedStateManager().getSplitLogWorkerCoordination(),
+        factory)) {
+        return Status.PREEMPTED;
+      }
+    } catch (InterruptedIOException iioe) {
+      LOG.warn("log splitting of " + name + " interrupted, resigning", iioe);
+      return Status.RESIGNED;
+    } catch (IOException e) {
+      if (e instanceof FileNotFoundException) {
+        // A wal file may not exist anymore. Nothing can be recovered so move on
+        LOG.warn("WAL {} does not exist anymore", name, e);
+        return Status.DONE;
+      }
+      Throwable cause = e.getCause();
+      if (e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException ||
+        cause instanceof ConnectException || cause instanceof SocketTimeoutException)) {
+        LOG.warn("log replaying of " + name + " can't connect to the target regionserver, " +
+          "resigning", e);
+        return Status.RESIGNED;
+      } else if (cause instanceof InterruptedException) {
+        LOG.warn("log splitting of " + name + " interrupted, resigning", e);
+        return Status.RESIGNED;
+      }
+      LOG.warn("log splitting of " + name + " failed, returning error", e);
+      return Status.ERR;
+    }
+    return Status.DONE;
   }
 
   @Override
@@ -191,6 +266,7 @@ public class SplitLogWorker implements Runnable {
    * {@link org.apache.hadoop.hbase.master.SplitLogManager} commit the work in
    * SplitLogManager.TaskFinisher
    */
+  @FunctionalInterface
   public interface TaskExecutor {
     enum Status {
       DONE(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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
index 8ecfede..4301ae7 100644
--- 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
@@ -32,13 +32,13 @@ 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 {
+public final class CombinedAsyncWriter implements AsyncWriter {
 
   private static final Logger LOG = LoggerFactory.getLogger(CombinedAsyncWriter.class);
 
-  protected final ImmutableList<AsyncWriter> writers;
+  private final ImmutableList<AsyncWriter> writers;
 
-  protected CombinedAsyncWriter(ImmutableList<AsyncWriter> writers) {
+  private CombinedAsyncWriter(ImmutableList<AsyncWriter> writers) {
     this.writers = writers;
   }
 
@@ -66,69 +66,29 @@ public abstract class CombinedAsyncWriter implements AsyncWriter {
     }
   }
 
-  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
+  @Override
+  public CompletableFuture<Long> sync() {
+    CompletableFuture<Long> future = new CompletableFuture<>();
+    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);
+      }
+    }));
+    return future;
   }
 
-  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);
-    }
+  public static CombinedAsyncWriter create(AsyncWriter writer, AsyncWriter... writers) {
+    return new CombinedAsyncWriter(
+      ImmutableList.<AsyncWriter> builder().add(writer).add(writers).build());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 a98567a..3967e78 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
@@ -25,6 +25,7 @@ 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.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
 
@@ -50,6 +51,13 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
     this.remoteWalDir = remoteWalDir;
   }
 
+  // will be overridden in testcase
+  @VisibleForTesting
+  protected AsyncWriter createCombinedAsyncWriter(AsyncWriter localWriter,
+      AsyncWriter remoteWriter) {
+    return CombinedAsyncWriter.create(remoteWriter, localWriter);
+  }
+
   @Override
   protected AsyncWriter createWriterInstance(Path path) throws IOException {
     AsyncWriter localWriter = super.createWriterInstance(path);
@@ -66,8 +74,7 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
         closeWriter(localWriter);
       }
     }
-    return CombinedAsyncWriter.create(CombinedAsyncWriter.Mode.SEQUENTIAL, remoteWriter,
-      localWriter);
+    return createCombinedAsyncWriter(localWriter, remoteWriter);
   }
 
   // Allow temporarily skipping the creation of remote writer. When failing to write to the remote

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 2199415..b04f0cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -288,4 +288,9 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
   public SyncReplicationPeerInfoProvider getSyncReplicationPeerInfoProvider() {
     return syncReplicationPeerInfoProvider;
   }
+
+  @Override
+  public ReplicationPeers getReplicationPeers() {
+    return replicationPeers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index f25b073..827cfa9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -652,7 +652,7 @@ public class ReplicationSourceManager implements ReplicationListener {
 
   private void removeRemoteWALs(String peerId, String remoteWALDir, Collection<String> wals)
       throws IOException {
-    Path remoteWALDirForPeer = ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId);
+    Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId);
     FileSystem fs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
     for (String wal : wals) {
       Path walFile = new Path(remoteWALDirForPeer, wal);

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
index 75274ea..170441b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
@@ -77,8 +77,7 @@ class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProv
       return false;
     }
     Pair<SyncReplicationState, SyncReplicationState> states =
-        peer.getSyncReplicationStateAndNewState();
+      peer.getSyncReplicationStateAndNewState();
     return checker.test(states.getFirst(), states.getSecond());
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 8a1f948..5b968db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -843,6 +843,15 @@ public abstract class FSUtils extends CommonFSUtils {
     return frags;
   }
 
+  public static void renameFile(FileSystem fs, Path src, Path dst) throws IOException {
+    if (fs.exists(dst) && !fs.delete(dst, false)) {
+      throw new IOException("Can not delete " + dst);
+    }
+    if (!fs.rename(src, dst)) {
+      throw new IOException("Can not rename from " + src + " to " + dst);
+    }
+  }
+
   /**
    * A {@link PathFilter} that returns only regular files.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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
index 8e82d8b..82f8a89 100644
--- 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
@@ -21,6 +21,8 @@ import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALArchiveDir
 import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALDirectoryName;
 
 import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Optional;
@@ -51,6 +53,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 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;
@@ -67,8 +70,9 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
 
   private static final Logger LOG = LoggerFactory.getLogger(SyncReplicationWALProvider.class);
 
+  // only for injecting errors for testcase, do not use it for other purpose.
   @VisibleForTesting
-  public static final String LOG_SUFFIX = ".syncrep";
+  public static final String DUAL_WAL_IMPL = "hbase.wal.sync.impl";
 
   private final WALProvider provider;
 
@@ -126,12 +130,35 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   }
 
   private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException {
-    return new DualAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf),
-      ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir),
-      CommonFSUtils.getWALRootDir(conf),
-      ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId),
-      getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
-      conf, listeners, true, getLogPrefix(peerId), LOG_SUFFIX, eventLoopGroup, channelClass);
+    Class<? extends DualAsyncFSWAL> clazz =
+      conf.getClass(DUAL_WAL_IMPL, DualAsyncFSWAL.class, DualAsyncFSWAL.class);
+    try {
+      Constructor<?> constructor = null;
+      for (Constructor<?> c : clazz.getDeclaredConstructors()) {
+        if (c.getParameterCount() > 0) {
+          constructor = c;
+          break;
+        }
+      }
+      if (constructor == null) {
+        throw new IllegalArgumentException("No valid constructor provided for class " + clazz);
+      }
+      constructor.setAccessible(true);
+      return (DualAsyncFSWAL) constructor.newInstance(
+        CommonFSUtils.getWALFileSystem(conf),
+        ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir),
+        CommonFSUtils.getWALRootDir(conf),
+        ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId),
+        getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
+        conf, listeners, true, getLogPrefix(peerId), ReplicationUtils.SYNC_WAL_SUFFIX,
+        eventLoopGroup, channelClass);
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new RuntimeException(e);
+    } catch (InvocationTargetException e) {
+      Throwable cause = e.getTargetException();
+      Throwables.propagateIfPossible(cause, IOException.class);
+      throw new RuntimeException(cause);
+    }
   }
 
   private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException {
@@ -304,7 +331,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
    * </p>
    */
   public static Optional<String> getSyncReplicationPeerIdFromWALName(String name) {
-    if (!name.endsWith(LOG_SUFFIX)) {
+    if (!name.endsWith(ReplicationUtils.SYNC_WAL_SUFFIX)) {
       // fast path to return earlier if the name is not for a sync replication peer.
       return Optional.empty();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 c6ffeea..6462234 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
@@ -1141,7 +1141,7 @@ public class TestReplicationAdmin {
       LOG.info("Expected error:", e);
     }
     TEST_UTIL.getTestFileSystem()
-      .mkdirs(ReplicationUtils.getRemoteWALDirForPeer(rootDir, ID_SECOND));
+      .mkdirs(ReplicationUtils.getPeerRemoteWALDir(rootDir, ID_SECOND));
     hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
     assertEquals(SyncReplicationState.ACTIVE,
       hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 07aa6a8..f73b4f1 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
@@ -18,8 +18,6 @@
 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;
@@ -39,23 +37,18 @@ 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 {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestCombinedAsyncWriter.class);
+    HBaseClassTestRule.forClass(TestCombinedAsyncWriter.class);
 
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
@@ -68,15 +61,6 @@ public class TestCombinedAsyncWriter {
   @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();
@@ -125,7 +109,7 @@ public class TestCombinedAsyncWriter {
         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)) {
+      CombinedAsyncWriter writer = CombinedAsyncWriter.create(writer1, writer2)) {
       ProtobufLogTestHelper.doWrite(new WriterOverAsyncWriter(writer), withTrailer, tableName,
         columnCount, recordCount, row, timestamp);
       try (ProtobufLogReader reader = (ProtobufLogReader) WALS.createReader(fs, path1)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java
new file mode 100644
index 0000000..fb3daf2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
+class DualAsyncFSWALForTest extends DualAsyncFSWAL {
+
+  private boolean localBroken;
+
+  private boolean remoteBroken;
+
+  private CountDownLatch arrive;
+
+  private CountDownLatch resume;
+
+  private final class MyCombinedAsyncWriter implements AsyncWriter {
+
+    private final AsyncWriter localWriter;
+
+    private final AsyncWriter remoteWriter;
+
+    public MyCombinedAsyncWriter(AsyncWriter localWriter, AsyncWriter remoteWriter) {
+      this.localWriter = localWriter;
+      this.remoteWriter = remoteWriter;
+    }
+
+    @Override
+    public long getLength() {
+      return localWriter.getLength();
+    }
+
+    @Override
+    public void close() throws IOException {
+      Closeables.close(localWriter, true);
+      Closeables.close(remoteWriter, true);
+    }
+
+    @Override
+    public CompletableFuture<Long> sync() {
+      CompletableFuture<Long> localFuture;
+      CompletableFuture<Long> remoteFuture;
+      if (!localBroken) {
+        localFuture = localWriter.sync();
+      } else {
+        localFuture = new CompletableFuture<>();
+        localFuture.completeExceptionally(new IOException("Inject error"));
+      }
+      if (!remoteBroken) {
+        remoteFuture = remoteWriter.sync();
+      } else {
+        remoteFuture = new CompletableFuture<>();
+        remoteFuture.completeExceptionally(new IOException("Inject error"));
+      }
+      return CompletableFuture.allOf(localFuture, remoteFuture).thenApply(v -> {
+        return localFuture.getNow(0L);
+      });
+    }
+
+    @Override
+    public void append(Entry entry) {
+      if (!localBroken) {
+        localWriter.append(entry);
+      }
+      if (!remoteBroken) {
+        remoteWriter.append(entry);
+      }
+    }
+  }
+
+  public DualAsyncFSWALForTest(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, remoteFs, rootDir, remoteWalDir, logDir, archiveDir, conf, listeners, failIfWALExists,
+      prefix, suffix, eventLoopGroup, channelClass);
+  }
+
+  @Override
+  protected AsyncWriter createCombinedAsyncWriter(AsyncWriter localWriter,
+      AsyncWriter remoteWriter) {
+    return new MyCombinedAsyncWriter(localWriter, remoteWriter);
+  }
+
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    if (arrive != null) {
+      arrive.countDown();
+      try {
+        resume.await();
+      } catch (InterruptedException e) {
+      }
+    }
+    if (localBroken || remoteBroken) {
+      throw new IOException("WAL broken");
+    }
+    return super.createWriterInstance(path);
+  }
+
+  public void setLocalBroken() {
+    this.localBroken = true;
+  }
+
+  public void setRemoteBroken() {
+    this.remoteBroken = true;
+  }
+
+  public void suspendLogRoll() {
+    arrive = new CountDownLatch(1);
+    resume = new CountDownLatch(1);
+  }
+
+  public void waitUntilArrive() throws InterruptedException {
+    arrive.await();
+  }
+
+  public void resumeLogRoll() {
+    resume.countDown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
index de679be..095be90 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -72,9 +72,9 @@ public class SyncReplicationTestBase {
 
   protected static String PEER_ID = "1";
 
-  protected static Path remoteWALDir1;
+  protected static Path REMOTE_WAL_DIR1;
 
-  protected static Path remoteWALDir2;
+  protected static Path REMOTE_WAL_DIR2;
 
   private static void initTestingUtility(HBaseTestingUtility util, String zkParent) {
     util.setZkCluster(ZK_UTIL.getZkCluster());
@@ -109,22 +109,22 @@ public class SyncReplicationTestBase {
     UTIL2.getAdmin().createTable(td);
     FileSystem fs1 = UTIL1.getTestFileSystem();
     FileSystem fs2 = UTIL2.getTestFileSystem();
-    remoteWALDir1 =
+    REMOTE_WAL_DIR1 =
       new Path(UTIL1.getMiniHBaseCluster().getMaster().getMasterFileSystem().getWALRootDir(),
         "remoteWALs").makeQualified(fs1.getUri(), fs1.getWorkingDirectory());
-    remoteWALDir2 =
+    REMOTE_WAL_DIR2 =
       new Path(UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem().getWALRootDir(),
         "remoteWALs").makeQualified(fs2.getUri(), fs2.getWorkingDirectory());
     UTIL1.getAdmin().addReplicationPeer(PEER_ID,
       ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey())
         .setReplicateAllUserTables(false)
         .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>()))
-        .setRemoteWALDir(remoteWALDir2.toUri().toString()).build());
+        .setRemoteWALDir(REMOTE_WAL_DIR2.toUri().toString()).build());
     UTIL2.getAdmin().addReplicationPeer(PEER_ID,
       ReplicationPeerConfig.newBuilder().setClusterKey(UTIL1.getClusterKey())
         .setReplicateAllUserTables(false)
         .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>()))
-        .setRemoteWALDir(remoteWALDir1.toUri().toString()).build());
+        .setRemoteWALDir(REMOTE_WAL_DIR1.toUri().toString()).build());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
index b663c44..fce0cdf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
@@ -37,8 +37,7 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestSyncReplicationActive.class);
-
+    HBaseClassTestRule.forClass(TestSyncReplicationActive.class);
 
   @Test
   public void testActive() throws Exception {
@@ -58,7 +57,7 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
     verifyNotReplicatedThroughRegion(UTIL2, 0, 100);
 
     // Ensure that there's no cluster id in remote log entries.
-    verifyNoClusterIdInRemoteLog(UTIL2, remoteWALDir2, PEER_ID);
+    verifyNoClusterIdInRemoteLog(UTIL2, REMOTE_WAL_DIR2, PEER_ID);
 
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.DOWNGRADE_ACTIVE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalCopyToRemote.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalCopyToRemote.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalCopyToRemote.java
new file mode 100644
index 0000000..cf8993b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalCopyToRemote.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSyncReplicationMoreLogsInLocalCopyToRemote extends SyncReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSyncReplicationMoreLogsInLocalCopyToRemote.class);
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TestSyncReplicationMoreLogsInLocalCopyToRemote.class);
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL1.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL,
+      DualAsyncFSWALForTest.class, DualAsyncFSWAL.class);
+    UTIL2.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL,
+      DualAsyncFSWALForTest.class, DualAsyncFSWAL.class);
+    SyncReplicationTestBase.setUp();
+  }
+
+  @Test
+  public void testSplitLog() throws Exception {
+    UTIL1.getAdmin().disableReplicationPeer(PEER_ID);
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.ACTIVE);
+    HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME);
+    DualAsyncFSWALForTest wal =
+      (DualAsyncFSWALForTest) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build());
+    wal.setRemoteBroken();
+    try (AsyncConnection conn =
+      ConnectionFactory.createAsyncConnection(UTIL1.getConfiguration()).get()) {
+      AsyncTable<?> table = conn.getTableBuilder(TABLE_NAME).setMaxAttempts(1).build();
+      try {
+        table.put(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, Bytes.toBytes(0))).get();
+        fail("Should fail since the rs will crash and we will not retry");
+      } catch (ExecutionException e) {
+        // expected
+        LOG.info("Expected error:", e);
+      }
+    }
+    UTIL1.waitFor(60000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) {
+          return table.exists(new Get(Bytes.toBytes(0)));
+        }
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "The row is still not available";
+      }
+    });
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    // We should have copied the local log to remote, so we should be able to get the value
+    try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
+      assertEquals(0, Bytes.toInt(table.get(new Get(Bytes.toBytes(0))).getValue(CF, CQ)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalGiveUpSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalGiveUpSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalGiveUpSplitting.java
new file mode 100644
index 0000000..9a6d242
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalGiveUpSplitting.java
@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSyncReplicationMoreLogsInLocalGiveUpSplitting extends SyncReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSyncReplicationMoreLogsInLocalGiveUpSplitting.class);
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TestSyncReplicationMoreLogsInLocalGiveUpSplitting.class);
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL1.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL,
+      DualAsyncFSWALForTest.class, DualAsyncFSWAL.class);
+    UTIL2.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL,
+      DualAsyncFSWALForTest.class, DualAsyncFSWAL.class);
+    SyncReplicationTestBase.setUp();
+  }
+
+  @Test
+  public void testSplitLog() throws Exception {
+    UTIL1.getAdmin().disableReplicationPeer(PEER_ID);
+    UTIL2.getAdmin().disableReplicationPeer(PEER_ID);
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.ACTIVE);
+    try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) {
+      table.put(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, Bytes.toBytes(0)));
+    }
+    HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME);
+    DualAsyncFSWALForTest wal =
+      (DualAsyncFSWALForTest) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build());
+    wal.setRemoteBroken();
+    wal.suspendLogRoll();
+    try (AsyncConnection conn =
+      ConnectionFactory.createAsyncConnection(UTIL1.getConfiguration()).get()) {
+      AsyncTable<?> table = conn.getTableBuilder(TABLE_NAME).setMaxAttempts(1)
+        .setWriteRpcTimeout(5, TimeUnit.SECONDS).build();
+      try {
+        table.put(new Put(Bytes.toBytes(1)).addColumn(CF, CQ, Bytes.toBytes(1))).get();
+        fail("Should fail since the rs will hang and we will get a rpc timeout");
+      } catch (ExecutionException e) {
+        // expected
+        LOG.info("Expected error:", e);
+      }
+    }
+    wal.waitUntilArrive();
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    wal.resumeLogRoll();
+    try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
+      assertEquals(0, Bytes.toInt(table.get(new Get(Bytes.toBytes(0))).getValue(CF, CQ)));
+      // we failed to write this entry to remote so it should not exist
+      assertFalse(table.exists(new Get(Bytes.toBytes(1))));
+    }
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    // make sure that the region is online. We can not use waitTableAvailable since the table in
+    // stand by state can not be read from client.
+    try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) {
+      try {
+        table.exists(new Get(Bytes.toBytes(0)));
+      } catch (DoNotRetryIOException | RetriesExhaustedException e) {
+        // expected
+        assertThat(e.getMessage(), containsString("STANDBY"));
+      }
+    }
+    HRegion region = UTIL1.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    // we give up splitting the whole wal file so this record will also be gone.
+    assertTrue(region.get(new Get(Bytes.toBytes(0))).isEmpty());
+    UTIL2.getAdmin().enableReplicationPeer(PEER_ID);
+    // finally it should be replicated back
+    waitUntilReplicationDone(UTIL1, 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java
index 7d380c1..0cd1846 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -66,12 +65,12 @@ public class TestSyncReplicationRemoveRemoteWAL extends SyncReplicationTestBase
       SyncReplicationState.ACTIVE);
 
     MasterFileSystem mfs = UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    Path remoteWALDir = ReplicationUtils.getRemoteWALDirForPeer(
+    Path remoteWALDir = ReplicationUtils.getPeerRemoteWALDir(
       new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME), PEER_ID);
     FileStatus[] remoteWALStatus = mfs.getWALFileSystem().listStatus(remoteWALDir);
     assertEquals(1, remoteWALStatus.length);
     Path remoteWAL = remoteWALStatus[0].getPath();
-    assertThat(remoteWAL.getName(), endsWith(SyncReplicationWALProvider.LOG_SUFFIX));
+    assertThat(remoteWAL.getName(), endsWith(ReplicationUtils.SYNC_WAL_SUFFIX));
     writeAndVerifyReplication(UTIL1, UTIL2, 0, 100);
 
     HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME);
@@ -81,7 +80,7 @@ public class TestSyncReplicationRemoveRemoteWAL extends SyncReplicationTestBase
     remoteWALStatus = mfs.getWALFileSystem().listStatus(remoteWALDir);
     assertEquals(1, remoteWALStatus.length);
     remoteWAL = remoteWALStatus[0].getPath();
-    assertThat(remoteWAL.getName(), endsWith(SyncReplicationWALProvider.LOG_SUFFIX));
+    assertThat(remoteWAL.getName(), endsWith(ReplicationUtils.SYNC_WAL_SUFFIX));
 
     UTIL1.getAdmin().disableReplicationPeer(PEER_ID);
     write(UTIL1, 100, 200);

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
index 8526af8..de409fc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java
@@ -97,25 +97,25 @@ public class TestSyncReplicationStandBy extends SyncReplicationTestBase {
     writeAndVerifyReplication(UTIL1, UTIL2, 0, 100);
 
     // Remove the peers in ACTIVE & STANDBY cluster.
-    FileSystem fs2 = remoteWALDir2.getFileSystem(UTIL2.getConfiguration());
-    Assert.assertTrue(fs2.exists(getRemoteWALDir(remoteWALDir2, PEER_ID)));
+    FileSystem fs2 = REMOTE_WAL_DIR2.getFileSystem(UTIL2.getConfiguration());
+    Assert.assertTrue(fs2.exists(getRemoteWALDir(REMOTE_WAL_DIR2, PEER_ID)));
 
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.DOWNGRADE_ACTIVE);
-    Assert.assertFalse(fs2.exists(getRemoteWALDir(remoteWALDir2, PEER_ID)));
-    Assert.assertFalse(fs2.exists(getReplayRemoteWALs(remoteWALDir2, PEER_ID)));
+    Assert.assertFalse(fs2.exists(getRemoteWALDir(REMOTE_WAL_DIR2, PEER_ID)));
+    Assert.assertFalse(fs2.exists(getReplayRemoteWALs(REMOTE_WAL_DIR2, PEER_ID)));
 
     UTIL1.getAdmin().removeReplicationPeer(PEER_ID);
-    verifyRemovedPeer(PEER_ID, remoteWALDir1, UTIL1);
+    verifyRemovedPeer(PEER_ID, REMOTE_WAL_DIR1, UTIL1);
 
     // Peer remoteWAL dir will be renamed to replay WAL dir when transit from S to DA, and the
     // replay WAL dir will be removed after replaying all WALs, so create a emtpy dir here to test
     // whether the removeReplicationPeer would remove the remoteWAL dir.
-    fs2.create(getRemoteWALDir(remoteWALDir2, PEER_ID));
-    fs2.create(getReplayRemoteWALs(remoteWALDir2, PEER_ID));
-    Assert.assertTrue(fs2.exists(getRemoteWALDir(remoteWALDir2, PEER_ID)));
-    Assert.assertTrue(fs2.exists(getReplayRemoteWALs(remoteWALDir2, PEER_ID)));
+    fs2.create(getRemoteWALDir(REMOTE_WAL_DIR2, PEER_ID));
+    fs2.create(getReplayRemoteWALs(REMOTE_WAL_DIR2, PEER_ID));
+    Assert.assertTrue(fs2.exists(getRemoteWALDir(REMOTE_WAL_DIR2, PEER_ID)));
+    Assert.assertTrue(fs2.exists(getReplayRemoteWALs(REMOTE_WAL_DIR2, PEER_ID)));
     UTIL2.getAdmin().removeReplicationPeer(PEER_ID);
-    verifyRemovedPeer(PEER_ID, remoteWALDir2, UTIL2);
+    verifyRemovedPeer(PEER_ID, REMOTE_WAL_DIR2, UTIL2);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
index ebb21a4..2563669 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -151,7 +152,8 @@ public class TestRecoverStandbyProcedure {
   }
 
   private void setupSyncReplicationWALs() throws IOException, StreamLacksCapabilityException {
-    Path peerRemoteWALDir = replaySyncReplicationWALManager.getPeerRemoteWALDir(PEER_ID);
+    Path peerRemoteWALDir = ReplicationUtils
+      .getPeerRemoteWALDir(replaySyncReplicationWALManager.getRemoteWALDir(), PEER_ID);
     if (!fs.exists(peerRemoteWALDir)) {
       fs.mkdirs(peerRemoteWALDir);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 d98b7f85..febe764 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
@@ -84,7 +84,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -618,7 +617,7 @@ public abstract class TestReplicationSourceManager {
     try {
       // make sure that we can deal with files which does not exist
       String walNameNotExists =
-        "remoteWAL-12345-" + slaveId + ".12345" + SyncReplicationWALProvider.LOG_SUFFIX;
+        "remoteWAL-12345-" + slaveId + ".12345" + ReplicationUtils.SYNC_WAL_SUFFIX;
       Path wal = new Path(logDir, walNameNotExists);
       manager.preLogRoll(wal);
       manager.postLogRoll(wal);
@@ -626,7 +625,7 @@ public abstract class TestReplicationSourceManager {
       Path remoteLogDirForPeer = new Path(remoteLogDir, slaveId);
       fs.mkdirs(remoteLogDirForPeer);
       String walName =
-        "remoteWAL-12345-" + slaveId + ".23456" + SyncReplicationWALProvider.LOG_SUFFIX;
+        "remoteWAL-12345-" + slaveId + ".23456" + ReplicationUtils.SYNC_WAL_SUFFIX;
       Path remoteWAL =
         new Path(remoteLogDirForPeer, walName).makeQualified(fs.getUri(), fs.getWorkingDirectory());
       fs.create(remoteWAL).close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/46866d9f/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 69ed44d..8189cef 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
@@ -86,7 +86,6 @@ public class TestSyncReplicationWALProvider {
     @Override
     public boolean checkState(TableName table,
         BiPredicate<SyncReplicationState, SyncReplicationState> checker) {
-      // TODO Implement SyncReplicationPeerInfoProvider.isInState
       return false;
     }
   }


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

Branch: refs/heads/HBASE-19064
Commit: 2fb1022b040977c81ef6094712de5587f40d897c
Parents: 2c4c1a8
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 19 18:38:39 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 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   |   4 -
 .../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(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2fb1022b/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 825ad17..4255086 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
@@ -434,6 +434,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/2fb1022b/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 4732f41..d98ab75 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
@@ -248,7 +248,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/2fb1022b/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 42b0dae..0495337 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/2fb1022b/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 61b9cfb..baa87a4 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
@@ -214,12 +214,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
     this.lowReplicationRollLimit = conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit",
       5);
     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 2);
-
     this.useHsync = conf.getBoolean(HRegion.WAL_HSYNC_CONF_KEY, HRegion.DEFAULT_WAL_HSYNC);
 
-    // 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/2fb1022b/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/2fb1022b/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/2fb1022b/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/2fb1022b/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 c920279..56edb75 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;
@@ -32,12 +31,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}.
@@ -62,6 +59,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),
@@ -74,15 +72,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/2fb1022b/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 12b63f5..7f33eda 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
@@ -27,6 +27,9 @@ import org.apache.yetus.audience.InterfaceAudience;
 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}.
@@ -57,7 +60,10 @@ public final 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/2fb1022b/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/2fb1022b/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/2fb1022b/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 24604d9..339fd6c 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;
@@ -132,13 +133,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);
@@ -150,9 +148,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;
   }
@@ -184,6 +183,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/2fb1022b/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 8828239..a7e4670 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
@@ -130,13 +130,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/2fb1022b/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/2fb1022b/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 3cf06d4..1490653 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
-  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(boolean forceSync) throws IOException {
-              if (throwSyncException) {
-                throw new IOException("FAKE! Failed to replace a bad datanode...");
-              }
-              w.sync(forceSync);
-            }
+    @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(boolean forceSync) throws IOException {
+          if (throwSyncException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...");
+          }
+          w.sync(forceSync);
+        }
 
-            @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
+  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/2fb1022b/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 1ff6b27..e95639f 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();
@@ -342,6 +340,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);
 
@@ -353,7 +352,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);
@@ -390,6 +388,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);
@@ -1165,6 +1164,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 {
@@ -1196,7 +1196,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);
@@ -2448,6 +2448,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);
@@ -2500,9 +2501,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;
@@ -3796,9 +3797,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();
@@ -3894,7 +3898,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/2fb1022b/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 ce83326..84f7973 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/2fb1022b/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/2fb1022b/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 29a75b8..84b8d6c 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
@@ -215,6 +215,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/2fb1022b/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 e7cdf1f..93c379c 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/2fb1022b/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 aece961..420585f 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(false);
     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/2fb1022b/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/2fb1022b/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/2fb1022b/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 cb8edc6..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(), TestCombinedAsyncWriter.class.getSimpleName());
+    WALS = new WALFactory(UTIL.getConfiguration(), TestCombinedAsyncWriter.class.getSimpleName());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/2fb1022b/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 7baaa6c..f288f74 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
@@ -74,8 +74,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
@@ -83,8 +85,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() {
@@ -92,6 +94,8 @@ public class TestFSHLog extends AbstractTestFSWAL {
         super.atHeadOfRingBufferEventHandlerAppend();
       }
     };
+    wal.init();
+    return wal;
   }
 
   @Test
@@ -100,6 +104,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);
@@ -142,7 +147,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/2fb1022b/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/2fb1022b/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 01f0dc6..453b742 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
@@ -103,8 +103,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/2fb1022b/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());
+  }
+}


[23/36] hbase git commit: HBASE-19990 Create remote wal directory when transitting to state S

Posted by zh...@apache.org.
HBASE-19990 Create remote wal directory when transitting to state S


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

Branch: refs/heads/HBASE-19064
Commit: 7f1dbdfdd58fcbc8490440ce0a202b12c10151eb
Parents: ca94b15
Author: zhangduo <zh...@apache.org>
Authored: Wed Feb 14 16:01:16 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../procedure2/ProcedureYieldException.java     |  9 ++++--
 .../hbase/replication/ReplicationUtils.java     |  2 ++
 .../hadoop/hbase/master/MasterFileSystem.java   | 19 ++++++-------
 .../master/procedure/MasterProcedureEnv.java    |  5 ++++
 ...ransitPeerSyncReplicationStateProcedure.java | 29 ++++++++++++++++----
 .../hbase/replication/TestSyncReplication.java  |  8 ++++++
 6 files changed, 55 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7f1dbdfd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java
index 0487ac5b..dbb9981 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java
@@ -15,16 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.procedure2;
 
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
-// TODO: Not used yet
+/**
+ * Indicate that a procedure wants to be rescheduled. Usually because there are something wrong but
+ * we do not want to fail the procedure.
+ * <p>
+ * TODO: need to support scheduling after a delay.
+ */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class ProcedureYieldException extends ProcedureException {
+
   /** default constructor */
   public ProcedureYieldException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f1dbdfd/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 d94cb00..e402d0f 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
@@ -41,6 +41,8 @@ public final class ReplicationUtils {
 
   public static final String REPLICATION_ATTR_NAME = "__rep__";
 
+  public static final String REMOTE_WAL_DIR_NAME = "remoteWALs";
+
   private ReplicationUtils() {
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f1dbdfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 864be02..7ccbd71 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -133,7 +134,6 @@ public class MasterFileSystem {
    * Idempotent.
    */
   private void createInitialFileSystemLayout() throws IOException {
-
     final String[] protectedSubDirs = new String[] {
         HConstants.BASE_NAMESPACE_DIR,
         HConstants.HFILE_ARCHIVE_DIRECTORY,
@@ -145,7 +145,8 @@ public class MasterFileSystem {
       HConstants.HREGION_LOGDIR_NAME,
       HConstants.HREGION_OLDLOGDIR_NAME,
       HConstants.CORRUPT_DIR_NAME,
-      WALProcedureStore.MASTER_PROCEDURE_LOGDIR
+      WALProcedureStore.MASTER_PROCEDURE_LOGDIR,
+      ReplicationUtils.REMOTE_WAL_DIR_NAME
     };
     // check if the root directory exists
     checkRootDir(this.rootdir, conf, this.fs);
@@ -192,7 +193,9 @@ public class MasterFileSystem {
     return this.fs;
   }
 
-  protected FileSystem getWALFileSystem() { return this.walFs; }
+  public FileSystem getWALFileSystem() {
+    return this.walFs;
+  }
 
   public Configuration getConfiguration() {
     return this.conf;
@@ -234,13 +237,9 @@ public class MasterFileSystem {
   }
 
   /**
-   * Get the rootdir.  Make sure its wholesome and exists before returning.
-   * @param rd
-   * @param c
-   * @param fs
-   * @return hbase.rootdir (after checks for existence and bootstrapping if
-   * needed populating the directory with necessary bootup files).
-   * @throws IOException
+   * Get the rootdir. Make sure its wholesome and exists before returning.
+   * @return hbase.rootdir (after checks for existence and bootstrapping if needed populating the
+   *         directory with necessary bootup files).
    */
   private Path checkRootDir(final Path rd, final Configuration c, final FileSystem fs)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f1dbdfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index 7fb187f..490879f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
@@ -142,6 +143,10 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return master.getReplicationPeerManager();
   }
 
+  public MasterFileSystem getMasterFileSystem() {
+    return master.getMasterFileSystem();
+  }
+
   public boolean isRunning() {
     if (this.master == null || this.master.getMasterProcedureExecutor() == null) return false;
     return master.getMasterProcedureExecutor().isRunning();

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f1dbdfd/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 69404a0..cc51890 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
@@ -20,14 +20,18 @@ package org.apache.hadoop.hbase.master.replication;
 import java.io.IOException;
 import java.util.List;
 import java.util.stream.Collectors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 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.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -44,7 +48,7 @@ public class TransitPeerSyncReplicationStateProcedure
     extends AbstractPeerProcedure<PeerSyncReplicationStateTransitionState> {
 
   private static final Logger LOG =
-    LoggerFactory.getLogger(TransitPeerSyncReplicationStateProcedure.class);
+      LoggerFactory.getLogger(TransitPeerSyncReplicationStateProcedure.class);
 
   private SyncReplicationState fromState;
 
@@ -67,8 +71,8 @@ public class TransitPeerSyncReplicationStateProcedure
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
     TransitPeerSyncReplicationStateStateData.Builder builder =
-      TransitPeerSyncReplicationStateStateData.newBuilder()
-        .setToState(ReplicationPeerConfigUtil.toSyncReplicationState(toState));
+        TransitPeerSyncReplicationStateStateData.newBuilder()
+          .setToState(ReplicationPeerConfigUtil.toSyncReplicationState(toState));
     if (fromState != null) {
       builder.setFromState(ReplicationPeerConfigUtil.toSyncReplicationState(fromState));
     }
@@ -79,7 +83,7 @@ public class TransitPeerSyncReplicationStateProcedure
   protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.deserializeStateData(serializer);
     TransitPeerSyncReplicationStateStateData data =
-      serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
+        serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
     toState = ReplicationPeerConfigUtil.toSyncReplicationState(data.getToState());
     if (data.hasFromState()) {
       fromState = ReplicationPeerConfigUtil.toSyncReplicationState(data.getFromState());
@@ -205,7 +209,22 @@ public class TransitPeerSyncReplicationStateProcedure
         }
         return Flow.HAS_MORE_STATE;
       case CREATE_DIR_FOR_REMOTE_WAL:
-        // TODO: create wal for write remote wal
+        MasterFileSystem mfs = env.getMasterFileSystem();
+        Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
+        Path remoteWALDirForPeer = new Path(remoteWALDir, peerId);
+        FileSystem walFs = mfs.getWALFileSystem();
+        try {
+          if (walFs.exists(remoteWALDirForPeer)) {
+            LOG.warn("Wal dir {} already exists, usually this should not happen, continue anyway",
+              remoteWALDirForPeer);
+          } else if (!walFs.mkdirs(remoteWALDirForPeer)) {
+            LOG.warn("Can not create remote wal dir {}", remoteWALDirForPeer);
+            throw new ProcedureYieldException();
+          }
+        } catch (IOException e) {
+          LOG.warn("Failed to create remote wal dir {}", remoteWALDirForPeer, e);
+          throw new ProcedureYieldException();
+        }
         setNextState(
           PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION);
         return Flow.HAS_MORE_STATE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/7f1dbdfd/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
index acddc4a..196019d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
@@ -19,7 +19,9 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -45,6 +47,7 @@ import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -154,8 +157,13 @@ public class TestSyncReplication {
 
   @Test
   public void testStandby() throws Exception {
+    MasterFileSystem mfs = UTIL2.getHBaseCluster().getMaster().getMasterFileSystem();
+    Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
+    Path remoteWALDirForPeer = new Path(remoteWALDir, PEER_ID);
+    assertFalse(mfs.getWALFileSystem().exists(remoteWALDirForPeer));
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.STANDBY);
+    assertTrue(mfs.getWALFileSystem().exists(remoteWALDirForPeer));
     try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
       assertDisallow(table, t -> t.get(new Get(Bytes.toBytes("row"))));
       assertDisallow(table,


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

Branch: refs/heads/HBASE-19064
Commit: 0d3c6676f4b3b6a5773f64d530667417d2ee4e92
Parents: 5f4bc77
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Feb 6 16:00:59 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 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/0d3c6676/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 97abc74..997a155 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;
@@ -220,6 +222,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/0d3c6676/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 f07a0d8..ff778a8 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
@@ -170,7 +170,7 @@ public class ReplicationPeerManager {
               " does not match new remote wal dir '" + peerConfig.getRemoteWALDir() + "'");
     }
 
-    if (oldPeerConfig.getRemoteWALDir() != null) {
+    if (oldPeerConfig.isSyncReplication()) {
       if (!ReplicationUtils.isNamespacesAndTableCFsEqual(oldPeerConfig, peerConfig)) {
         throw new DoNotRetryIOException(
           "Changing the replicated namespace/table config on a synchronous replication " +
@@ -199,8 +199,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));
@@ -324,9 +324,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/0d3c6676/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/0d3c6676/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/0d3c6676/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";


[35/36] hbase git commit: HBASE-19782 Reject the replication request when peer is DA or A state

Posted by zh...@apache.org.
HBASE-19782 Reject the replication request when peer is DA or A state


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

Branch: refs/heads/HBASE-19064
Commit: e4676d957621258fbb4180154b420eed93bc7a18
Parents: d1f4b41
Author: huzheng <op...@gmail.com>
Authored: Fri Mar 2 18:05:29 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/protobuf/ReplicationProtbufUtil.java  |  2 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  2 +-
 .../hbase/regionserver/HRegionServer.java       |  5 +--
 .../hbase/regionserver/RSRpcServices.java       | 25 +++++++++--
 .../RejectReplicationRequestStateChecker.java   | 45 ++++++++++++++++++++
 .../ReplaySyncReplicationWALCallable.java       | 24 ++++++-----
 .../replication/regionserver/Replication.java   |  2 +-
 .../regionserver/ReplicationSink.java           | 16 +++----
 .../SyncReplicationPeerInfoProvider.java        | 11 ++---
 .../SyncReplicationPeerInfoProviderImpl.java    | 13 +++---
 .../SyncReplicationPeerMappingManager.java      |  5 +--
 .../hbase/wal/SyncReplicationWALProvider.java   |  7 +--
 .../replication/SyncReplicationTestBase.java    | 32 ++++++++++++++
 .../replication/TestSyncReplicationActive.java  | 13 +++++-
 .../regionserver/TestReplicationSink.java       |  5 +--
 .../regionserver/TestWALEntrySinkFilter.java    |  3 +-
 .../wal/TestSyncReplicationWALProvider.java     |  6 +--
 17 files changed, 163 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index 81dd59e..e01f881 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.SizedCellScanner;
@@ -45,7 +46,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminServic
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
 
 @InterfaceAudience.Private
 public class ReplicationProtbufUtil {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 42a86c4..8dbccc5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1990,7 +1990,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private boolean shouldForbidMajorCompaction() {
     if (rsServices != null && rsServices.getReplicationSourceService() != null) {
       return rsServices.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
-          .checkState(getRegionInfo(), ForbidMajorCompactionChecker.get());
+          .checkState(getRegionInfo().getTable(), ForbidMajorCompactionChecker.get());
     }
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/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 4605280..5052a0b 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
@@ -2475,10 +2475,9 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * @return Return the object that implements the replication
-   * sink executorService.
+   * @return Return the object that implements the replication sink executorService.
    */
-  ReplicationSinkService getReplicationSinkService() {
+  public ReplicationSinkService getReplicationSinkService() {
     return replicationSinkHandler;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 5316ac5..bdb86d0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -121,6 +121,7 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.regionserver.RejectReplicationRequestStateChecker;
 import org.apache.hadoop.hbase.replication.regionserver.RejectRequestsFromClientStateChecker;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
@@ -2204,9 +2205,26 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
   }
 
+  private void checkShouldRejectReplicationRequest(List<WALEntry> entries) throws IOException {
+    ReplicationSourceService replicationSource = regionServer.getReplicationSourceService();
+    if (replicationSource == null || entries.isEmpty()) {
+      return;
+    }
+    // We can ensure that all entries are for one peer, so only need to check one entry's
+    // table name. if the table hit sync replication at peer side and the peer cluster
+    // is (or is transiting to) state ACTIVE or DOWNGRADE_ACTIVE, we should reject to apply
+    // those entries according to the design doc.
+    TableName table = TableName.valueOf(entries.get(0).getKey().getTableName().toByteArray());
+    if (replicationSource.getSyncReplicationPeerInfoProvider().checkState(table,
+      RejectReplicationRequestStateChecker.get())) {
+      throw new DoNotRetryIOException(
+          "Reject to apply to sink cluster because sync replication state of sink cluster "
+              + "is ACTIVE or DOWNGRADE_ACTIVE, table: " + table);
+    }
+  }
+
   /**
    * Replicate WAL entries on the region server.
-   *
    * @param controller the RPC controller
    * @param request the request
    * @throws ServiceException
@@ -2220,7 +2238,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       if (regionServer.replicationSinkHandler != null) {
         requestCount.increment();
         List<WALEntry> entries = request.getEntryList();
-        CellScanner cellScanner = ((HBaseRpcController)controller).cellScanner();
+        checkShouldRejectReplicationRequest(entries);
+        CellScanner cellScanner = ((HBaseRpcController) controller).cellScanner();
         regionServer.getRegionServerCoprocessorHost().preReplicateLogEntries();
         regionServer.replicationSinkHandler.replicateLogEntries(entries, cellScanner,
           request.getReplicationClusterId(), request.getSourceBaseNamespaceDirPath(),
@@ -2435,7 +2454,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
   private boolean shouldRejectRequestsFromClient(HRegion region) {
     return regionServer.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
-      .checkState(region.getRegionInfo(), RejectRequestsFromClientStateChecker.get());
+      .checkState(region.getRegionInfo().getTable(), RejectRequestsFromClientStateChecker.get());
   }
 
   private void rejectIfInStandByState(HRegion region) throws DoNotRetryIOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java
new file mode 100644
index 0000000..9ad0af2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed 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.function.BiPredicate;
+
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Check whether we need to reject the replication request from source cluster.
+ */
+@InterfaceAudience.Private
+public class RejectReplicationRequestStateChecker
+    implements BiPredicate<SyncReplicationState, SyncReplicationState> {
+
+  private static final RejectReplicationRequestStateChecker INST =
+      new RejectReplicationRequestStateChecker();
+
+  @Override
+  public boolean test(SyncReplicationState state, SyncReplicationState newState) {
+    return state == SyncReplicationState.ACTIVE || state == SyncReplicationState.DOWNGRADE_ACTIVE
+        || newState == SyncReplicationState.ACTIVE
+        || newState == SyncReplicationState.DOWNGRADE_ACTIVE;
+  }
+
+  public static RejectReplicationRequestStateChecker get() {
+    return INST;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
index c9c5ef6..3cf065c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -46,6 +44,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ReplaySyncReplicationWALParameter;
 
 /**
@@ -81,14 +80,19 @@ public class ReplaySyncReplicationWALCallable implements RSProcedureCallable {
       throw initError;
     }
     LOG.info("Received a replay sync replication wal {} event, peerId={}", wal, peerId);
-    try (Reader reader = getReader()) {
-      List<Entry> entries = readWALEntries(reader);
-      while (!entries.isEmpty()) {
-        Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> pair = ReplicationProtbufUtil
-            .buildReplicateWALEntryRequest(entries.toArray(new Entry[entries.size()]));
-        HBaseRpcController controller = new HBaseRpcControllerImpl(pair.getSecond());
-        rs.getRSRpcServices().replicateWALEntry(controller, pair.getFirst());
-        entries = readWALEntries(reader);
+    if (rs.getReplicationSinkService() != null) {
+      try (Reader reader = getReader()) {
+        List<Entry> entries = readWALEntries(reader);
+        while (!entries.isEmpty()) {
+          Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> pair = ReplicationProtbufUtil
+              .buildReplicateWALEntryRequest(entries.toArray(new Entry[entries.size()]));
+          ReplicateWALEntryRequest request = pair.getFirst();
+          rs.getReplicationSinkService().replicateLogEntries(request.getEntryList(),
+            pair.getSecond(), request.getReplicationClusterId(),
+            request.getSourceBaseNamespaceDirPath(), request.getSourceHFileArchiveDirPath());
+          // Read next entries.
+          entries = readWALEntries(reader);
+        }
       }
     }
     return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 2846d2c..2199415 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -275,7 +275,7 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
     List<ReplicationSourceInterface> oldSources = this.replicationManager.getOldSources();
     for (ReplicationSourceInterface source : oldSources) {
       if (source instanceof ReplicationSource) {
-        sourceMetricsList.add(((ReplicationSource) source).getSourceMetrics());
+        sourceMetricsList.add(source.getSourceMetrics());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index eb09a3a..a334b16 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -28,6 +28,7 @@ import java.util.Map.Entry;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -93,9 +94,8 @@ public class ReplicationSink {
 
   /**
    * Create a sink for replication
-   *
-   * @param conf                conf object
-   * @param stopper             boolean to tell this thread to stop
+   * @param conf conf object
+   * @param stopper boolean to tell this thread to stop
    * @throws IOException thrown when HDFS goes bad or bad file name
    */
   public ReplicationSink(Configuration conf, Stoppable stopper)
@@ -104,16 +104,15 @@ public class ReplicationSink {
     decorateConf();
     this.metrics = new MetricsSink();
     this.walEntrySinkFilter = setupWALEntrySinkFilter();
-    String className =
-        conf.get("hbase.replication.source.fs.conf.provider",
-          DefaultSourceFSConfigurationProvider.class.getCanonicalName());
+    String className = conf.get("hbase.replication.source.fs.conf.provider",
+      DefaultSourceFSConfigurationProvider.class.getCanonicalName());
     try {
       Class<? extends SourceFSConfigurationProvider> c =
           Class.forName(className).asSubclass(SourceFSConfigurationProvider.class);
       this.provider = c.getDeclaredConstructor().newInstance();
     } catch (Exception e) {
       throw new IllegalArgumentException(
-        "Configured source fs configuration provider class " + className + " throws error.", e);
+          "Configured source fs configuration provider class " + className + " throws error.", e);
     }
   }
 
@@ -178,8 +177,7 @@ public class ReplicationSink {
       Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap = null;
 
       for (WALEntry entry : entries) {
-        TableName table =
-            TableName.valueOf(entry.getKey().getTableName().toByteArray());
+        TableName table = TableName.valueOf(entry.getKey().getTableName().toByteArray());
         if (this.walEntrySinkFilter != null) {
           if (this.walEntrySinkFilter.filter(table, entry.getKey().getWriteTime())) {
             // Skip Cells in CellScanner associated with this entry.

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java
index 66fe3be..cfe525a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java
@@ -19,7 +19,8 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.util.Optional;
 import java.util.function.BiPredicate;
-import org.apache.hadoop.hbase.client.RegionInfo;
+
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -31,17 +32,17 @@ import org.apache.yetus.audience.InterfaceAudience;
 public interface SyncReplicationPeerInfoProvider {
 
   /**
-   * Return the peer id and remote WAL directory if the region is synchronously replicated and the
+   * Return the peer id and remote WAL directory if the table is synchronously replicated and the
    * state is {@link SyncReplicationState#ACTIVE}.
    */
-  Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info);
+  Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(TableName table);
 
   /**
-   * Check whether the give region is contained in a sync replication peer which can pass the state
+   * Check whether the given table is contained in a sync replication peer which can pass the state
    * checker.
    * <p>
    * Will call the checker with current sync replication state and new sync replication state.
    */
-  boolean checkState(RegionInfo info,
+  boolean checkState(TableName table,
       BiPredicate<SyncReplicationState, SyncReplicationState> checker);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
index cb33dab..75274ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
@@ -19,7 +19,8 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.util.Optional;
 import java.util.function.BiPredicate;
-import org.apache.hadoop.hbase.client.RegionInfo;
+
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
@@ -40,11 +41,11 @@ class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProv
   }
 
   @Override
-  public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
-    if (info == null) {
+  public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(TableName table) {
+    if (table == null) {
       return Optional.empty();
     }
-    String peerId = mapping.getPeerId(info);
+    String peerId = mapping.getPeerId(table);
     if (peerId == null) {
       return Optional.empty();
     }
@@ -65,9 +66,9 @@ class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProv
   }
 
   @Override
-  public boolean checkState(RegionInfo info,
+  public boolean checkState(TableName table,
       BiPredicate<SyncReplicationState, SyncReplicationState> checker) {
-    String peerId = mapping.getPeerId(info);
+    String peerId = mapping.getPeerId(table);
     if (peerId == null) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java
index 64216cb..5d19f72 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -42,7 +41,7 @@ class SyncReplicationPeerMappingManager {
     peerConfig.getTableCFsMap().keySet().forEach(table2PeerId::remove);
   }
 
-  String getPeerId(RegionInfo info) {
-    return table2PeerId.get(info.getTable());
+  String getPeerId(TableName tableName) {
+    return table2PeerId.get(tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/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
index 3cd356d42..3b56aa2 100644
--- 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
@@ -33,6 +33,7 @@ import java.util.function.BiPredicate;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
@@ -160,7 +161,7 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
     }
     WAL wal = null;
     Optional<Pair<String, String>> peerIdAndRemoteWALDir =
-      peerInfoProvider.getPeerIdAndRemoteWALDir(region);
+        peerInfoProvider.getPeerIdAndRemoteWALDir(region.getTable());
     if (peerIdAndRemoteWALDir.isPresent()) {
       Pair<String, String> pair = peerIdAndRemoteWALDir.get();
       wal = getWAL(pair.getFirst(), pair.getSecond());
@@ -273,12 +274,12 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
       implements SyncReplicationPeerInfoProvider {
 
     @Override
-    public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
+    public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(TableName table) {
       return Optional.empty();
     }
 
     @Override
-    public boolean checkState(RegionInfo info,
+    public boolean checkState(TableName table,
         BiPredicate<SyncReplicationState, SyncReplicationState> checker) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
index 30dbdb5..0d5fce8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -25,11 +25,13 @@ import java.util.ArrayList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -37,9 +39,15 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
@@ -182,4 +190,28 @@ public class SyncReplicationTestBase {
     Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
     return new Path(remoteWALDir, PEER_ID);
   }
+
+  protected void verifyReplicationRequestRejection(HBaseTestingUtility utility,
+      boolean expectedRejection) throws Exception {
+    HRegionServer regionServer = utility.getRSForFirstRegionInTable(TABLE_NAME);
+    ClusterConnection connection = regionServer.getClusterConnection();
+    Entry[] entries = new Entry[10];
+    for (int i = 0; i < entries.length; i++) {
+      entries[i] =
+          new Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TABLE_NAME, 0), new WALEdit());
+    }
+    if (!expectedRejection) {
+      ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),
+        entries, null, null, null);
+    } else {
+      try {
+        ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),
+          entries, null, null, null);
+        Assert.fail("Should throw IOException when sync-replication state is in A or DA");
+      } catch (DoNotRetryIOException e) {
+        Assert.assertTrue(e.getMessage().contains("Reject to apply to sink cluster"));
+        Assert.assertTrue(e.getMessage().contains(TABLE_NAME.toString()));
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
index f4fb5fe..bff4572 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java
@@ -29,7 +29,8 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestSyncReplicationActive.class);
+      HBaseClassTestRule.forClass(TestSyncReplicationActive.class);
+
 
   @Test
   public void testActive() throws Exception {
@@ -37,13 +38,21 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
       SyncReplicationState.STANDBY);
     UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.ACTIVE);
+
+    // confirm that peer with state A will reject replication request.
+    verifyReplicationRequestRejection(UTIL1, true);
+    verifyReplicationRequestRejection(UTIL2, false);
+
     UTIL1.getAdmin().disableReplicationPeer(PEER_ID);
     write(UTIL1, 0, 100);
     Thread.sleep(2000);
     // peer is disabled so no data have been replicated
     verifyNotReplicatedThroughRegion(UTIL2, 0, 100);
+
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.DOWNGRADE_ACTIVE);
+    // confirm that peer with state DA will reject replication request.
+    verifyReplicationRequestRejection(UTIL2, true);
     // confirm that the data is there after we convert the peer to DA
     verify(UTIL2, 0, 100);
 
@@ -59,6 +68,8 @@ public class TestSyncReplicationActive extends SyncReplicationTestBase {
     // confirm that we can convert to DA even if the remote slave cluster is down
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.DOWNGRADE_ACTIVE);
+    // confirm that peer with state DA will reject replication request.
+    verifyReplicationRequestRejection(UTIL2, true);
     write(UTIL2, 200, 300);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index aa6c39c..2d6c28f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -129,8 +129,7 @@ public class TestReplicationSink {
       TestSourceFSConfigurationProvider.class.getCanonicalName());
 
     TEST_UTIL.startMiniCluster(3);
-    SINK =
-      new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE);
+    SINK = new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE);
     table1 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAME1);
     table2 = TEST_UTIL.createTable(TABLE_NAME2, FAM_NAME2);
     Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
@@ -419,7 +418,7 @@ public class TestReplicationSink {
     return builder.build();
   }
 
-  private WALEntry.Builder createWALEntryBuilder(TableName table) {
+  public static WALEntry.Builder createWALEntryBuilder(TableName table) {
     WALEntry.Builder builder = WALEntry.newBuilder();
     builder.setAssociatedCellCount(1);
     WALKey.Builder keyBuilder = WALKey.newBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java
index 6299065..fd9ff29 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellBuilder;
@@ -127,7 +128,7 @@ public class TestWALEntrySinkFilter {
     conf.setClass(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY,
         IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
     conf.setClass("hbase.client.connection.impl", DevNullConnection.class,
-        Connection.class);
+      Connection.class);
     ReplicationSink sink = new ReplicationSink(conf, STOPPABLE);
     // Create some dumb walentries.
     List< org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry > entries =

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4676d95/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 3263fe8..69ed44d 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
@@ -75,8 +75,8 @@ public class TestSyncReplicationWALProvider {
   public static final class InfoProvider implements SyncReplicationPeerInfoProvider {
 
     @Override
-    public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
-      if (info.getTable().equals(TABLE)) {
+    public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(TableName table) {
+      if (table != null && table.equals(TABLE)) {
         return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR));
       } else {
         return Optional.empty();
@@ -84,7 +84,7 @@ public class TestSyncReplicationWALProvider {
     }
 
     @Override
-    public boolean checkState(RegionInfo info,
+    public boolean checkState(TableName table,
         BiPredicate<SyncReplicationState, SyncReplicationState> checker) {
       // TODO Implement SyncReplicationPeerInfoProvider.isInState
       return false;


[27/36] hbase git commit: HBASE-20370 Also remove the wal file in remote cluster when we finish replicating a file

Posted by zh...@apache.org.
HBASE-20370 Also remove the wal file in remote cluster when we finish replicating a file


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

Branch: refs/heads/HBASE-19064
Commit: d1f4b41876b7a0227927160771e898e5737bc135
Parents: 327af33
Author: zhangduo <zh...@apache.org>
Authored: Tue Apr 17 09:04:56 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationUtils.java     |  36 ++++++-
 .../regionserver/ReplicationSource.java         |  38 +++----
 .../ReplicationSourceInterface.java             |  21 +++-
 .../regionserver/ReplicationSourceManager.java  | 108 ++++++++++++++-----
 .../regionserver/ReplicationSourceShipper.java  |  27 ++---
 .../hbase/wal/SyncReplicationWALProvider.java   |  11 +-
 .../replication/ReplicationSourceDummy.java     |  20 ++--
 .../TestReplicationSourceManager.java           | 101 ++++++++++++-----
 8 files changed, 246 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d1f4b418/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 cb22f57..66e9b01 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,14 +22,17 @@ 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Helper class for replication.
@@ -37,6 +40,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public final class ReplicationUtils {
 
+  private static final Logger LOG = LoggerFactory.getLogger(ReplicationUtils.class);
+
   public static final String REPLICATION_ATTR_NAME = "__rep__";
 
   public static final String REMOTE_WAL_DIR_NAME = "remoteWALs";
@@ -176,4 +181,33 @@ public final class ReplicationUtils {
       return tableCFs != null && tableCFs.containsKey(tableName);
     }
   }
+
+  public static FileSystem getRemoteWALFileSystem(Configuration conf, String remoteWALDir)
+      throws IOException {
+    return new Path(remoteWALDir).getFileSystem(conf);
+  }
+
+  public static Path getRemoteWALDirForPeer(String remoteWALDir, String peerId) {
+    return new Path(remoteWALDir, peerId);
+  }
+
+  /**
+   * Do the sleeping logic
+   * @param msg Why we sleep
+   * @param sleepForRetries the base sleep time.
+   * @param sleepMultiplier by how many times the default sleeping time is augmented
+   * @param maxRetriesMultiplier the max retry multiplier
+   * @return True if <code>sleepMultiplier</code> is &lt; <code>maxRetriesMultiplier</code>
+   */
+  public static boolean sleepForRetries(String msg, long sleepForRetries, int sleepMultiplier,
+      int maxRetriesMultiplier) {
+    try {
+      LOG.trace("{}, sleeping {} times {}", msg, sleepForRetries, sleepMultiplier);
+      Thread.sleep(sleepForRetries * sleepMultiplier);
+    } catch (InterruptedException e) {
+      LOG.debug("Interrupted while sleeping between retries");
+      Thread.currentThread().interrupt();
+    }
+    return sleepMultiplier < maxRetriesMultiplier;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d1f4b418/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 4051efe..f25a232 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -89,8 +89,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
 
   protected Configuration conf;
   protected ReplicationQueueInfo replicationQueueInfo;
-  // id of the peer cluster this source replicates to
-  private String peerId;
 
   // The manager of all sources to which we ping back our progress
   protected ReplicationSourceManager manager;
@@ -168,8 +166,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
 
     this.queueId = queueId;
     this.replicationQueueInfo = new ReplicationQueueInfo(queueId);
-    // ReplicationQueueInfo parses the peerId out of the znode for us
-    this.peerId = this.replicationQueueInfo.getPeerId();
     this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
 
     defaultBandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
@@ -177,8 +173,8 @@ public class ReplicationSource implements ReplicationSourceInterface {
     this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0);
     this.totalBufferUsed = manager.getTotalBufferUsed();
     this.walFileLengthProvider = walFileLengthProvider;
-    LOG.info("queueId=" + queueId + ", ReplicationSource : " + peerId
-        + ", currentBandwidth=" + this.currentBandwidth);
+    LOG.info("queueId={}, ReplicationSource : {}, currentBandwidth={}", queueId,
+      replicationPeer.getId(), this.currentBandwidth);
   }
 
   private void decorateConf() {
@@ -215,6 +211,7 @@ public class ReplicationSource implements ReplicationSourceInterface {
   @Override
   public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
       throws ReplicationException {
+    String peerId = replicationPeer.getId();
     Map<TableName, List<String>> tableCFMap = replicationPeer.getTableCFs();
     if (tableCFMap != null) {
       List<String> tableCfs = tableCFMap.get(tableName);
@@ -274,8 +271,8 @@ public class ReplicationSource implements ReplicationSourceInterface {
       tableDescriptors = ((HRegionServer) server).getTableDescriptors();
     }
     replicationEndpoint
-        .init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), fs, peerId,
-            clusterId, replicationPeer, metrics, tableDescriptors, server));
+      .init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), fs,
+        replicationPeer.getId(), clusterId, replicationPeer, metrics, tableDescriptors, server));
     replicationEndpoint.start();
     replicationEndpoint.awaitRunning(waitOnEndpointSeconds, TimeUnit.SECONDS);
   }
@@ -357,8 +354,8 @@ public class ReplicationSource implements ReplicationSourceInterface {
     if (peerBandwidth != currentBandwidth) {
       currentBandwidth = peerBandwidth;
       throttler.setBandwidth((double) currentBandwidth / 10.0);
-      LOG.info("ReplicationSource : " + peerId
-          + " bandwidth throttling changed, currentBandWidth=" + currentBandwidth);
+      LOG.info("ReplicationSource : {} bandwidth throttling changed, currentBandWidth={}",
+        replicationPeer.getId(), currentBandwidth);
     }
   }
 
@@ -387,15 +384,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
     return sleepMultiplier < maxRetriesMultiplier;
   }
 
-  /**
-   * check whether the peer is enabled or not
-   * @return true if the peer is enabled, otherwise false
-   */
-  @Override
-  public boolean isPeerEnabled() {
-    return replicationPeer.isPeerEnabled();
-  }
-
   private void initialize() {
     int sleepMultiplier = 1;
     while (this.isSourceActive()) {
@@ -529,11 +517,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
   }
 
   @Override
-  public String getPeerId() {
-    return this.peerId;
-  }
-
-  @Override
   public Path getCurrentPath() {
     // only for testing
     for (ReplicationSourceShipper worker : workerThreads.values()) {
@@ -616,6 +599,11 @@ public class ReplicationSource implements ReplicationSourceInterface {
     return server.getServerName();
   }
 
+  @Override
+  public ReplicationPeer getPeer() {
+    return replicationPeer;
+  }
+
   Server getServer() {
     return server;
   }
@@ -623,4 +611,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
   ReplicationQueueStorage getQueueStorage() {
     return queueStorage;
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d1f4b418/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index 090b465..3ce5bfe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -104,10 +104,17 @@ public interface ReplicationSourceInterface {
 
   /**
    * Get the id that the source is replicating to.
-   *
    * @return peer id
    */
-  String getPeerId();
+  default String getPeerId() {
+    return getPeer().getId();
+  }
+
+  /**
+   * Get the replication peer instance.
+   * @return the replication peer instance
+   */
+  ReplicationPeer getPeer();
 
   /**
    * Get a string representation of the current statistics
@@ -119,9 +126,17 @@ public interface ReplicationSourceInterface {
   /**
    * @return peer enabled or not
    */
-  boolean isPeerEnabled();
+  default boolean isPeerEnabled() {
+    return getPeer().isPeerEnabled();
+  }
 
   /**
+   * @return whether this is sync replication peer.
+   */
+  default boolean isSyncReplication() {
+    return getPeer().getPeerConfig().isSyncReplication();
+  }
+  /**
    * @return active or not
    */
   boolean isSourceActive();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d1f4b418/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 4212597..cbeba23 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -85,20 +87,20 @@ import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFacto
  * operations.</li>
  * <li>Need synchronized on {@link #walsById}. There are four methods which modify it,
  * {@link #addPeer(String)}, {@link #removePeer(String)},
- * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and {@link #preLogRoll(Path)}.
+ * {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} and {@link #preLogRoll(Path)}.
  * {@link #walsById} is a ConcurrentHashMap and there is a Lock for peer id in
  * {@link PeerProcedureHandlerImpl}. So there is no race between {@link #addPeer(String)} and
- * {@link #removePeer(String)}. {@link #cleanOldLogs(NavigableSet, String, boolean, String)} is
- * called by {@link ReplicationSourceInterface}. So no race with {@link #addPeer(String)}.
+ * {@link #removePeer(String)}. {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)}
+ * is called by {@link ReplicationSourceInterface}. So no race with {@link #addPeer(String)}.
  * {@link #removePeer(String)} will terminate the {@link ReplicationSourceInterface} firstly, then
  * remove the wals from {@link #walsById}. So no race with {@link #removePeer(String)}. The only
- * case need synchronized is {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and
+ * case need synchronized is {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} and
  * {@link #preLogRoll(Path)}.</li>
  * <li>No need synchronized on {@link #walsByIdRecoveredQueues}. There are three methods which
  * modify it, {@link #removePeer(String)} ,
- * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and
+ * {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} and
  * {@link ReplicationSourceManager.NodeFailoverWorker#run()}.
- * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} is called by
+ * {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} is called by
  * {@link ReplicationSourceInterface}. {@link #removePeer(String)} will terminate the
  * {@link ReplicationSourceInterface} firstly, then remove the wals from
  * {@link #walsByIdRecoveredQueues}. And {@link ReplicationSourceManager.NodeFailoverWorker#run()}
@@ -154,9 +156,15 @@ public class ReplicationSourceManager implements ReplicationListener {
 
   private final boolean replicationForBulkLoadDataEnabled;
 
-
   private AtomicLong totalBufferUsed = new AtomicLong();
 
+  // How long should we sleep for each retry when deleting remote wal files for sync replication
+  // peer.
+  private final long sleepForRetries;
+  // Maximum number of retries before taking bold actions when deleting remote wal files for sync
+  // replication peer.
+  private final int maxRetriesMultiplier;
+
   /**
    * Creates a replication manager and sets the watch on all the other registered region servers
    * @param queueStorage the interface for manipulating replication queues
@@ -204,8 +212,11 @@ public class ReplicationSourceManager implements ReplicationListener {
     tfb.setDaemon(true);
     this.executor.setThreadFactory(tfb.build());
     this.latestPaths = new HashSet<Path>();
-    replicationForBulkLoadDataEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
-      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
+    this.replicationForBulkLoadDataEnabled = conf.getBoolean(
+      HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
+    this.sleepForRetries = this.conf.getLong("replication.source.sync.sleepforretries", 1000);
+    this.maxRetriesMultiplier =
+      this.conf.getInt("replication.source.sync.maxretriesmultiplier", 60);
   }
 
   /**
@@ -494,16 +505,15 @@ public class ReplicationSourceManager implements ReplicationListener {
   /**
    * This method will log the current position to storage. And also clean old logs from the
    * replication queue.
-   * @param queueId id of the replication queue
-   * @param queueRecovered indicates if this queue comes from another region server
+   * @param source the replication source
    * @param entryBatch the wal entry batch we just shipped
    */
-  public void logPositionAndCleanOldLogs(String queueId, boolean queueRecovered,
+  public void logPositionAndCleanOldLogs(ReplicationSourceInterface source,
       WALEntryBatch entryBatch) {
     String fileName = entryBatch.getLastWalPath().getName();
-    abortWhenFail(() -> this.queueStorage.setWALPosition(server.getServerName(), queueId, fileName,
-      entryBatch.getLastWalPosition(), entryBatch.getLastSeqIds()));
-    cleanOldLogs(fileName, entryBatch.isEndOfFile(), queueId, queueRecovered);
+    abortWhenFail(() -> this.queueStorage.setWALPosition(server.getServerName(),
+      source.getQueueId(), fileName, entryBatch.getLastWalPosition(), entryBatch.getLastSeqIds()));
+    cleanOldLogs(fileName, entryBatch.isEndOfFile(), source);
   }
 
   /**
@@ -511,36 +521,84 @@ public class ReplicationSourceManager implements ReplicationListener {
    * file is closed and has no more entries.
    * @param log Path to the log
    * @param inclusive whether we should also remove the given log file
-   * @param queueId id of the replication queue
-   * @param queueRecovered Whether this is a recovered queue
+   * @param source the replication source
    */
   @VisibleForTesting
-  void cleanOldLogs(String log, boolean inclusive, String queueId, boolean queueRecovered) {
+  void cleanOldLogs(String log, boolean inclusive, ReplicationSourceInterface source) {
     String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log);
-    if (queueRecovered) {
-      NavigableSet<String> wals = walsByIdRecoveredQueues.get(queueId).get(logPrefix);
+    if (source.isRecovered()) {
+      NavigableSet<String> wals = walsByIdRecoveredQueues.get(source.getQueueId()).get(logPrefix);
       if (wals != null) {
-        cleanOldLogs(wals, log, inclusive, queueId);
+        cleanOldLogs(wals, log, inclusive, source);
       }
     } else {
       // synchronized on walsById to avoid race with preLogRoll
       synchronized (this.walsById) {
-        NavigableSet<String> wals = walsById.get(queueId).get(logPrefix);
+        NavigableSet<String> wals = walsById.get(source.getQueueId()).get(logPrefix);
         if (wals != null) {
-          cleanOldLogs(wals, log, inclusive, queueId);
+          cleanOldLogs(wals, log, inclusive, source);
+        }
+      }
+    }
+  }
+
+  private void removeRemoteWALs(String peerId, String remoteWALDir, Set<String> wals)
+      throws IOException {
+    Path remoteWALDirForPeer = ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId);
+    FileSystem fs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
+    for (String wal : wals) {
+      Path walFile = new Path(remoteWALDirForPeer, wal);
+      try {
+        if (!fs.delete(walFile, false) && fs.exists(walFile)) {
+          throw new IOException("Can not delete " + walFile);
         }
+      } catch (FileNotFoundException e) {
+        // Just ignore since this means the file has already been deleted.
+        // The javadoc of the FileSystem.delete methods does not specify the behavior of deleting an
+        // inexistent file, so here we deal with both, i.e, check the return value of the
+        // FileSystem.delete, and also catch FNFE.
+        LOG.debug("The remote wal {} has already been deleted?", walFile, e);
       }
     }
   }
 
-  private void cleanOldLogs(NavigableSet<String> wals, String key, boolean inclusive, String id) {
+  private void cleanOldLogs(NavigableSet<String> wals, String key, boolean inclusive,
+      ReplicationSourceInterface source) {
     NavigableSet<String> walSet = wals.headSet(key, inclusive);
     if (walSet.isEmpty()) {
       return;
     }
     LOG.debug("Removing {} logs in the list: {}", walSet.size(), walSet);
+    // The intention here is that, we want to delete the remote wal files ASAP as it may effect the
+    // failover time if you want to transit the remote cluster from S to A. And the infinite retry
+    // is not a problem, as if we can not contact with the remote HDFS cluster, then usually we can
+    // not contact with the HBase cluster either, so the replication will be blocked either.
+    if (source.isSyncReplication()) {
+      String peerId = source.getPeerId();
+      String remoteWALDir = source.getPeer().getPeerConfig().getRemoteWALDir();
+      LOG.debug("Removing {} logs from remote dir {} in the list: {}", walSet.size(), remoteWALDir,
+        walSet);
+      for (int sleepMultiplier = 0;;) {
+        try {
+          removeRemoteWALs(peerId, remoteWALDir, walSet);
+          break;
+        } catch (IOException e) {
+          LOG.warn("Failed to delete remote wals from remote dir {} for peer {}", remoteWALDir,
+            peerId);
+        }
+        if (!source.isSourceActive()) {
+          // skip the following operations
+          return;
+        }
+        if (ReplicationUtils.sleepForRetries("Failed to delete remote wals", sleepForRetries,
+          sleepMultiplier, maxRetriesMultiplier)) {
+          sleepMultiplier++;
+        }
+      }
+    }
+    String queueId = source.getQueueId();
     for (String wal : walSet) {
-      abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), id, wal));
+      abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), queueId, wal));
     }
     walSet.clear();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d1f4b418/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
index 11fd660..3f97b5e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
+import static org.apache.hadoop.hbase.replication.ReplicationUtils.sleepForRetries;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.PriorityBlockingQueue;
@@ -91,7 +93,7 @@ public class ReplicationSourceShipper extends Thread {
       if (!source.isPeerEnabled()) {
         // The peer enabled check is in memory, not expensive, so do not need to increase the
         // sleep interval as it may cause a long lag when we enable the peer.
-        sleepForRetries("Replication is disabled", 1);
+        sleepForRetries("Replication is disabled", sleepForRetries, 1, maxRetriesMultiplier);
         continue;
       }
       try {
@@ -189,7 +191,8 @@ public class ReplicationSourceShipper extends Thread {
       } catch (Exception ex) {
         LOG.warn("{} threw unknown exception:",
           source.getReplicationEndpoint().getClass().getName(), ex);
-        if (sleepForRetries("ReplicationEndpoint threw exception", sleepMultiplier)) {
+        if (sleepForRetries("ReplicationEndpoint threw exception", sleepForRetries, sleepMultiplier,
+          maxRetriesMultiplier)) {
           sleepMultiplier++;
         }
       }
@@ -228,8 +231,7 @@ public class ReplicationSourceShipper extends Thread {
     // position and the file will be removed soon in cleanOldLogs.
     if (batch.isEndOfFile() || !batch.getLastWalPath().equals(currentPath) ||
       batch.getLastWalPosition() != currentPosition) {
-      source.getSourceManager().logPositionAndCleanOldLogs(source.getQueueId(),
-        source.isRecovered(), batch);
+      source.getSourceManager().logPositionAndCleanOldLogs(source, batch);
       updated = true;
     }
     // if end of file is true, then we can just skip to the next file in queue.
@@ -282,21 +284,4 @@ public class ReplicationSourceShipper extends Thread {
   public boolean isFinished() {
     return state == WorkerState.FINISHED;
   }
-
-  /**
-   * Do the sleeping logic
-   * @param msg Why we sleep
-   * @param sleepMultiplier by how many times the default sleeping time is augmented
-   * @return True if <code>sleepMultiplier</code> is &lt; <code>maxRetriesMultiplier</code>
-   */
-  public boolean sleepForRetries(String msg, int sleepMultiplier) {
-    try {
-      LOG.trace("{}, sleeping {} times {}", msg, sleepForRetries, sleepMultiplier);
-      Thread.sleep(this.sleepForRetries * sleepMultiplier);
-    } catch (InterruptedException e) {
-      LOG.debug("Interrupted while sleeping between retries");
-      Thread.currentThread().interrupt();
-    }
-    return sleepMultiplier < maxRetriesMultiplier;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d1f4b418/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
index 9cbb095..3cd356d42 100644
--- 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
@@ -33,11 +33,10 @@ import java.util.function.BiPredicate;
 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.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
 import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider;
@@ -118,10 +117,10 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
   }
 
   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),
+    return new DualAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf),
+      ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir),
+      CommonFSUtils.getWALRootDir(conf),
+      ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId),
       getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
       conf, listeners, true, getLogPrefix(peerId), LOG_SUFFIX, eventLoopGroup, channelClass);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d1f4b418/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index ec6ec96..67f793d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -40,12 +39,13 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
  */
 public class ReplicationSourceDummy implements ReplicationSourceInterface {
 
-  ReplicationSourceManager manager;
-  String peerClusterId;
-  Path currentPath;
-  MetricsSource metrics;
-  WALFileLengthProvider walFileLengthProvider;
-  AtomicBoolean startup = new AtomicBoolean(false);
+  private ReplicationSourceManager manager;
+  private ReplicationPeer replicationPeer;
+  private String peerClusterId;
+  private Path currentPath;
+  private MetricsSource metrics;
+  private WALFileLengthProvider walFileLengthProvider;
+  private AtomicBoolean startup = new AtomicBoolean(false);
 
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
@@ -56,6 +56,7 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
     this.peerClusterId = peerClusterId;
     this.metrics = metrics;
     this.walFileLengthProvider = walFileLengthProvider;
+    this.replicationPeer = rp;
   }
 
   @Override
@@ -153,4 +154,9 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   public ServerName getServerWALsBelongTo() {
     return null;
   }
+
+  @Override
+  public ReplicationPeer getPeer() {
+    return replicationPeer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d1f4b418/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 482f49a..5ea3173 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
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -49,19 +51,19 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 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.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
@@ -71,6 +73,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.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
@@ -133,9 +136,9 @@ public abstract class TestReplicationSourceManager {
 
   protected static ZKWatcher zkw;
 
-  protected static HTableDescriptor htd;
+  protected static TableDescriptor htd;
 
-  protected static HRegionInfo hri;
+  protected static RegionInfo hri;
 
   protected static final byte[] r1 = Bytes.toBytes("r1");
 
@@ -156,6 +159,8 @@ public abstract class TestReplicationSourceManager {
 
   protected static Path logDir;
 
+  protected static Path remoteLogDir;
+
   protected static CountDownLatch latch;
 
   protected static List<String> files = new ArrayList<>();
@@ -185,10 +190,9 @@ public abstract class TestReplicationSourceManager {
     ZKClusterId.setClusterId(zkw, new ClusterId());
     FSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir());
     fs = FileSystem.get(conf);
-    oldLogDir = new Path(utility.getDataTestDir(),
-        HConstants.HREGION_OLDLOGDIR_NAME);
-    logDir = new Path(utility.getDataTestDir(),
-        HConstants.HREGION_LOGDIR_NAME);
+    oldLogDir = utility.getDataTestDir(HConstants.HREGION_OLDLOGDIR_NAME);
+    logDir = utility.getDataTestDir(HConstants.HREGION_LOGDIR_NAME);
+    remoteLogDir = utility.getDataTestDir(ReplicationUtils.REMOTE_WAL_DIR_NAME);
     replication = new Replication();
     replication.initialize(new DummyServer(), fs, logDir, oldLogDir, null);
     managerOfCluster = getManagerFromCluster();
@@ -205,19 +209,16 @@ public abstract class TestReplicationSourceManager {
     }
     waitPeer(slaveId, manager, true);
 
-    htd = new HTableDescriptor(test);
-    HColumnDescriptor col = new HColumnDescriptor(f1);
-    col.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
-    htd.addFamily(col);
-    col = new HColumnDescriptor(f2);
-    col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
-    htd.addFamily(col);
+    htd = TableDescriptorBuilder.newBuilder(test)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(f1)
+        .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(f2)).build();
 
     scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
+    for(byte[] fam : htd.getColumnFamilyNames()) {
       scopes.put(fam, 0);
     }
-    hri = new HRegionInfo(htd.getTableName(), r1, r2);
+    hri = RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(r1).setEndKey(r2).build();
   }
 
   private static ReplicationSourceManager getManagerFromCluster() {
@@ -248,6 +249,7 @@ public abstract class TestReplicationSourceManager {
   private void cleanLogDir() throws IOException {
     fs.delete(logDir, true);
     fs.delete(oldLogDir, true);
+    fs.delete(remoteLogDir, true);
   }
 
   @Before
@@ -286,10 +288,10 @@ public abstract class TestReplicationSourceManager {
       .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager));
     final WAL wal = wals.getWAL(hri);
     manager.init();
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame"));
-    htd.addFamily(new HColumnDescriptor(f1));
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf("tableame"))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(f1)).build();
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
+    for(byte[] fam : htd.getColumnFamilyNames()) {
       scopes.put(fam, 0);
     }
     // Testing normal log rolling every 20
@@ -329,7 +331,11 @@ public abstract class TestReplicationSourceManager {
 
     wal.rollWriter();
 
-    manager.logPositionAndCleanOldLogs("1", false,
+    ReplicationSourceInterface source = mock(ReplicationSourceInterface.class);
+    when(source.getQueueId()).thenReturn("1");
+    when(source.isRecovered()).thenReturn(false);
+    when(source.isSyncReplication()).thenReturn(false);
+    manager.logPositionAndCleanOldLogs(source,
       new WALEntryBatch(0, manager.getSources().get(0).getCurrentPath()));
 
     wal.append(hri,
@@ -404,7 +410,11 @@ public abstract class TestReplicationSourceManager {
     assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
     String id = "1-" + server.getServerName().getServerName();
     assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id).get(group));
-    manager.cleanOldLogs(file2, false, id, true);
+    ReplicationSourceInterface source = mock(ReplicationSourceInterface.class);
+    when(source.getQueueId()).thenReturn(id);
+    when(source.isRecovered()).thenReturn(true);
+    when(source.isSyncReplication()).thenReturn(false);
+    manager.cleanOldLogs(file2, false, source);
     // log1 should be deleted
     assertEquals(Sets.newHashSet(file2), manager.getWalsByIdRecoveredQueues().get(id).get(group));
   }
@@ -488,14 +498,13 @@ public abstract class TestReplicationSourceManager {
    * corresponding ReplicationSourceInterface correctly cleans up the corresponding
    * replication queue and ReplicationPeer.
    * See HBASE-16096.
-   * @throws Exception
    */
   @Test
   public void testPeerRemovalCleanup() throws Exception{
     String replicationSourceImplName = conf.get("replication.replicationsource.implementation");
     final String peerId = "FakePeer";
-    final ReplicationPeerConfig peerConfig = new ReplicationPeerConfig()
-        .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase");
+    final ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
+      .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase").build();
     try {
       DummyServer server = new DummyServer();
       ReplicationQueueStorage rq = ReplicationStorageFactory
@@ -504,7 +513,7 @@ public abstract class TestReplicationSourceManager {
       // initialization to throw an exception.
       conf.set("replication.replicationsource.implementation",
           FailInitializeDummyReplicationSource.class.getName());
-      final ReplicationPeers rp = manager.getReplicationPeers();
+      manager.getReplicationPeers();
       // Set up the znode and ReplicationPeer for the fake peer
       // Don't wait for replication source to initialize, we know it won't.
       addPeerAndWait(peerId, peerConfig, false);
@@ -549,8 +558,8 @@ public abstract class TestReplicationSourceManager {
   @Test
   public void testRemovePeerMetricsCleanup() throws Exception {
     final String peerId = "DummyPeer";
-    final ReplicationPeerConfig peerConfig = new ReplicationPeerConfig()
-        .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase");
+    final ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
+      .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase").build();
     try {
       MetricsReplicationSourceSource globalSource = getGlobalSource();
       final int globalLogQueueSizeInitial = globalSource.getSizeOfLogQueue();
@@ -582,6 +591,40 @@ public abstract class TestReplicationSourceManager {
     }
   }
 
+  @Test
+  public void testRemoveRemoteWALs() throws IOException {
+    // make sure that we can deal with files which does not exist
+    String walNameNotExists = "remoteWAL.0";
+    Path wal = new Path(logDir, walNameNotExists);
+    manager.preLogRoll(wal);
+    manager.postLogRoll(wal);
+
+    Path remoteLogDirForPeer = new Path(remoteLogDir, slaveId);
+    fs.mkdirs(remoteLogDirForPeer);
+    String walName = "remoteWAL.1";
+    Path remoteWAL =
+      new Path(remoteLogDirForPeer, walName).makeQualified(fs.getUri(), fs.getWorkingDirectory());
+    fs.create(remoteWAL).close();
+    wal = new Path(logDir, walName);
+    manager.preLogRoll(wal);
+    manager.postLogRoll(wal);
+
+    ReplicationSourceInterface source = mock(ReplicationSourceInterface.class);
+    when(source.getPeerId()).thenReturn(slaveId);
+    when(source.getQueueId()).thenReturn(slaveId);
+    when(source.isRecovered()).thenReturn(false);
+    when(source.isSyncReplication()).thenReturn(true);
+    ReplicationPeerConfig config = mock(ReplicationPeerConfig.class);
+    when(config.getRemoteWALDir())
+      .thenReturn(remoteLogDir.makeQualified(fs.getUri(), fs.getWorkingDirectory()).toString());
+    ReplicationPeer peer = mock(ReplicationPeer.class);
+    when(peer.getPeerConfig()).thenReturn(config);
+    when(source.getPeer()).thenReturn(peer);
+    manager.cleanOldLogs(walName, true, source);
+
+    assertFalse(fs.exists(remoteWAL));
+  }
+
   /**
    * Add a peer and wait for it to initialize
    * @param peerId


[17/36] hbase git commit: HBASE-20576 Check remote WAL directory when creating peer and transiting peer to A

Posted by zh...@apache.org.
HBASE-20576 Check remote WAL directory when creating peer and transiting peer to A


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

Branch: refs/heads/HBASE-19064
Commit: 25cac93a7499877a2cc2e2e8ebbf23c27561e3dc
Parents: 85f2294
Author: zhangduo <zh...@apache.org>
Authored: Tue May 15 15:07:40 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerManager.java     | 19 +++--
 ...ransitPeerSyncReplicationStateProcedure.java | 73 +++++++++++++-------
 .../replication/TestReplicationAdmin.java       | 57 ++++++++++++---
 3 files changed, 110 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/25cac93a/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 e1d8b51..8e49137 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
@@ -31,6 +32,7 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -45,7 +47,6 @@ 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.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -193,9 +194,9 @@ public class ReplicationPeerManager {
   }
 
   /**
-   * @return the old state, and whether the peer is enabled.
+   * @return the old desciption of the peer
    */
-  Pair<SyncReplicationState, Boolean> preTransitPeerSyncReplicationState(String peerId,
+  ReplicationPeerDescription preTransitPeerSyncReplicationState(String peerId,
       SyncReplicationState state) throws DoNotRetryIOException {
     ReplicationPeerDescription desc = checkPeerExists(peerId);
     SyncReplicationState fromState = desc.getSyncReplicationState();
@@ -204,7 +205,7 @@ public class ReplicationPeerManager {
       throw new DoNotRetryIOException("Can not transit current cluster state from " + fromState +
         " to " + state + " for peer id=" + peerId);
     }
-    return Pair.newPair(fromState, desc.isEnabled());
+    return desc;
   }
 
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
@@ -384,6 +385,16 @@ public class ReplicationPeerManager {
           "Only support replicated table config for sync replication peer");
       }
     }
+    Path remoteWALDir = new Path(peerConfig.getRemoteWALDir());
+    if (!remoteWALDir.isAbsolute()) {
+      throw new DoNotRetryIOException(
+        "The remote WAL directory " + peerConfig.getRemoteWALDir() + " is not absolute");
+    }
+    URI remoteWALDirUri = remoteWALDir.toUri();
+    if (remoteWALDirUri.getScheme() == null || remoteWALDirUri.getAuthority() == null) {
+      throw new DoNotRetryIOException("The remote WAL directory " + peerConfig.getRemoteWALDir() +
+        " is not qualified, you must provide scheme and authority");
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/25cac93a/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 0175296..ebe7a93 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
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.stream.Collectors;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -31,9 +32,9 @@ 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.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -113,10 +114,20 @@ public class TransitPeerSyncReplicationStateProcedure
     if (cpHost != null) {
       cpHost.preTransitReplicationPeerSyncReplicationState(peerId, toState);
     }
-    Pair<SyncReplicationState, Boolean> pair =
+    ReplicationPeerDescription desc =
       env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, toState);
-    fromState = pair.getFirst();
-    enabled = pair.getSecond();
+    if (toState == SyncReplicationState.ACTIVE) {
+      Path remoteWALDirForPeer =
+        ReplicationUtils.getRemoteWALDirForPeer(desc.getPeerConfig().getRemoteWALDir(), peerId);
+      // check whether the remote wal directory is present
+      if (!remoteWALDirForPeer.getFileSystem(env.getMasterConfiguration())
+        .exists(remoteWALDirForPeer)) {
+        throw new DoNotRetryIOException(
+          "The remote WAL directory " + remoteWALDirForPeer + " does not exist");
+      }
+    }
+    fromState = desc.getSyncReplicationState();
+    enabled = desc.isEnabled();
   }
 
   private void postTransit(MasterProcedureEnv env) throws IOException {
@@ -152,6 +163,36 @@ public class TransitPeerSyncReplicationStateProcedure
     }
   }
 
+  private void setNextStateAfterRefreshBegin() {
+    if (fromState.equals(SyncReplicationState.ACTIVE)) {
+      setNextState(toState.equals(SyncReplicationState.STANDBY)
+        ? PeerSyncReplicationStateTransitionState.REMOVE_ALL_REPLICATION_QUEUES_IN_PEER
+        : PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
+    } else if (fromState.equals(SyncReplicationState.DOWNGRADE_ACTIVE)) {
+      setNextState(toState.equals(SyncReplicationState.STANDBY)
+        ? PeerSyncReplicationStateTransitionState.REMOVE_ALL_REPLICATION_QUEUES_IN_PEER
+        : PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
+    } else {
+      assert toState.equals(SyncReplicationState.DOWNGRADE_ACTIVE);
+      setNextState(PeerSyncReplicationStateTransitionState.REPLAY_REMOTE_WAL_IN_PEER);
+    }
+  }
+
+  private void setNextStateAfterRefreshEnd() {
+    if (toState == SyncReplicationState.STANDBY) {
+      setNextState(
+        enabled ? PeerSyncReplicationStateTransitionState.SYNC_REPLICATION_SET_PEER_ENABLED
+          : PeerSyncReplicationStateTransitionState.CREATE_DIR_FOR_REMOTE_WAL);
+    } else {
+      setNextState(
+        PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION);
+    }
+  }
+
+  private void replayRemoteWAL() {
+    addChildProcedure(new RecoverStandbyProcedure[] { new RecoverStandbyProcedure(peerId) });
+  }
+
   @Override
   protected Flow executeFromState(MasterProcedureEnv env,
       PeerSyncReplicationStateTransitionState state)
@@ -191,21 +232,10 @@ public class TransitPeerSyncReplicationStateProcedure
         addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
           .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 0))
           .toArray(RefreshPeerProcedure[]::new));
-        if (fromState.equals(SyncReplicationState.ACTIVE)) {
-          setNextState(toState.equals(SyncReplicationState.STANDBY)
-            ? PeerSyncReplicationStateTransitionState.REMOVE_ALL_REPLICATION_QUEUES_IN_PEER
-            : PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
-        } else if (fromState.equals(SyncReplicationState.DOWNGRADE_ACTIVE)) {
-          setNextState(toState.equals(SyncReplicationState.STANDBY)
-            ? PeerSyncReplicationStateTransitionState.REMOVE_ALL_REPLICATION_QUEUES_IN_PEER
-            : PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
-        } else {
-          assert toState.equals(SyncReplicationState.DOWNGRADE_ACTIVE);
-          setNextState(PeerSyncReplicationStateTransitionState.REPLAY_REMOTE_WAL_IN_PEER);
-        }
+        setNextStateAfterRefreshBegin();
         return Flow.HAS_MORE_STATE;
       case REPLAY_REMOTE_WAL_IN_PEER:
-        addChildProcedure(new RecoverStandbyProcedure(peerId));
+        replayRemoteWAL();
         setNextState(
           PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE);
         return Flow.HAS_MORE_STATE;
@@ -248,14 +278,7 @@ public class TransitPeerSyncReplicationStateProcedure
         addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
           .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 1))
           .toArray(RefreshPeerProcedure[]::new));
-        if (toState == SyncReplicationState.STANDBY) {
-          setNextState(
-            enabled ? PeerSyncReplicationStateTransitionState.SYNC_REPLICATION_SET_PEER_ENABLED
-              : PeerSyncReplicationStateTransitionState.CREATE_DIR_FOR_REMOTE_WAL);
-        } else {
-          setNextState(
-            PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION);
-        }
+        setNextStateAfterRefreshEnd();
         return Flow.HAS_MORE_STATE;
       case SYNC_REPLICATION_SET_PEER_ENABLED:
         try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/25cac93a/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 ac98283..c6ffeea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.replication.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.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterReplicationEndpointForTest;
 import org.apache.hadoop.hbase.replication.regionserver.TestReplicator.ReplicationEndpointForTest;
@@ -981,34 +982,37 @@ public class TestReplicationAdmin {
     ReplicationPeerConfig rpc = hbaseAdmin.getReplicationPeerConfig(ID_ONE);
     assertNull(rpc.getRemoteWALDir());
 
+    builder.setRemoteWALDir("hdfs://srv2:8888/hbase");
     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
+      LOG.info("Expected error:", e);
     }
 
     builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_SECOND);
-    builder.setRemoteWALDir(rootDir);
+    builder.setRemoteWALDir("whatever");
 
     try {
       hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
       fail("Only support replicated table config for sync replication");
     } catch (Exception e) {
       // OK
+      LOG.info("Expected error:", e);
     }
 
     builder.setReplicateAllUserTables(false);
+    Set<String> namespaces = new HashSet<String>();
+    namespaces.add("ns1");
+    builder.setNamespaces(namespaces);
     try {
-      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
+      LOG.info("Expected error:", e);
     }
 
     builder.setNamespaces(null);
@@ -1017,21 +1021,41 @@ public class TestReplicationAdmin {
       fail("Only support replicated table config for sync replication, and tables can't be empty");
     } catch (Exception e) {
       // OK
+      LOG.info("Expected error:", e);
     }
 
     Map<TableName, List<String>> tableCfs = new HashMap<>();
+    tableCfs.put(tableName, Arrays.asList("cf1"));
+    builder.setTableCFsMap(tableCfs);
     try {
-      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
+      LOG.info("Expected error:", e);
     }
 
     tableCfs = new HashMap<>();
     tableCfs.put(tableName, new ArrayList<>());
     builder.setTableCFsMap(tableCfs);
+    try {
+      hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+      fail("The remote WAL dir must be absolute");
+    } catch (Exception e) {
+      // OK
+      LOG.info("Expected error:", e);
+    }
+
+    builder.setRemoteWALDir("/hbase/remoteWALs");
+    try {
+      hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+      fail("The remote WAL dir must be qualified");
+    } catch (Exception e) {
+      // OK
+      LOG.info("Expected error:", e);
+    }
+
+    builder.setRemoteWALDir(rootDir);
     hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
     rpc = hbaseAdmin.getReplicationPeerConfig(ID_SECOND);
     assertEquals(rootDir, rpc.getRemoteWALDir());
@@ -1042,6 +1066,7 @@ public class TestReplicationAdmin {
       fail("Change remote wal dir is not allowed");
     } catch (Exception e) {
       // OK
+      LOG.info("Expected error:", e);
     }
 
     try {
@@ -1050,6 +1075,7 @@ public class TestReplicationAdmin {
       fail("Change remote wal dir is not allowed");
     } catch (Exception e) {
       // OK
+      LOG.info("Expected error:", e);
     }
 
     try {
@@ -1062,6 +1088,7 @@ public class TestReplicationAdmin {
         "Change replicated table config on an existing synchronous peer is not allowed");
     } catch (Exception e) {
       // OK
+      LOG.info("Expected error:", e);
     }
   }
 
@@ -1079,13 +1106,13 @@ public class TestReplicationAdmin {
     try {
       hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE,
         SyncReplicationState.DOWNGRADE_ACTIVE);
-      fail("Can't transit cluster state if replication peer don't config remote wal dir");
+      fail("Can't transit sync replication state if replication peer don't config remote wal dir");
     } catch (Exception e) {
       // OK
+      LOG.info("Expected error:", e);
     }
 
     Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("remoteWAL");
-    TEST_UTIL.getTestFileSystem().mkdirs(new Path(rootDir, ID_SECOND));
     builder = ReplicationPeerConfig.newBuilder();
     builder.setClusterKey(KEY_SECOND);
     builder.setRemoteWALDir(rootDir.makeQualified(TEST_UTIL.getTestFileSystem().getUri(),
@@ -1106,6 +1133,15 @@ public class TestReplicationAdmin {
     assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
       hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
 
+    try {
+      hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
+      fail("Can't transit sync replication state to ACTIVE if remote wal dir does not exist");
+    } catch (Exception e) {
+      // OK
+      LOG.info("Expected error:", e);
+    }
+    TEST_UTIL.getTestFileSystem()
+      .mkdirs(ReplicationUtils.getRemoteWALDirForPeer(rootDir, ID_SECOND));
     hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
     assertEquals(SyncReplicationState.ACTIVE,
       hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
@@ -1133,9 +1169,10 @@ public class TestReplicationAdmin {
       hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
     try {
       hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
-      fail("Can't transit cluster state from STANDBY to ACTIVE");
+      fail("Can't transit sync replication state from STANDBY to ACTIVE");
     } catch (Exception e) {
       // OK
+      LOG.info("Expected error:", e);
     }
     hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND,
       SyncReplicationState.DOWNGRADE_ACTIVE);


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

Branch: refs/heads/HBASE-19064
Commit: 5f4bc7718de8b0347158c2789e69fe282175e94e
Parents: e62f8a0
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 26 16:50:48 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 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   | 24 +++++++++-----------
 .../hadoop/hbase/master/MasterRpcServices.java  |  9 ++++----
 ...ransitPeerSyncReplicationStateProcedure.java |  9 ++++----
 .../TestReplicationSourceManager.java           |  2 +-
 8 files changed, 67 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5f4bc771/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 6cbe05b..331795c 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
@@ -403,7 +403,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
@@ -411,17 +411,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/5f4bc771/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/5f4bc771/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 1269cc7..db07bab 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
@@ -1871,10 +1871,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/5f4bc771/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 3564ae4..61ba131 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -66,11 +66,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/5f4bc771/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 ad3c435..9107cf6 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,13 +17,13 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
 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.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -97,7 +97,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);
@@ -181,29 +181,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/5f4bc771/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 5de5681..b502c16 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
@@ -85,7 +85,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;
@@ -1947,13 +1946,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/5f4bc771/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/5f4bc771/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 04c7aad..b058da3 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
@@ -175,7 +175,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);
 


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

Branch: refs/heads/HBASE-19064
Commit: 30efc3350df027d369094b93ad2fd65063e902f4
Parents: 40a73a5
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 11 21:08:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  21 +--
 .../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, 533 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/30efc335/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 d032d83..4732f41 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
@@ -607,10 +607,14 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
+  protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException {
+    return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, this.blocksize,
+      eventLoopGroup, channelClass);
+  }
+
   @Override
   protected AsyncWriter createWriterInstance(Path path) throws IOException {
-    return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false,
-        this.blocksize, eventLoopGroup, channelClass);
+    return createAsyncWriter(fs, path);
   }
 
   private void waitForSafePoint() {
@@ -632,13 +636,12 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  private long closeWriter() {
-    AsyncWriter oldWriter = this.writer;
-    if (oldWriter != null) {
-      long fileLength = oldWriter.getLength();
+  protected final long closeWriter(AsyncWriter writer) {
+    if (writer != null) {
+      long fileLength = writer.getLength();
       closeExecutor.execute(() -> {
         try {
-          oldWriter.close();
+          writer.close();
         } catch (IOException e) {
           LOG.warn("close old writer failed", e);
         }
@@ -654,7 +657,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
       throws IOException {
     Preconditions.checkNotNull(nextWriter);
     waitForSafePoint();
-    long oldFileLen = closeWriter();
+    long oldFileLen = closeWriter(this.writer);
     logRollAndSetupWalProps(oldPath, newPath, oldFileLen);
     this.writer = nextWriter;
     if (nextWriter instanceof AsyncProtobufLogWriter) {
@@ -679,7 +682,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   @Override
   protected void doShutdown() throws IOException {
     waitForSafePoint();
-    closeWriter();
+    closeWriter(this.writer);
     closeExecutor.shutdown();
     try {
       if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/30efc335/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/30efc335/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..42b0dae
--- /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) {
+        closeWriter(localWriter);
+      }
+    }
+    return CombinedAsyncWriter.create(CombinedAsyncWriter.Mode.SEQUENTIAL, remoteWriter,
+      localWriter);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/30efc335/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/30efc335/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..aece961
--- /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(false);
+    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/30efc335/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/30efc335/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..cb8edc6
--- /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(), 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/30efc335/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 2d938d4..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(false);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/30efc335/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..9d938b0
--- /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 append(Entry entry) throws IOException {
+    asyncWriter.append(entry);
+  }
+
+  @Override
+  public void sync(boolean forceSync) 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());
+    }
+  }
+}


[04/36] hbase git commit: Revert "TestWALProcedureStoreOnHDFS failing on branch-1"

Posted by zh...@apache.org.
Revert "TestWALProcedureStoreOnHDFS failing on branch-1"

This reverts commit dcfa01448c1693308256b41fe43aca6df41fd3b8.


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

Branch: refs/heads/HBASE-19064
Commit: 31ae8dc7f3a8087fd48a9a59ba113f803ea0ea8d
Parents: dcfa014
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed May 30 14:44:22 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed May 30 14:44:22 2018 -0700

----------------------------------------------------------------------
 .../hbase/master/procedure/TestWALProcedureStoreOnHDFS.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/31ae8dc7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
index 6f61916..99699ff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -91,8 +92,7 @@ public class TestWALProcedureStoreOnHDFS {
     store.recoverLease();
   }
 
-  // No @After
-  @SuppressWarnings("JUnit4TearDownNotRun")
+  @After
   public void tearDown() throws Exception {
     store.stop(false);
     UTIL.getDFSCluster().getFileSystem().delete(store.getWALDir(), true);


[05/36] hbase git commit: HBASE-20646 TestWALProcedureStoreOnHDFS failing on branch-1

Posted by zh...@apache.org.
HBASE-20646 TestWALProcedureStoreOnHDFS failing on branch-1


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

Branch: refs/heads/HBASE-19064
Commit: b889c8a2215258b11d1d9476d5ab12f472e18d67
Parents: 31ae8dc
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed May 30 14:44:30 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed May 30 14:44:30 2018 -0700

----------------------------------------------------------------------
 .../hbase/master/procedure/TestWALProcedureStoreOnHDFS.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b889c8a2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
index 99699ff..6f61916 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestWALProcedureStoreOnHDFS.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -92,7 +91,8 @@ public class TestWALProcedureStoreOnHDFS {
     store.recoverLease();
   }
 
-  @After
+  // No @After
+  @SuppressWarnings("JUnit4TearDownNotRun")
   public void tearDown() throws Exception {
     store.stop(false);
     UTIL.getDFSCluster().getFileSystem().delete(store.getWALDir(), true);


[08/36] hbase git commit: HBASE-19999 Remove the SYNC_REPLICATION_ENABLED flag

Posted by zh...@apache.org.
HBASE-19999 Remove the SYNC_REPLICATION_ENABLED flag


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

Branch: refs/heads/HBASE-19064
Commit: 679a4ac04a8a0a6756e533d2330fb27508a1bfa9
Parents: ca384b1
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Mar 9 11:30:25 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationUtils.java      |  2 --
 .../hadoop/hbase/regionserver/HRegionServer.java | 13 ++++---------
 .../hbase/wal/SyncReplicationWALProvider.java    | 19 ++++++++++++++++++-
 .../org/apache/hadoop/hbase/wal/WALFactory.java  | 18 ++++++++++++++++--
 .../hbase/replication/TestSyncReplication.java   |  1 -
 .../master/TestRecoverStandbyProcedure.java      |  2 --
 .../wal/TestSyncReplicationWALProvider.java      |  2 --
 7 files changed, 38 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/679a4ac0/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 e402d0f..cb22f57 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
@@ -37,8 +37,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public final class ReplicationUtils {
 
-  public static final String SYNC_REPLICATION_ENABLED = "hbase.replication.sync.enabled";
-
   public static final String REPLICATION_ATTR_NAME = "__rep__";
 
   public static final String REMOTE_WAL_DIR_NAME = "remoteWALs";

http://git-wip-us.apache.org/repos/asf/hbase/blob/679a4ac0/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 3aadb9c..51f9fd0 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
@@ -1803,10 +1803,8 @@ public class HRegionServer extends HasThread implements
   private void setupWALAndReplication() throws IOException {
     boolean isMasterNoTableOrSystemTableOnly = this instanceof HMaster &&
       (!LoadBalancer.isTablesOnMaster(conf) || LoadBalancer.isSystemTablesOnlyOnMaster(conf));
-    if (isMasterNoTableOrSystemTableOnly) {
-      conf.setBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, false);
-    }
-    WALFactory factory = new WALFactory(conf, serverName.toString());
+    WALFactory factory =
+        new WALFactory(conf, serverName.toString(), !isMasterNoTableOrSystemTableOnly);
     if (!isMasterNoTableOrSystemTableOnly) {
       // TODO Replication make assumptions here based on the default filesystem impl
       Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
@@ -1925,11 +1923,8 @@ public class HRegionServer extends HasThread implements
     }
     this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER,
       conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2));
-
-    if (conf.getBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, false)) {
-      this.executorService.startExecutorService(ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL,
-        conf.getInt("hbase.regionserver.executor.replay.sync.replication.wal.threads", 2));
-    }
+    this.executorService.startExecutorService(ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL,
+      conf.getInt("hbase.regionserver.executor.replay.sync.replication.wal.threads", 1));
 
     Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
     uncaughtExceptionHandler);

http://git-wip-us.apache.org/repos/asf/hbase/blob/679a4ac0/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
index 282aa21..54287fe 100644
--- 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
@@ -29,6 +29,7 @@ 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.function.BiPredicate;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
@@ -67,7 +68,8 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
 
   private final WALProvider provider;
 
-  private SyncReplicationPeerInfoProvider peerInfoProvider;
+  private SyncReplicationPeerInfoProvider peerInfoProvider =
+      new DefaultSyncReplicationPeerInfoProvider();
 
   private WALFactory factory;
 
@@ -235,4 +237,19 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
       safeClose(peerId2WAL.remove(peerId));
     }
   }
+
+  private static class DefaultSyncReplicationPeerInfoProvider
+      implements SyncReplicationPeerInfoProvider {
+
+    @Override
+    public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
+      return Optional.empty();
+    }
+
+    @Override
+    public boolean checkState(RegionInfo info,
+        BiPredicate<SyncReplicationState, SyncReplicationState> checker) {
+      return false;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/679a4ac0/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 2e43eb4..afe043f 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,6 @@ 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.ReplicationUtils;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -151,6 +150,21 @@ public class WALFactory {
    *          to make a directory
    */
   public WALFactory(Configuration conf, String factoryId) throws IOException {
+    // default enableSyncReplicationWALProvider is true, only disable SyncReplicationWALProvider
+    // for HMaster or HRegionServer which take system table only. See HBASE-19999
+    this(conf, factoryId, true);
+  }
+
+  /**
+   * @param conf must not be null, will keep a reference to read params in later reader/writer
+   *          instances.
+   * @param factoryId a unique identifier for this factory. used i.e. by filesystem implementations
+   *          to make a directory
+   * @param enableSyncReplicationWALProvider whether wrap the wal provider to a
+   *          {@link SyncReplicationWALProvider}
+   */
+  public WALFactory(Configuration conf, String factoryId, boolean enableSyncReplicationWALProvider)
+      throws IOException {
     // until we've moved reader/writer construction down into providers, this initialization must
     // happen prior to provider initialization, in case they need to instantiate a reader/writer.
     timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000);
@@ -162,7 +176,7 @@ public class WALFactory {
     // end required early initialization
     if (conf.getBoolean("hbase.regionserver.hlog.enabled", true)) {
       WALProvider provider = createProvider(getProviderClass(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
-      if (conf.getBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, false)) {
+      if (enableSyncReplicationWALProvider) {
         provider = new SyncReplicationWALProvider(provider);
       }
       provider.init(this, conf, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/679a4ac0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
index cc84dab..288dcbf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
@@ -84,7 +84,6 @@ public class TestSyncReplication {
   private static void initTestingUtility(HBaseTestingUtility util, String zkParent) {
     util.setZkCluster(ZK_UTIL.getZkCluster());
     Configuration conf = util.getConfiguration();
-    conf.setBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, true);
     conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkParent);
     conf.setInt("replication.source.size.capacity", 102400);
     conf.setLong("replication.source.sleepforretries", 100);

http://git-wip-us.apache.org/repos/asf/hbase/blob/679a4ac0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
index f3d61bb..ebb21a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
-import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -102,7 +101,6 @@ public class TestRecoverStandbyProcedure {
 
   @BeforeClass
   public static void setupCluster() throws Exception {
-    UTIL.getConfiguration().setBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, true);
     UTIL.startMiniCluster(RS_NUMBER);
     UTIL.getHBaseCluster().waitForActiveAndReadyMaster();
     conf = UTIL.getConfiguration();

http://git-wip-us.apache.org/repos/asf/hbase/blob/679a4ac0/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 488d9fb..3263fe8 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
@@ -36,7 +36,6 @@ 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.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -94,7 +93,6 @@ public class TestSyncReplicationWALProvider {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    UTIL.getConfiguration().setBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, true);
     UTIL.startMiniDFSCluster(3);
     FACTORY = new WALFactory(UTIL.getConfiguration(), "test");
     ((SyncReplicationWALProvider) FACTORY.getWALProvider()).setPeerInfoProvider(new InfoProvider());


[15/36] 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/0aba887d/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/0aba887d/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 d1f1344..5f86365 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
@@ -338,6 +339,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/0aba887d/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 9a79658..934fa11 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -393,6 +393,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/0aba887d/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 f3ab749..f2ec014 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
                             SERIAL])
 
         peers.each do |peer|
@@ -67,7 +67,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, config.isSerial])
         end

http://git-wip-us.apache.org/repos/asf/hbase/blob/0aba887d/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/0aba887d/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 5d04fbb..9d364ce 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
@@ -513,6 +516,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 }


[22/36] hbase git commit: HBASE-19973 Implement a procedure to replay sync replication wal for standby cluster

Posted by zh...@apache.org.
HBASE-19973 Implement a procedure to replay sync replication wal for standby cluster


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

Branch: refs/heads/HBASE-19064
Commit: ca384b1731429ff8d37fb3032e5c9151426796b6
Parents: 7c326e6
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Mar 2 18:43:25 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/MasterProcedure.proto     |  22 +++
 .../apache/hadoop/hbase/executor/EventType.java |   9 +-
 .../hadoop/hbase/executor/ExecutorType.java     |   3 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   9 +
 .../hadoop/hbase/master/MasterServices.java     |   6 +
 .../procedure/PeerProcedureInterface.java       |   3 +-
 .../hbase/master/procedure/PeerQueue.java       |   3 +-
 .../replication/RecoverStandbyProcedure.java    | 114 +++++++++++
 .../ReplaySyncReplicationWALManager.java        | 139 +++++++++++++
 .../ReplaySyncReplicationWALProcedure.java      | 193 +++++++++++++++++++
 .../hbase/regionserver/HRegionServer.java       |   9 +-
 .../ReplaySyncReplicationWALCallable.java       | 149 ++++++++++++++
 .../SyncReplicationPeerInfoProviderImpl.java    |   3 +
 .../org/apache/hadoop/hbase/util/FSUtils.java   |   5 +
 .../hbase/master/MockNoopMasterServices.java    |   8 +-
 .../master/TestRecoverStandbyProcedure.java     | 186 ++++++++++++++++++
 16 files changed, 854 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/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 e60881f..d58608a 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -468,3 +468,25 @@ message TransitPeerSyncReplicationStateStateData {
   optional SyncReplicationState fromState = 1;
   required SyncReplicationState toState = 2;
 }
+
+enum RecoverStandbyState {
+  RENAME_SYNC_REPLICATION_WALS_DIR = 1;
+  INIT_WORKERS = 2;
+  DISPATCH_TASKS = 3;
+  REMOVE_SYNC_REPLICATION_WALS_DIR = 4;
+}
+
+message RecoverStandbyStateData {
+  required string peer_id = 1;
+}
+
+message ReplaySyncReplicationWALStateData {
+  required string peer_id = 1;
+  required string wal = 2;
+  optional ServerName target_server = 3;
+}
+
+message ReplaySyncReplicationWALParameter {
+  required string peer_id = 1;
+  required string wal = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
index 922deb8..ad38d1c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
@@ -281,7 +281,14 @@ public enum EventType {
    *
    * RS_REFRESH_PEER
    */
-  RS_REFRESH_PEER (84, ExecutorType.RS_REFRESH_PEER);
+  RS_REFRESH_PEER(84, ExecutorType.RS_REFRESH_PEER),
+
+  /**
+   * RS replay sync replication wal.<br>
+   *
+   * RS_REPLAY_SYNC_REPLICATION_WAL
+   */
+  RS_REPLAY_SYNC_REPLICATION_WAL(85, ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL);
 
   private final int code;
   private final ExecutorType executor;

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/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 e7e585d..484f28f 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.AddPeerProcedure;
 import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
+import org.apache.hadoop.hbase.master.replication.ReplaySyncReplicationWALManager;
 import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure;
 import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
@@ -339,6 +340,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   // manager of replication
   private ReplicationPeerManager replicationPeerManager;
 
+  private ReplaySyncReplicationWALManager replaySyncReplicationWALManager;
+
   // buffer for "fatal error" notices from region servers
   // in the cluster. This is only used for assisting
   // operations/debugging.
@@ -828,6 +831,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     initializeMemStoreChunkCreator();
     this.fileSystemManager = new MasterFileSystem(conf);
     this.walManager = new MasterWalManager(this);
+    this.replaySyncReplicationWALManager = new ReplaySyncReplicationWALManager(this);
 
     // enable table descriptors cache
     this.tableDescriptors.setCacheOn();
@@ -3670,4 +3674,9 @@ public class HMaster extends HRegionServer implements MasterServices {
   public SnapshotQuotaObserverChore getSnapshotQuotaObserverChore() {
     return this.snapshotQuotaChore;
   }
+
+  @Override
+  public ReplaySyncReplicationWALManager getReplaySyncReplicationWALManager() {
+    return this.replaySyncReplicationWALManager;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/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 76aa2d6..c5b9200 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.ReplaySyncReplicationWALManager;
 import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
@@ -465,6 +466,11 @@ public interface MasterServices extends Server {
   ReplicationPeerManager getReplicationPeerManager();
 
   /**
+   * Returns the {@link ReplaySyncReplicationWALManager}.
+   */
+  ReplaySyncReplicationWALManager getReplaySyncReplicationWALManager();
+
+  /**
    * Update the peerConfig for the specified peer
    * @param peerId a short name that identifies the peer
    * @param peerConfig new config for the peer

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java
index 1ae0c2f..25feb7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java
@@ -49,6 +49,7 @@ class PeerQueue extends Queue<String> {
   }
 
   private static boolean requirePeerExclusiveLock(PeerProcedureInterface proc) {
-    return proc.getPeerOperationType() != PeerOperationType.REFRESH;
+    return proc.getPeerOperationType() != PeerOperationType.REFRESH
+        && proc.getPeerOperationType() != PeerOperationType.REPLAY_SYNC_REPLICATION_WAL;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java
new file mode 100644
index 0000000..e9e3a97
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RecoverStandbyProcedure.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RecoverStandbyState;
+
+@InterfaceAudience.Private
+public class RecoverStandbyProcedure extends AbstractPeerProcedure<RecoverStandbyState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RecoverStandbyProcedure.class);
+
+  public RecoverStandbyProcedure() {
+  }
+
+  public RecoverStandbyProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, RecoverStandbyState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    ReplaySyncReplicationWALManager replaySyncReplicationWALManager =
+        env.getMasterServices().getReplaySyncReplicationWALManager();
+    switch (state) {
+      case RENAME_SYNC_REPLICATION_WALS_DIR:
+        try {
+          replaySyncReplicationWALManager.renamePeerRemoteWALDir(peerId);
+        } catch (IOException e) {
+          LOG.warn("Failed to rename remote wal dir for peer id={}", peerId, e);
+          setFailure("master-recover-standby", e);
+          return Flow.NO_MORE_STATE;
+        }
+        setNextState(RecoverStandbyState.INIT_WORKERS);
+        return Flow.HAS_MORE_STATE;
+      case INIT_WORKERS:
+        replaySyncReplicationWALManager.initPeerWorkers(peerId);
+        setNextState(RecoverStandbyState.DISPATCH_TASKS);
+        return Flow.HAS_MORE_STATE;
+      case DISPATCH_TASKS:
+        addChildProcedure(getReplayWALs(replaySyncReplicationWALManager).stream()
+            .map(wal -> new ReplaySyncReplicationWALProcedure(peerId,
+                replaySyncReplicationWALManager.removeWALRootPath(wal)))
+            .toArray(ReplaySyncReplicationWALProcedure[]::new));
+        setNextState(RecoverStandbyState.REMOVE_SYNC_REPLICATION_WALS_DIR);
+        return Flow.HAS_MORE_STATE;
+      case REMOVE_SYNC_REPLICATION_WALS_DIR:
+        try {
+          replaySyncReplicationWALManager.removePeerReplayWALDir(peerId);
+        } catch (IOException e) {
+          LOG.warn("Failed to cleanup replay wals dir for peer id={}, , retry", peerId, e);
+          throw new ProcedureYieldException();
+        }
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+  }
+
+  private List<Path> getReplayWALs(ReplaySyncReplicationWALManager replaySyncReplicationWALManager)
+      throws ProcedureYieldException {
+    try {
+      return replaySyncReplicationWALManager.getReplayWALs(peerId);
+    } catch (IOException e) {
+      LOG.warn("Failed to get replay wals for peer id={}, , retry", peerId, e);
+      throw new ProcedureYieldException();
+    }
+  }
+
+  @Override
+  protected RecoverStandbyState getState(int stateId) {
+    return RecoverStandbyState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(RecoverStandbyState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected RecoverStandbyState getInitialState() {
+    return RecoverStandbyState.RENAME_SYNC_REPLICATION_WALS_DIR;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.RECOVER_STANDBY;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
new file mode 100644
index 0000000..72f5c37
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALManager.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class ReplaySyncReplicationWALManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReplaySyncReplicationWALManager.class);
+
+  private static final String REPLAY_SUFFIX = "-replay";
+
+  private final MasterServices services;
+
+  private final Configuration conf;
+
+  private final FileSystem fs;
+
+  private final Path walRootDir;
+
+  private final Path remoteWALDir;
+
+  private final Map<String, BlockingQueue<ServerName>> availServers = new HashMap<>();
+
+  public ReplaySyncReplicationWALManager(MasterServices services) {
+    this.services = services;
+    this.conf = services.getConfiguration();
+    this.fs = services.getMasterFileSystem().getWALFileSystem();
+    this.walRootDir = services.getMasterFileSystem().getWALRootDir();
+    this.remoteWALDir = new Path(this.walRootDir, ReplicationUtils.REMOTE_WAL_DIR_NAME);
+  }
+
+  public Path getPeerRemoteWALDir(String peerId) {
+    return new Path(this.remoteWALDir, peerId);
+  }
+
+  private Path getPeerReplayWALDir(String peerId) {
+    return getPeerRemoteWALDir(peerId).suffix(REPLAY_SUFFIX);
+  }
+
+  public void createPeerRemoteWALDir(String peerId) throws IOException {
+    Path peerRemoteWALDir = getPeerRemoteWALDir(peerId);
+    if (!fs.exists(peerRemoteWALDir) && !fs.mkdirs(peerRemoteWALDir)) {
+      throw new IOException("Unable to mkdir " + peerRemoteWALDir);
+    }
+  }
+
+  public void renamePeerRemoteWALDir(String peerId) throws IOException {
+    Path peerRemoteWALDir = getPeerRemoteWALDir(peerId);
+    Path peerReplayWALDir = peerRemoteWALDir.suffix(REPLAY_SUFFIX);
+    if (fs.exists(peerRemoteWALDir)) {
+      if (!fs.rename(peerRemoteWALDir, peerReplayWALDir)) {
+        throw new IOException("Failed rename remote wal dir from " + peerRemoteWALDir + " to "
+            + peerReplayWALDir + " for peer id=" + peerId);
+      }
+      LOG.info("Rename remote wal dir from {} to {} for peer id={}", remoteWALDir, peerReplayWALDir,
+        peerId);
+    } else if (!fs.exists(peerReplayWALDir)) {
+      throw new IOException("Remote wal dir " + peerRemoteWALDir + " and replay wal dir "
+          + peerReplayWALDir + " not exist for peer id=" + peerId);
+    }
+  }
+
+  public List<Path> getReplayWALs(String peerId) throws IOException {
+    Path peerReplayWALDir = getPeerReplayWALDir(peerId);
+    List<Path> replayWals = new ArrayList<>();
+    RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(peerReplayWALDir, false);
+    while (iterator.hasNext()) {
+      replayWals.add(iterator.next().getPath());
+    }
+    return replayWals;
+  }
+
+  public void removePeerReplayWALDir(String peerId) throws IOException {
+    Path peerReplayWALDir = getPeerReplayWALDir(peerId);
+    if (fs.exists(peerReplayWALDir) && !fs.delete(peerReplayWALDir, true)) {
+      throw new IOException(
+          "Failed to remove replay wals dir " + peerReplayWALDir + " for peer id=" + peerId);
+    }
+  }
+
+  public void initPeerWorkers(String peerId) {
+    BlockingQueue<ServerName> servers = new LinkedBlockingQueue<>();
+    services.getServerManager().getOnlineServers().keySet()
+        .forEach(server -> servers.offer(server));
+    availServers.put(peerId, servers);
+  }
+
+  public ServerName getAvailServer(String peerId, long timeout, TimeUnit unit)
+      throws InterruptedException {
+    return availServers.get(peerId).poll(timeout, unit);
+  }
+
+  public void addAvailServer(String peerId, ServerName server) {
+    availServers.get(peerId).offer(server);
+  }
+
+  public String removeWALRootPath(Path path) {
+    String pathStr = path.toString();
+    // remove the "/" too.
+    return pathStr.substring(walRootDir.toString().length() + 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALProcedure.java
new file mode 100644
index 0000000..8d8a65a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplaySyncReplicationWALProcedure.java
@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.ServerOperation;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
+import org.apache.hadoop.hbase.replication.regionserver.ReplaySyncReplicationWALCallable;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ReplaySyncReplicationWALParameter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ReplaySyncReplicationWALStateData;
+
+@InterfaceAudience.Private
+public class ReplaySyncReplicationWALProcedure extends Procedure<MasterProcedureEnv>
+    implements RemoteProcedure<MasterProcedureEnv, ServerName>, PeerProcedureInterface {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReplaySyncReplicationWALProcedure.class);
+
+  private static final long DEFAULT_WAIT_AVAILABLE_SERVER_TIMEOUT = 10000;
+
+  private String peerId;
+
+  private ServerName targetServer = null;
+
+  private String wal;
+
+  private boolean dispatched;
+
+  private ProcedureEvent<?> event;
+
+  private boolean succ;
+
+  public ReplaySyncReplicationWALProcedure() {
+  }
+
+  public ReplaySyncReplicationWALProcedure(String peerId, String wal) {
+    this.peerId = peerId;
+    this.wal = wal;
+  }
+
+  @Override
+  public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
+    return new ServerOperation(this, getProcId(), ReplaySyncReplicationWALCallable.class,
+        ReplaySyncReplicationWALParameter.newBuilder().setPeerId(peerId).setWal(wal).build()
+            .toByteArray());
+  }
+
+  @Override
+  public void remoteCallFailed(MasterProcedureEnv env, ServerName remote, IOException exception) {
+    complete(env, exception);
+  }
+
+  @Override
+  public void remoteOperationCompleted(MasterProcedureEnv env) {
+    complete(env, null);
+  }
+
+  @Override
+  public void remoteOperationFailed(MasterProcedureEnv env, RemoteProcedureException error) {
+    complete(env, error);
+  }
+
+  private void complete(MasterProcedureEnv env, Throwable error) {
+    if (event == null) {
+      LOG.warn("procedure event for {} is null, maybe the procedure is created when recovery",
+        getProcId());
+      return;
+    }
+    ReplaySyncReplicationWALManager replaySyncReplicationWALManager =
+        env.getMasterServices().getReplaySyncReplicationWALManager();
+    if (error != null) {
+      LOG.warn("Replay sync replication wal {} on {} failed for peer id={}", wal, targetServer,
+        peerId, error);
+      this.succ = false;
+    } else {
+      LOG.warn("Replay sync replication wal {} on {} suceeded for peer id={}", wal, targetServer,
+        peerId);
+      this.succ = true;
+      replaySyncReplicationWALManager.addAvailServer(peerId, targetServer);
+    }
+    event.wake(env.getProcedureScheduler());
+    event = null;
+  }
+
+  @Override
+  protected Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
+      throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
+    if (dispatched) {
+      if (succ) {
+        return null;
+      }
+      // retry
+      dispatched = false;
+    }
+
+    // Try poll a available server
+    if (targetServer == null) {
+      targetServer = env.getMasterServices().getReplaySyncReplicationWALManager()
+          .getAvailServer(peerId, DEFAULT_WAIT_AVAILABLE_SERVER_TIMEOUT, TimeUnit.MILLISECONDS);
+      if (targetServer == null) {
+        LOG.info("No available server to replay wal {} for peer id={}, retry", wal, peerId);
+        throw new ProcedureYieldException();
+      }
+    }
+
+    // Dispatch task to target server
+    if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) {
+      LOG.info(
+        "Can not add remote operation for replay wal {} on {} for peer id={}, "
+            + "this usually because the server is already dead, " + "retry",
+        wal, targetServer, peerId);
+      targetServer = null;
+      throw new ProcedureYieldException();
+    }
+    dispatched = true;
+    event = new ProcedureEvent<>(this);
+    event.suspendIfNotReady(this);
+    throw new ProcedureSuspendedException();
+  }
+
+  @Override
+  protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    return false;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    ReplaySyncReplicationWALStateData.Builder builder =
+        ReplaySyncReplicationWALStateData.newBuilder().setPeerId(peerId).setWal(wal);
+    if (targetServer != null) {
+      builder.setTargetServer(ProtobufUtil.toServerName(targetServer));
+    }
+    serializer.serialize(builder.build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    ReplaySyncReplicationWALStateData data =
+        serializer.deserialize(ReplaySyncReplicationWALStateData.class);
+    peerId = data.getPeerId();
+    wal = data.getWal();
+    if (data.hasTargetServer()) {
+      targetServer = ProtobufUtil.toServerName(data.getTargetServer());
+    }
+  }
+
+  @Override
+  public String getPeerId() {
+    return peerId;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.REPLAY_SYNC_REPLICATION_WAL;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/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 41969fd..3aadb9c 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
@@ -1926,6 +1926,11 @@ public class HRegionServer extends HasThread implements
     this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER,
       conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2));
 
+    if (conf.getBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, false)) {
+      this.executorService.startExecutorService(ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL,
+        conf.getInt("hbase.regionserver.executor.replay.sync.replication.wal.threads", 2));
+    }
+
     Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
     uncaughtExceptionHandler);
     this.cacheFlusher.start(uncaughtExceptionHandler);
@@ -2871,14 +2876,14 @@ public class HRegionServer extends HasThread implements
   /**
    * @return Return the walRootDir.
    */
-  protected Path getWALRootDir() {
+  public Path getWALRootDir() {
     return walRootDir;
   }
 
   /**
    * @return Return the walFs.
    */
-  protected FileSystem getWALFileSystem() {
+  public FileSystem getWALFileSystem() {
     return walFs;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
new file mode 100644
index 0000000..8dfe3a2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.ArrayList;
+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.CellScanner;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ReplaySyncReplicationWALParameter;
+
+/**
+ * This callable executed at RS side to replay sync replication wal.
+ */
+@InterfaceAudience.Private
+public class ReplaySyncReplicationWALCallable implements RSProcedureCallable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReplaySyncReplicationWALCallable.class);
+
+  private static final String REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE =
+      "hbase.replay.sync.replication.wal.batch.size";
+
+  private static final long DEFAULT_REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE = 8 * 1024 * 1024;
+
+  private HRegionServer rs;
+
+  private FileSystem fs;
+
+  private Configuration conf;
+
+  private String peerId;
+
+  private String wal;
+
+  private Exception initError;
+
+  private long batchSize;
+
+  @Override
+  public Void call() throws Exception {
+    if (initError != null) {
+      throw initError;
+    }
+    LOG.info("Received a replay sync replication wal {} event, peerId={}", wal, peerId);
+    try (Reader reader = getReader()) {
+      List<Entry> entries = readWALEntries(reader);
+      while (!entries.isEmpty()) {
+        Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> pair = ReplicationProtbufUtil
+            .buildReplicateWALEntryRequest(entries.toArray(new Entry[entries.size()]));
+        HBaseRpcController controller = new HBaseRpcControllerImpl(pair.getSecond());
+        rs.getRSRpcServices().replicateWALEntry(controller, pair.getFirst());
+        entries = readWALEntries(reader);
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public void init(byte[] parameter, HRegionServer rs) {
+    this.rs = rs;
+    this.fs = rs.getWALFileSystem();
+    this.conf = rs.getConfiguration();
+    try {
+      ReplaySyncReplicationWALParameter param =
+          ReplaySyncReplicationWALParameter.parseFrom(parameter);
+      this.peerId = param.getPeerId();
+      this.wal = param.getWal();
+      this.batchSize = rs.getConfiguration().getLong(REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE,
+        DEFAULT_REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE);
+    } catch (InvalidProtocolBufferException e) {
+      initError = e;
+    }
+  }
+
+  @Override
+  public EventType getEventType() {
+    return EventType.RS_REPLAY_SYNC_REPLICATION_WAL;
+  }
+
+  private Reader getReader() throws IOException {
+    Path path = new Path(rs.getWALRootDir(), wal);
+    long length = rs.getWALFileSystem().getFileStatus(path).getLen();
+    try {
+      FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf);
+      return WALFactory.createReader(rs.getWALFileSystem(), path, rs.getConfiguration());
+    } catch (EOFException e) {
+      if (length <= 0) {
+        LOG.warn("File is empty. Could not open {} for reading because {}", path, e);
+        return null;
+      }
+      throw e;
+    }
+  }
+
+  private List<Entry> readWALEntries(Reader reader) throws IOException {
+    List<Entry> entries = new ArrayList<>();
+    if (reader == null) {
+      return entries;
+    }
+    long size = 0;
+    Entry entry = reader.next();
+    while (entry != null) {
+      entries.add(entry);
+      size += entry.getEdit().heapSize();
+      if (size > batchSize) {
+        break;
+      }
+      entry = reader.next();
+    }
+    return entries;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
index 973e049..e4afc33 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
@@ -41,6 +41,9 @@ class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProv
 
   @Override
   public Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
+    if (info == null) {
+      return Optional.empty();
+    }
     String peerId = mapping.getPeerId(info);
     if (peerId == null) {
       return Optional.empty();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 53db140..8a1f948 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -939,6 +939,11 @@ public abstract class FSUtils extends CommonFSUtils {
     }
   }
 
+  public void recoverFileLease(final FileSystem fs, final Path p, Configuration conf)
+      throws IOException {
+    recoverFileLease(fs, p, conf, null);
+  }
+
   /**
    * Recover file lease. Used when a file might be suspect
    * to be had been left open by another process.

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/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 dce062c..60132a2 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.ReplaySyncReplicationWALManager;
 import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
@@ -490,4 +491,9 @@ public class MockNoopMasterServices implements MasterServices {
     SyncReplicationState clusterState) throws ReplicationException, IOException {
     return 0;
   }
-}
+
+  @Override
+  public ReplaySyncReplicationWALManager getReplaySyncReplicationWALManager() {
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca384b17/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
new file mode 100644
index 0000000..f3d61bb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.master;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+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.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.RecoverStandbyProcedure;
+import org.apache.hadoop.hbase.master.replication.ReplaySyncReplicationWALManager;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
+import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({MasterTests.class, LargeTests.class})
+public class TestRecoverStandbyProcedure {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestRecoverStandbyProcedure.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestRecoverStandbyProcedure.class);
+
+  private static final TableName tableName = TableName.valueOf("TestRecoverStandbyProcedure");
+
+  private static final RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build();
+
+  private static final byte[] family = Bytes.toBytes("CF");
+
+  private static final byte[] qualifier = Bytes.toBytes("q");
+
+  private static final long timestamp = System.currentTimeMillis();
+
+  private static final int ROW_COUNT = 1000;
+
+  private static final int WAL_NUMBER = 10;
+
+  private static final int RS_NUMBER = 3;
+
+  private static final String PEER_ID = "1";
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static ReplaySyncReplicationWALManager replaySyncReplicationWALManager;
+
+  private static ProcedureExecutor<MasterProcedureEnv> procExec;
+
+  private static FileSystem fs;
+
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    UTIL.getConfiguration().setBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, true);
+    UTIL.startMiniCluster(RS_NUMBER);
+    UTIL.getHBaseCluster().waitForActiveAndReadyMaster();
+    conf = UTIL.getConfiguration();
+    HMaster master = UTIL.getHBaseCluster().getMaster();
+    fs = master.getMasterFileSystem().getWALFileSystem();
+    replaySyncReplicationWALManager = master.getReplaySyncReplicationWALManager();
+    procExec = master.getMasterProcedureExecutor();
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Before
+  public void setupBeforeTest() throws IOException {
+    UTIL.createTable(tableName, family);
+  }
+
+  @After
+  public void tearDownAfterTest() throws IOException {
+    try (Admin admin = UTIL.getAdmin()) {
+      if (admin.isTableEnabled(tableName)) {
+        admin.disableTable(tableName);
+      }
+      admin.deleteTable(tableName);
+    }
+  }
+
+  @Test
+  public void testRecoverStandby() throws IOException, StreamLacksCapabilityException {
+    setupSyncReplicationWALs();
+    long procId = procExec.submitProcedure(new RecoverStandbyProcedure(PEER_ID));
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < WAL_NUMBER * ROW_COUNT; i++) {
+        Result result = table.get(new Get(Bytes.toBytes(i)).setTimestamp(timestamp));
+        assertNotNull(result);
+        assertEquals(i, Bytes.toInt(result.getValue(family, qualifier)));
+      }
+    }
+  }
+
+  private void setupSyncReplicationWALs() throws IOException, StreamLacksCapabilityException {
+    Path peerRemoteWALDir = replaySyncReplicationWALManager.getPeerRemoteWALDir(PEER_ID);
+    if (!fs.exists(peerRemoteWALDir)) {
+      fs.mkdirs(peerRemoteWALDir);
+    }
+    for (int i = 0; i < WAL_NUMBER; i++) {
+      try (ProtobufLogWriter writer = new ProtobufLogWriter()) {
+        Path wal = new Path(peerRemoteWALDir, "srv1,8888." + i + ".syncrep");
+        writer.init(fs, wal, conf, true, WALUtil.getWALBlockSize(conf, fs, peerRemoteWALDir));
+        List<Entry> entries = setupWALEntries(i * ROW_COUNT, (i + 1) * ROW_COUNT);
+        for (Entry entry : entries) {
+          writer.append(entry);
+        }
+        writer.sync(false);
+        LOG.info("Created wal {} to replay for peer id={}", wal, PEER_ID);
+      }
+    }
+  }
+
+  private List<Entry> setupWALEntries(int startRow, int endRow) {
+    return IntStream.range(startRow, endRow)
+        .mapToObj(i -> createWALEntry(Bytes.toBytes(i), Bytes.toBytes(i)))
+        .collect(Collectors.toList());
+  }
+
+  private Entry createWALEntry(byte[] row, byte[] value) {
+    WALKeyImpl key = new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName, 1);
+    WALEdit edit = new WALEdit();
+    edit.add(new KeyValue(row, family, qualifier, timestamp, value));
+    return new Entry(key, edit);
+  }
+}


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

Branch: refs/heads/HBASE-19064
Commit: e62f8a02173c0d010e831dcc883c3cddbef71389
Parents: 0aba887
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 25 20:09:00 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 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/e62f8a02/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/e62f8a02/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 d656466..ff3f662 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/e62f8a02/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 4d602ca..a54f339 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
@@ -134,7 +134,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/e62f8a02/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/e62f8a02/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/e62f8a02/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/e62f8a02/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/e62f8a02/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/e62f8a02/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 339fd6c..737ccfb 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;
@@ -188,7 +188,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 */
@@ -197,9 +197,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/e62f8a02/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 8170893..04c7aad 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
@@ -173,6 +173,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/e62f8a02/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/e62f8a02/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());
-  }
-}


[07/36] hbase git commit: HBASE-20653 Add missing observer hooks for region server group to MasterObserver

Posted by zh...@apache.org.
HBASE-20653 Add missing observer hooks for region server group to MasterObserver

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


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

Branch: refs/heads/HBASE-19064
Commit: 40a73a5ca73c9e9e2ff9be1bf823056b108686af
Parents: c19fbf2
Author: Nihal Jain <ni...@gmail.com>
Authored: Wed May 30 23:37:48 2018 +0530
Committer: tedyu <yu...@gmail.com>
Committed: Wed May 30 21:29:07 2018 -0700

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     | 24 ++++++
 .../hadoop/hbase/rsgroup/TestRSGroups.java      | 86 ++++++++++++++++++++
 .../hbase/rsgroup/TestRSGroupsWithACL.java      | 50 ++++++------
 .../hbase/coprocessor/MasterObserver.java       | 62 ++++++++++++++
 .../hbase/master/MasterCoprocessorHost.java     | 72 ++++++++++++++++
 5 files changed, 267 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index fa7537a..2efc3a4 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -161,11 +161,17 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, group="
               + groupName);
       try {
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().preGetRSGroupInfo(groupName);
+        }
         checkPermission("getRSGroupInfo");
         RSGroupInfo rsGroupInfo = groupAdminServer.getRSGroupInfo(groupName);
         if (rsGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(rsGroupInfo));
         }
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().postGetRSGroupInfo(groupName);
+        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }
@@ -180,11 +186,17 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, table="
           + tableName);
       try {
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().preGetRSGroupInfoOfTable(tableName);
+        }
         checkPermission("getRSGroupInfoOfTable");
         RSGroupInfo RSGroupInfo = groupAdminServer.getRSGroupInfoOfTable(tableName);
         if (RSGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
         }
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().postGetRSGroupInfoOfTable(tableName);
+        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }
@@ -312,10 +324,16 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       ListRSGroupInfosResponse.Builder builder = ListRSGroupInfosResponse.newBuilder();
       LOG.info(master.getClientIdAuditPrefix() + " list rsgroup");
       try {
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().preListRSGroups();
+        }
         checkPermission("listRSGroup");
         for (RSGroupInfo RSGroupInfo : groupAdminServer.listRSGroups()) {
           builder.addRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
         }
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().postListRSGroups();
+        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }
@@ -331,11 +349,17 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server="
           + hp);
       try {
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().preGetRSGroupInfoOfServer(hp);
+        }
         checkPermission("getRSGroupInfoOfServer");
         RSGroupInfo info = groupAdminServer.getRSGroupOfServer(hp);
         if (info != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(info));
         }
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().postGetRSGroupInfoOfServer(hp);
+        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
index 3e74f81..c2fc0f1 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
@@ -292,6 +292,14 @@ public class TestRSGroups extends TestRSGroupsBase {
     boolean postRemoveServersCalled = false;
     boolean preMoveServersAndTables = false;
     boolean postMoveServersAndTables = false;
+    boolean preGetRSGroupInfoCalled = false;
+    boolean postGetRSGroupInfoCalled = false;
+    boolean preGetRSGroupInfoOfTableCalled = false;
+    boolean postGetRSGroupInfoOfTableCalled = false;
+    boolean preListRSGroupsCalled = false;
+    boolean postListRSGroupsCalled = false;
+    boolean preGetRSGroupInfoOfServerCalled = false;
+    boolean postGetRSGroupInfoOfServerCalled = false;
 
     @Override
     public Optional<MasterObserver> getMasterObserver() {
@@ -370,7 +378,85 @@ public class TestRSGroups extends TestRSGroupsBase {
         String groupName, boolean balancerRan) throws IOException {
       postBalanceRSGroupCalled = true;
     }
+
+    @Override
+    public void preGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final String groupName) throws IOException {
+      preGetRSGroupInfoCalled = true;
+    }
+
+    @Override
+    public void postGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final String groupName) throws IOException {
+      postGetRSGroupInfoCalled = true;
+    }
+
+    @Override
+    public void preGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final TableName tableName) throws IOException {
+      preGetRSGroupInfoOfTableCalled = true;
+    }
+
+    @Override
+    public void postGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final TableName tableName) throws IOException {
+      postGetRSGroupInfoOfTableCalled = true;
+    }
+
+    @Override
+    public void preListRSGroups(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+        throws IOException {
+      preListRSGroupsCalled = true;
+    }
+
+    @Override
+    public void postListRSGroups(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+        throws IOException {
+      postListRSGroupsCalled = true;
+    }
+
+    @Override
+    public void preGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final Address server) throws IOException {
+      preGetRSGroupInfoOfServerCalled = true;
+    }
+
+    @Override
+    public void postGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final Address server) throws IOException {
+      postGetRSGroupInfoOfServerCalled = true;
+    }
+  }
+
+  @Test
+  public void testGetRSGroupInfoCPHookCalled() throws Exception {
+    rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
+    assertTrue(observer.preGetRSGroupInfoCalled);
+    assertTrue(observer.postGetRSGroupInfoCalled);
+  }
+
+  @Test
+  public void testGetRSGroupInfoOfTableCPHookCalled() throws Exception {
+    rsGroupAdmin.getRSGroupInfoOfTable(TableName.META_TABLE_NAME);
+    assertTrue(observer.preGetRSGroupInfoOfTableCalled);
+    assertTrue(observer.postGetRSGroupInfoOfTableCalled);
+  }
+
+  @Test
+  public void testListRSGroupsCPHookCalled() throws Exception {
+    rsGroupAdmin.listRSGroups();
+    assertTrue(observer.preListRSGroupsCalled);
+    assertTrue(observer.postListRSGroupsCalled);
   }
+
+  @Test
+  public void testGetRSGroupInfoOfServerCPHookCalled() throws Exception {
+    ServerName masterServerName = ((MiniHBaseCluster) cluster).getMaster().getServerName();
+    rsGroupAdmin.getRSGroupOfServer(masterServerName.getAddress());
+    assertTrue(observer.preGetRSGroupInfoOfServerCalled);
+    assertTrue(observer.postGetRSGroupInfoOfServerCalled);
+  }
+
   @Test
   public void testMoveServersAndTables() throws Exception {
     super.testMoveServersAndTables();

http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/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 afdff71..a63626d 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
@@ -223,9 +223,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -235,9 +233,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -247,9 +243,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -259,9 +253,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -271,9 +263,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -283,9 +273,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -295,9 +283,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -307,9 +293,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -319,9 +303,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -331,6 +313,20 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
+    validateAdminPermissions(action);
+  }
+
+  @Test
+  public void testRemoveServers() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("removeServers");
+      return null;
+    };
+
+    validateAdminPermissions(action);
+  }
+
+  private void validateAdminPermissions(AccessTestAction action) throws Exception {
     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
         USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/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 f60a04d..3175af3 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
@@ -1213,6 +1213,68 @@ public interface MasterObserver {
       Set<Address> servers) throws IOException {}
 
   /**
+   * Called before getting region server group info of the passed groupName.
+   * @param ctx the environment to interact with the framework and master
+   * @param groupName name of the group to get RSGroupInfo for
+   */
+  default void preGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final String groupName) throws IOException {}
+
+  /**
+   * Called after getting region server group info of the passed groupName.
+   * @param ctx the environment to interact with the framework and master
+   * @param groupName name of the group to get RSGroupInfo for
+   */
+  default void postGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final String groupName) throws IOException {}
+
+  /**
+   * Called before getting region server group info of the passed tableName.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName name of the table to get RSGroupInfo for
+   */
+  default void preGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final TableName tableName) throws IOException {}
+
+  /**
+   * Called after getting region server group info of the passed tableName.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName name of the table to get RSGroupInfo for
+   */
+  default void postGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final TableName tableName) throws IOException {}
+
+  /**
+   * Called before listing region server group information.
+   * @param ctx the environment to interact with the framework and master
+   */
+  default void preListRSGroups(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException {}
+
+  /**
+   * Called after listing region server group information.
+   * @param ctx the environment to interact with the framework and master
+   */
+  default void postListRSGroups(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException {}
+
+  /**
+   * Called before getting region server group info of the passed server.
+   * @param ctx the environment to interact with the framework and master
+   * @param server server to get RSGroupInfo for
+   */
+  default void preGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final Address server) throws IOException {}
+
+  /**
+   * Called after getting region server group info of the passed server.
+   * @param ctx the environment to interact with the framework and master
+   * @param server server to get RSGroupInfo for
+   */
+  default void postGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final Address server) throws IOException {}
+
+  /**
    * Called before add a replication peer
    * @param ctx the environment to interact with the framework and master
    * @param peerId a short name that identifies the peer

http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/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 072ae8a..e563cd4 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
@@ -1404,6 +1404,78 @@ public class MasterCoprocessorHost
     });
   }
 
+  public void preGetRSGroupInfo(final String groupName) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetRSGroupInfo(this, groupName);
+      }
+    });
+  }
+
+  public void postGetRSGroupInfo(final String groupName) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetRSGroupInfo(this, groupName);
+      }
+    });
+  }
+
+  public void preGetRSGroupInfoOfTable(final TableName tableName) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetRSGroupInfoOfTable(this, tableName);
+      }
+    });
+  }
+
+  public void postGetRSGroupInfoOfTable(final TableName tableName) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetRSGroupInfoOfTable(this, tableName);
+      }
+    });
+  }
+
+  public void preListRSGroups() throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preListRSGroups(this);
+      }
+    });
+  }
+
+  public void postListRSGroups() throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postListRSGroups(this);
+      }
+    });
+  }
+
+  public void preGetRSGroupInfoOfServer(final Address server) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetRSGroupInfoOfServer(this, server);
+      }
+    });
+  }
+
+  public void postGetRSGroupInfoOfServer(final Address server) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetRSGroupInfoOfServer(this, server);
+      }
+    });
+  }
+
   public void preAddReplicationPeer(final String peerId, final ReplicationPeerConfig peerConfig)
       throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {


[31/36] hbase git commit: HBASE-19082 Reject read/write from client but accept write from replication in state S

Posted by zh...@apache.org.
HBASE-19082 Reject read/write from client but accept write from replication in state S


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

Branch: refs/heads/HBASE-19064
Commit: ca94b153935176f9d7b7806c4259111ad6e345a2
Parents: 31f8fe0
Author: zhangduo <zh...@apache.org>
Authored: Mon Feb 12 18:20:18 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HConstants.java     |   3 -
 .../src/main/protobuf/MasterProcedure.proto     |   3 +-
 .../hbase/replication/ReplicationUtils.java     |   4 +
 ...ransitPeerSyncReplicationStateProcedure.java |  10 +
 .../hadoop/hbase/regionserver/HRegion.java      |   5 +-
 .../hbase/regionserver/HRegionServer.java       |   2 +-
 .../hbase/regionserver/RSRpcServices.java       |  88 ++++++--
 .../RejectRequestsFromClientStateChecker.java   |  44 ++++
 .../regionserver/ReplicationSink.java           |  72 ++++---
 .../SyncReplicationPeerInfoProvider.java        |  10 +-
 .../SyncReplicationPeerInfoProviderImpl.java    |  19 +-
 .../hbase/wal/SyncReplicationWALProvider.java   |   3 +
 .../org/apache/hadoop/hbase/wal/WALFactory.java |   4 +-
 .../hbase/replication/TestSyncReplication.java  | 200 +++++++++++++++++++
 .../wal/TestSyncReplicationWALProvider.java     |   8 +-
 15 files changed, 401 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 522c2cf..9241682 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1355,9 +1355,6 @@ public final class HConstants {
 
   public static final String NOT_IMPLEMENTED = "Not implemented";
 
-  // TODO: need to find a better place to hold it.
-  public static final String SYNC_REPLICATION_ENABLED = "hbase.replication.sync.enabled";
-
   private HConstants() {
     // Can't be instantiated with this ctor.
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/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 56ac0d0..e60881f 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -397,7 +397,8 @@ enum PeerSyncReplicationStateTransitionState {
   REOPEN_ALL_REGIONS_IN_PEER = 5;
   TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE = 6;
   REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END = 7;
-  POST_PEER_SYNC_REPLICATION_STATE_TRANSITION = 8;
+  CREATE_DIR_FOR_REMOTE_WAL = 8;
+  POST_PEER_SYNC_REPLICATION_STATE_TRANSITION = 9;
 }
 
 message PeerModificationStateData {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/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 e4dea83..d94cb00 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
@@ -37,6 +37,10 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public final class ReplicationUtils {
 
+  public static final String SYNC_REPLICATION_ENABLED = "hbase.replication.sync.enabled";
+
+  public static final String REPLICATION_ATTR_NAME = "__rep__";
+
   private ReplicationUtils() {
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/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 8fc932f..69404a0 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
@@ -197,8 +197,18 @@ public class TransitPeerSyncReplicationStateProcedure
         addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
           .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 1))
           .toArray(RefreshPeerProcedure[]::new));
+        if (toState == SyncReplicationState.STANDBY) {
+          setNextState(PeerSyncReplicationStateTransitionState.CREATE_DIR_FOR_REMOTE_WAL);
+        } else {
+          setNextState(
+            PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION);
+        }
+        return Flow.HAS_MORE_STATE;
+      case CREATE_DIR_FOR_REMOTE_WAL:
+        // TODO: create wal for write remote wal
         setNextState(
           PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION);
+        return Flow.HAS_MORE_STATE;
       case POST_PEER_SYNC_REPLICATION_STATE_TRANSITION:
         try {
           postTransit(env);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 404eb1a..05fb036 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -4331,12 +4331,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   /**
    * Add updates first to the wal and then add values to memstore.
+   * <p>
    * Warning: Assumption is caller has lock on passed in row.
    * @param edits Cell updates by column
-   * @throws IOException
    */
-  void put(final byte [] row, byte [] family, List<Cell> edits)
-  throws IOException {
+  void put(final byte[] row, byte[] family, List<Cell> edits) throws IOException {
     NavigableMap<byte[], List<Cell>> familyMap;
     familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/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 9c23750..41969fd 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
@@ -1804,7 +1804,7 @@ public class HRegionServer extends HasThread implements
     boolean isMasterNoTableOrSystemTableOnly = this instanceof HMaster &&
       (!LoadBalancer.isTablesOnMaster(conf) || LoadBalancer.isSystemTablesOnlyOnMaster(conf));
     if (isMasterNoTableOrSystemTableOnly) {
-      conf.setBoolean(HConstants.SYNC_REPLICATION_ENABLED, false);
+      conf.setBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, false);
     }
     WALFactory factory = new WALFactory(conf, serverName.toString());
     if (!isMasterNoTableOrSystemTableOnly) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 8828a22..5316ac5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -120,6 +120,8 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack
 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.regionserver.RejectRequestsFromClientStateChecker;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
@@ -2431,6 +2433,18 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return region.execService(execController, serviceCall);
   }
 
+  private boolean shouldRejectRequestsFromClient(HRegion region) {
+    return regionServer.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
+      .checkState(region.getRegionInfo(), RejectRequestsFromClientStateChecker.get());
+  }
+
+  private void rejectIfInStandByState(HRegion region) throws DoNotRetryIOException {
+    if (shouldRejectRequestsFromClient(region)) {
+      throw new DoNotRetryIOException(
+        region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state.");
+    }
+  }
+
   /**
    * Get data from a table.
    *
@@ -2439,8 +2453,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * @throws ServiceException
    */
   @Override
-  public GetResponse get(final RpcController controller,
-      final GetRequest request) throws ServiceException {
+  public GetResponse get(final RpcController controller, final GetRequest request)
+      throws ServiceException {
     long before = EnvironmentEdgeManager.currentTime();
     OperationQuota quota = null;
     HRegion region = null;
@@ -2449,6 +2463,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       requestCount.increment();
       rpcGetRequestCount.increment();
       region = getRegion(request.getRegion());
+      rejectIfInStandByState(region);
 
       GetResponse.Builder builder = GetResponse.newBuilder();
       ClientProtos.Get get = request.getGet();
@@ -2587,16 +2602,45 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
   }
 
+  private void failRegionAction(MultiResponse.Builder responseBuilder,
+      RegionActionResult.Builder regionActionResultBuilder, RegionAction regionAction,
+      CellScanner cellScanner, Throwable error) {
+    rpcServer.getMetrics().exception(error);
+    regionActionResultBuilder.setException(ResponseConverter.buildException(error));
+    responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
+    // All Mutations in this RegionAction not executed as we can not see the Region online here
+    // in this RS. Will be retried from Client. Skipping all the Cells in CellScanner
+    // corresponding to these Mutations.
+    if (cellScanner != null) {
+      skipCellsForMutations(regionAction.getActionList(), cellScanner);
+    }
+  }
+
+  private boolean isReplicationRequest(Action action) {
+    // replication request can only be put or delete.
+    if (!action.hasMutation()) {
+      return false;
+    }
+    MutationProto mutation = action.getMutation();
+    MutationType type = mutation.getMutateType();
+    if (type != MutationType.PUT && type != MutationType.DELETE) {
+      return false;
+    }
+    // replication will set a special attribute so we can make use of it to decide whether a request
+    // is for replication.
+    return mutation.getAttributeList().stream().map(p -> p.getName())
+      .filter(n -> n.equals(ReplicationUtils.REPLICATION_ATTR_NAME)).findAny().isPresent();
+  }
+
   /**
    * Execute multiple actions on a table: get, mutate, and/or execCoprocessor
-   *
    * @param rpcc the RPC controller
    * @param request the multi request
    * @throws ServiceException
    */
   @Override
   public MultiResponse multi(final RpcController rpcc, final MultiRequest request)
-  throws ServiceException {
+      throws ServiceException {
     try {
       checkOpen();
     } catch (IOException ie) {
@@ -2636,17 +2680,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         region = getRegion(regionSpecifier);
         quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList());
       } catch (IOException e) {
-        rpcServer.getMetrics().exception(e);
-        regionActionResultBuilder.setException(ResponseConverter.buildException(e));
-        responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
-        // All Mutations in this RegionAction not executed as we can not see the Region online here
-        // in this RS. Will be retried from Client. Skipping all the Cells in CellScanner
-        // corresponding to these Mutations.
-        skipCellsForMutations(regionAction.getActionList(), cellScanner);
+        failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner, e);
         continue;  // For this region it's a failure.
       }
-
+      boolean rejectIfFromClient = shouldRejectRequestsFromClient(region);
       if (regionAction.hasAtomic() && regionAction.getAtomic()) {
+        // We only allow replication in standby state and it will not set the atomic flag.
+        if (rejectIfFromClient) {
+          failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner,
+            new DoNotRetryIOException(
+              region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state"));
+          quota.close();
+          continue;
+        }
         // How does this call happen?  It may need some work to play well w/ the surroundings.
         // Need to return an item per Action along w/ Action index.  TODO.
         try {
@@ -2677,6 +2723,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           regionActionResultBuilder.setException(ResponseConverter.buildException(e));
         }
       } else {
+        if (rejectIfFromClient && regionAction.getActionCount() > 0 &&
+          !isReplicationRequest(regionAction.getAction(0))) {
+          // fail if it is not a replication request
+          failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner,
+            new DoNotRetryIOException(
+              region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state"));
+          quota.close();
+          continue;
+        }
         // doNonAtomicRegionMutation manages the exception internally
         if (context != null && closeCallBack == null) {
           // An RpcCallBack that creates a list of scanners that needs to perform callBack
@@ -2692,7 +2747,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
       quota.close();
       ClientProtos.RegionLoadStats regionLoadStats = region.getLoadStatistics();
-      if(regionLoadStats != null) {
+      if (regionLoadStats != null) {
         regionStats.put(regionSpecifier, regionLoadStats);
       }
     }
@@ -2751,8 +2806,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * @param request the mutate request
    */
   @Override
-  public MutateResponse mutate(final RpcController rpcc,
-      final MutateRequest request) throws ServiceException {
+  public MutateResponse mutate(final RpcController rpcc, final MutateRequest request)
+      throws ServiceException {
     // rpc controller is how we bring in data via the back door;  it is unprotobuf'ed data.
     // It is also the conduit via which we pass back data.
     HBaseRpcController controller = (HBaseRpcController)rpcc;
@@ -2772,6 +2827,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       requestCount.increment();
       rpcMutateRequestCount.increment();
       region = getRegion(request.getRegion());
+      rejectIfInStandByState(region);
       MutateResponse.Builder builder = MutateResponse.newBuilder();
       MutationProto mutation = request.getMutation();
       if (!region.getRegionInfo().isMetaRegion()) {
@@ -2941,6 +2997,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                 "'hbase.client.scanner.timeout.period' configuration.");
       }
     }
+    rejectIfInStandByState(rsh.r);
     RegionInfo hri = rsh.s.getRegionInfo();
     // Yes, should be the same instance
     if (regionServer.getOnlineRegion(hri.getRegionName()) != rsh.r) {
@@ -2967,6 +3024,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   private RegionScannerHolder newRegionScanner(ScanRequest request, ScanResponse.Builder builder)
       throws IOException {
     HRegion region = getRegion(request.getRegion());
+    rejectIfInStandByState(region);
     ClientProtos.Scan protoScan = request.getScan();
     boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand();
     Scan scan = ProtobufUtil.toScan(protoScan);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectRequestsFromClientStateChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectRequestsFromClientStateChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectRequestsFromClientStateChecker.java
new file mode 100644
index 0000000..8e68f0f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectRequestsFromClientStateChecker.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.util.function.BiPredicate;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Check whether we need to reject the request from client.
+ */
+@InterfaceAudience.Private
+public class RejectRequestsFromClientStateChecker
+    implements BiPredicate<SyncReplicationState, SyncReplicationState> {
+
+  private static final RejectRequestsFromClientStateChecker INST =
+    new RejectRequestsFromClientStateChecker();
+
+  @Override
+  public boolean test(SyncReplicationState state, SyncReplicationState newState) {
+    // reject requests from client if we are in standby state, or we are going to transit to standby
+    // state.
+    return state == SyncReplicationState.STANDBY || newState == SyncReplicationState.STANDBY;
+  }
+
+  public static RejectRequestsFromClientStateChecker get() {
+    return INST;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index fb4e0f9..eb09a3a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -29,7 +28,6 @@ import java.util.Map.Entry;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -41,9 +39,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -52,13 +47,18 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 
 /**
  * <p>
@@ -82,10 +82,10 @@ public class ReplicationSink {
   private final Configuration conf;
   // Volatile because of note in here -- look for double-checked locking:
   // http://www.oracle.com/technetwork/articles/javase/bloch-effective-08-qa-140880.html
-  private volatile Connection sharedHtableCon;
+  private volatile Connection sharedConn;
   private final MetricsSink metrics;
   private final AtomicLong totalReplicatedEdits = new AtomicLong();
-  private final Object sharedHtableConLock = new Object();
+  private final Object sharedConnLock = new Object();
   // Number of hfiles that we successfully replicated
   private long hfilesReplicated = 0;
   private SourceFSConfigurationProvider provider;
@@ -108,12 +108,12 @@ public class ReplicationSink {
         conf.get("hbase.replication.source.fs.conf.provider",
           DefaultSourceFSConfigurationProvider.class.getCanonicalName());
     try {
-      @SuppressWarnings("rawtypes")
-      Class c = Class.forName(className);
-      this.provider = (SourceFSConfigurationProvider) c.getDeclaredConstructor().newInstance();
+      Class<? extends SourceFSConfigurationProvider> c =
+          Class.forName(className).asSubclass(SourceFSConfigurationProvider.class);
+      this.provider = c.getDeclaredConstructor().newInstance();
     } catch (Exception e) {
-      throw new IllegalArgumentException("Configured source fs configuration provider class "
-          + className + " throws error.", e);
+      throw new IllegalArgumentException(
+        "Configured source fs configuration provider class " + className + " throws error.", e);
     }
   }
 
@@ -221,6 +221,8 @@ public class ReplicationSink {
                 clusterIds.add(toUUID(clusterId));
               }
               mutation.setClusterIds(clusterIds);
+              mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,
+                HConstants.EMPTY_BYTE_ARRAY);
               addToHashMultiMap(rowMap, table, clusterIds, mutation);
             }
             if (CellUtil.isDelete(cell)) {
@@ -374,11 +376,11 @@ public class ReplicationSink {
    */
   public void stopReplicationSinkServices() {
     try {
-      if (this.sharedHtableCon != null) {
-        synchronized (sharedHtableConLock) {
-          if (this.sharedHtableCon != null) {
-            this.sharedHtableCon.close();
-            this.sharedHtableCon = null;
+      if (this.sharedConn != null) {
+        synchronized (sharedConnLock) {
+          if (this.sharedConn != null) {
+            this.sharedConn.close();
+            this.sharedConn = null;
           }
         }
       }
@@ -394,14 +396,12 @@ public class ReplicationSink {
    * @param allRows list of actions
    * @throws IOException
    */
-  protected void batch(TableName tableName, Collection<List<Row>> allRows) throws IOException {
+  private void batch(TableName tableName, Collection<List<Row>> allRows) throws IOException {
     if (allRows.isEmpty()) {
       return;
     }
-    Table table = null;
-    try {
-      Connection connection = getConnection();
-      table = connection.getTable(tableName);
+    Connection connection = getConnection();
+    try (Table table = connection.getTable(tableName)) {
       for (List<Row> rows : allRows) {
         table.batch(rows, null);
       }
@@ -414,21 +414,18 @@ public class ReplicationSink {
       throw rewde;
     } catch (InterruptedException ix) {
       throw (InterruptedIOException) new InterruptedIOException().initCause(ix);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
     }
   }
 
   private Connection getConnection() throws IOException {
     // See https://en.wikipedia.org/wiki/Double-checked_locking
-    Connection connection = sharedHtableCon;
+    Connection connection = sharedConn;
     if (connection == null) {
-      synchronized (sharedHtableConLock) {
-        connection = sharedHtableCon;
+      synchronized (sharedConnLock) {
+        connection = sharedConn;
         if (connection == null) {
-          connection = sharedHtableCon = ConnectionFactory.createConnection(conf);
+          connection = ConnectionFactory.createConnection(conf);
+          sharedConn = connection;
         }
       }
     }
@@ -441,9 +438,10 @@ public class ReplicationSink {
    * of the last edit that was applied
    */
   public String getStats() {
-    return this.totalReplicatedEdits.get() == 0 ? "" : "Sink: " +
-      "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
-      ", total replicated edits: " + this.totalReplicatedEdits;
+    long total = this.totalReplicatedEdits.get();
+    return total == 0 ? ""
+        : "Sink: " + "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() +
+          ", total replicated edits: " + total;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java
index 92f2c52..66fe3be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.util.Optional;
+import java.util.function.BiPredicate;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Pair;
@@ -36,8 +37,11 @@ public interface SyncReplicationPeerInfoProvider {
   Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info);
 
   /**
-   * Check whether the give region is contained in a sync replication peer which is in the given
-   * state.
+   * Check whether the give region is contained in a sync replication peer which can pass the state
+   * checker.
+   * <p>
+   * Will call the checker with current sync replication state and new sync replication state.
    */
-  boolean isInState(RegionInfo info, SyncReplicationState state);
+  boolean checkState(RegionInfo info,
+      BiPredicate<SyncReplicationState, SyncReplicationState> checker);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
index 32159e6..973e049 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java
@@ -18,8 +18,9 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.util.Optional;
+import java.util.function.BiPredicate;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.replication.ReplicationPeer;
+import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Pair;
@@ -44,11 +45,14 @@ class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProv
     if (peerId == null) {
       return Optional.empty();
     }
-    ReplicationPeer peer = replicationPeers.getPeer(peerId);
+    ReplicationPeerImpl peer = replicationPeers.getPeer(peerId);
     if (peer == null) {
       return Optional.empty();
     }
-    if (peer.getSyncReplicationState() == SyncReplicationState.ACTIVE) {
+    Pair<SyncReplicationState, SyncReplicationState> states =
+        peer.getSyncReplicationStateAndNewState();
+    if (states.getFirst() == SyncReplicationState.ACTIVE &&
+      states.getSecond() == SyncReplicationState.NONE) {
       return Optional.of(Pair.newPair(peerId, peer.getPeerConfig().getRemoteWALDir()));
     } else {
       return Optional.empty();
@@ -56,16 +60,19 @@ class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProv
   }
 
   @Override
-  public boolean isInState(RegionInfo info, SyncReplicationState state) {
+  public boolean checkState(RegionInfo info,
+      BiPredicate<SyncReplicationState, SyncReplicationState> checker) {
     String peerId = mapping.getPeerId(info);
     if (peerId == null) {
       return false;
     }
-    ReplicationPeer peer = replicationPeers.getPeer(peerId);
+    ReplicationPeerImpl peer = replicationPeers.getPeer(peerId);
     if (peer == null) {
       return false;
     }
-    return peer.getSyncReplicationState() == state;
+    Pair<SyncReplicationState, SyncReplicationState> states =
+        peer.getSyncReplicationStateAndNewState();
+    return checker.test(states.getFirst(), states.getSecond());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/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
index e3de6b4..ac4b4cd 100644
--- 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
@@ -141,6 +141,9 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen
 
   @Override
   public WAL getWAL(RegionInfo region) throws IOException {
+    if (region == null) {
+      return provider.getWAL(region);
+    }
     Optional<Pair<String, String>> peerIdAndRemoteWALDir =
       peerInfoProvider.getPeerIdAndRemoteWALDir(region);
     if (peerIdAndRemoteWALDir.isPresent()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/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 78355a1..2e43eb4 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
@@ -24,10 +24,10 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -162,7 +162,7 @@ public class WALFactory {
     // end required early initialization
     if (conf.getBoolean("hbase.regionserver.hlog.enabled", true)) {
       WALProvider provider = createProvider(getProviderClass(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
-      if (conf.getBoolean(HConstants.SYNC_REPLICATION_ENABLED, false)) {
+      if (conf.getBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, false)) {
         provider = new SyncReplicationWALProvider(provider);
       }
       provider.init(this, conf, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
new file mode 100644
index 0000000..acddc4a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSyncReplication {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestSyncReplication.class);
+
+  private static final HBaseZKTestingUtility ZK_UTIL = new HBaseZKTestingUtility();
+
+  private static final HBaseTestingUtility UTIL1 = new HBaseTestingUtility();
+
+  private static final HBaseTestingUtility UTIL2 = new HBaseTestingUtility();
+
+  private static TableName TABLE_NAME = TableName.valueOf("SyncRep");
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  private static byte[] CQ = Bytes.toBytes("cq");
+
+  private static String PEER_ID = "1";
+
+  private static void initTestingUtility(HBaseTestingUtility util, String zkParent) {
+    util.setZkCluster(ZK_UTIL.getZkCluster());
+    Configuration conf = util.getConfiguration();
+    conf.setBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, true);
+    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkParent);
+    conf.setInt("replication.source.size.capacity", 102400);
+    conf.setLong("replication.source.sleepforretries", 100);
+    conf.setInt("hbase.regionserver.maxlogs", 10);
+    conf.setLong("hbase.master.logcleaner.ttl", 10);
+    conf.setInt("zookeeper.recovery.retry", 1);
+    conf.setInt("zookeeper.recovery.retry.intervalmill", 10);
+    conf.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
+    conf.setInt("replication.stats.thread.period.seconds", 5);
+    conf.setBoolean("hbase.tests.use.shortcircuit.reads", false);
+    conf.setLong("replication.sleep.before.failover", 2000);
+    conf.setInt("replication.source.maxretriesmultiplier", 10);
+    conf.setFloat("replication.source.ratio", 1.0f);
+    conf.setBoolean("replication.source.eof.autorecovery", true);
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    ZK_UTIL.startMiniZKCluster();
+    initTestingUtility(UTIL1, "/cluster1");
+    initTestingUtility(UTIL2, "/cluster2");
+    UTIL1.startMiniCluster(3);
+    UTIL2.startMiniCluster(3);
+    TableDescriptor td =
+        TableDescriptorBuilder.newBuilder(TABLE_NAME).addColumnFamily(ColumnFamilyDescriptorBuilder
+          .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build();
+    UTIL1.getAdmin().createTable(td);
+    UTIL2.getAdmin().createTable(td);
+    FileSystem fs1 = UTIL1.getTestFileSystem();
+    FileSystem fs2 = UTIL2.getTestFileSystem();
+    Path remoteWALDir1 =
+        new Path(UTIL1.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
+          "remoteWALs").makeQualified(fs1.getUri(), fs1.getWorkingDirectory());
+    Path remoteWALDir2 =
+        new Path(UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem().getRootDir(),
+          "remoteWALs").makeQualified(fs2.getUri(), fs2.getWorkingDirectory());
+    UTIL1.getAdmin().addReplicationPeer(PEER_ID,
+      ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey())
+        .setReplicateAllUserTables(false)
+        .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>()))
+        .setRemoteWALDir(remoteWALDir2.toUri().toString()).build());
+    UTIL2.getAdmin().addReplicationPeer(PEER_ID,
+      ReplicationPeerConfig.newBuilder().setClusterKey(UTIL1.getClusterKey())
+        .setReplicateAllUserTables(false)
+        .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>()))
+        .setRemoteWALDir(remoteWALDir1.toUri().toString()).build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL1.shutdownMiniCluster();
+    UTIL2.shutdownMiniCluster();
+    ZK_UTIL.shutdownMiniZKCluster();
+  }
+
+  @FunctionalInterface
+  private interface TableAction {
+
+    void call(Table table) throws IOException;
+  }
+
+  private void assertDisallow(Table table, TableAction action) throws IOException {
+    try {
+      action.call(table);
+    } catch (DoNotRetryIOException | RetriesExhaustedException e) {
+      // expected
+      assertThat(e.getMessage(), containsString("STANDBY"));
+    }
+  }
+
+  @Test
+  public void testStandby() throws Exception {
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
+      assertDisallow(table, t -> t.get(new Get(Bytes.toBytes("row"))));
+      assertDisallow(table,
+        t -> t.put(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row"))));
+      assertDisallow(table, t -> t.delete(new Delete(Bytes.toBytes("row"))));
+      assertDisallow(table, t -> t.incrementColumnValue(Bytes.toBytes("row"), CF, CQ, 1));
+      assertDisallow(table,
+        t -> t.append(new Append(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row"))));
+      assertDisallow(table,
+        t -> t.get(Arrays.asList(new Get(Bytes.toBytes("row")), new Get(Bytes.toBytes("row1")))));
+      assertDisallow(table,
+        t -> t
+          .put(Arrays.asList(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")),
+            new Put(Bytes.toBytes("row1")).addColumn(CF, CQ, Bytes.toBytes("row1")))));
+      assertDisallow(table, t -> t.mutateRow(new RowMutations(Bytes.toBytes("row"))
+        .add((Mutation) new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")))));
+    }
+    // But we should still allow replication writes
+    try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    // The reject check is in RSRpcService so we can still read through HRegion
+    HRegion region = UTIL2.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    UTIL2.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return !region.get(new Get(Bytes.toBytes(99))).isEmpty();
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Replication has not been catched up yet";
+      }
+    });
+    for (int i = 0; i < 100; i++) {
+      assertEquals(i, Bytes.toInt(region.get(new Get(Bytes.toBytes(i))).getValue(CF, CQ)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca94b153/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 986228c..488d9fb 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
@@ -24,10 +24,10 @@ import static org.junit.Assert.assertThat;
 
 import java.io.IOException;
 import java.util.Optional;
+import java.util.function.BiPredicate;
 import org.apache.hadoop.fs.Path;
 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.client.RegionInfo;
@@ -36,6 +36,7 @@ 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.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -84,7 +85,8 @@ public class TestSyncReplicationWALProvider {
     }
 
     @Override
-    public boolean isInState(RegionInfo info, SyncReplicationState state) {
+    public boolean checkState(RegionInfo info,
+        BiPredicate<SyncReplicationState, SyncReplicationState> checker) {
       // TODO Implement SyncReplicationPeerInfoProvider.isInState
       return false;
     }
@@ -92,7 +94,7 @@ public class TestSyncReplicationWALProvider {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    UTIL.getConfiguration().setBoolean(HConstants.SYNC_REPLICATION_ENABLED, true);
+    UTIL.getConfiguration().setBoolean(ReplicationUtils.SYNC_REPLICATION_ENABLED, true);
     UTIL.startMiniDFSCluster(3);
     FACTORY = new WALFactory(UTIL.getConfiguration(), "test");
     ((SyncReplicationWALProvider) FACTORY.getWALProvider()).setPeerInfoProvider(new InfoProvider());


[10/36] hbase git commit: HBASE-20458 Support removing a WAL from LogRoller

Posted by zh...@apache.org.
HBASE-20458 Support removing a WAL from LogRoller


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

Branch: refs/heads/HBASE-19064
Commit: 3e2cc7370c4d1c077f4577eb6886f0aeacda387e
Parents: 90a0bb2
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Apr 23 16:31:54 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/LogRoller.java    | 29 +++++--
 .../hbase/regionserver/wal/AbstractFSWAL.java   |  7 +-
 .../regionserver/wal/WALClosedException.java    | 47 ++++++++++
 .../hbase/regionserver/TestLogRoller.java       | 90 ++++++++++++++++++++
 .../regionserver/wal/AbstractTestFSWAL.java     |  9 ++
 5 files changed, 171 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3e2cc737/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 55c5219..ab0083f 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -30,6 +32,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.regionserver.wal.WALClosedException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -177,17 +180,24 @@ public class LogRoller extends HasThread implements Closeable {
       rollLock.lock(); // FindBugs UL_UNRELEASED_LOCK_EXCEPTION_PATH
       try {
         this.lastrolltime = now;
-        for (Entry<WAL, Boolean> entry : walNeedsRoll.entrySet()) {
+        for (Iterator<Entry<WAL, Boolean>> iter = walNeedsRoll.entrySet().iterator(); iter
+            .hasNext();) {
+          Entry<WAL, Boolean> entry = iter.next();
           final WAL wal = entry.getKey();
           // Force the roll if the logroll.period is elapsed or if a roll was requested.
           // The returned value is an array of actual region names.
-          final byte [][] regionsToFlush = wal.rollWriter(periodic ||
-              entry.getValue().booleanValue());
-          walNeedsRoll.put(wal, Boolean.FALSE);
-          if (regionsToFlush != null) {
-            for (byte[] r : regionsToFlush) {
-              scheduleFlush(r);
+          try {
+            final byte[][] regionsToFlush =
+                wal.rollWriter(periodic || entry.getValue().booleanValue());
+            walNeedsRoll.put(wal, Boolean.FALSE);
+            if (regionsToFlush != null) {
+              for (byte[] r : regionsToFlush) {
+                scheduleFlush(r);
+              }
             }
+          } catch (WALClosedException e) {
+            LOG.warn("WAL has been closed. Skipping rolling of writer and just remove it", e);
+            iter.remove();
           }
         }
       } catch (FailedLogCloseException e) {
@@ -252,4 +262,9 @@ public class LogRoller extends HasThread implements Closeable {
     running = false;
     interrupt();
   }
+
+  @VisibleForTesting
+  Map<WAL, Boolean> getWalNeedsRoll() {
+    return this.walNeedsRoll;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3e2cc737/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 4255086..72ad8b8 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
@@ -754,15 +754,14 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
     rollWriterLock.lock();
     try {
+      if (this.closed) {
+        throw new WALClosedException("WAL has been closed");
+      }
       // Return if nothing to flush.
       if (!force && this.writer != null && this.numEntries.get() <= 0) {
         return null;
       }
       byte[][] regionsToFlush = null;
-      if (this.closed) {
-        LOG.debug("WAL closed. Skipping rolling of writer");
-        return regionsToFlush;
-      }
       try (TraceScope scope = TraceUtil.createTrace("FSHLog.rollWriter")) {
         Path oldPath = getOldPath();
         Path newPath = getNewPath();

http://git-wip-us.apache.org/repos/asf/hbase/blob/3e2cc737/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALClosedException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALClosedException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALClosedException.java
new file mode 100644
index 0000000..ac6aad0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALClosedException.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed 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 org.apache.hadoop.hbase.regionserver.LogRoller;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Thrown when {@link LogRoller} try to roll writer but the WAL already was closed. This may
+ * happened when peer's sync replication state was transited from
+ * {@link SyncReplicationState#ACTIVE} to {@link SyncReplicationState#DOWNGRADE_ACTIVE} and the
+ * region's WAL was changed to a new one. But the old WAL was still left in {@link LogRoller}.
+ */
+@InterfaceAudience.Private
+public class WALClosedException extends IOException {
+
+  private static final long serialVersionUID = -3183198896865290678L;
+
+  public WALClosedException() {
+    super();
+  }
+
+  /**
+   * @param msg exception message
+   */
+  public WALClosedException(String msg) {
+    super(msg);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3e2cc737/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java
new file mode 100644
index 0000000..e8c9423
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed 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.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({RegionServerTests.class, MediumTests.class})
+public class TestLogRoller {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestLogRoller.class);
+
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static final int logRollPeriod = 20 * 1000;
+
+  @Before
+  public void setup() throws Exception {
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.period", logRollPeriod);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testRemoveClosedWAL() throws Exception {
+    HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
+    Configuration conf = rs.getConfiguration();
+    LogRoller logRoller = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0).getWalRoller();
+    int originalSize = logRoller.getWalNeedsRoll().size();
+    FSHLog wal1 = new FSHLog(rs.getWALFileSystem(), rs.getWALRootDir(),
+        AbstractFSWALProvider.getWALDirectoryName(rs.getServerName().getServerName()), conf);
+    logRoller.addWAL(wal1);
+    FSHLog wal2 = new FSHLog(rs.getWALFileSystem(), rs.getWALRootDir(),
+      AbstractFSWALProvider.getWALDirectoryName(rs.getServerName().getServerName()), conf);
+    logRoller.addWAL(wal2);
+    FSHLog wal3 = new FSHLog(rs.getWALFileSystem(), rs.getWALRootDir(),
+      AbstractFSWALProvider.getWALDirectoryName(rs.getServerName().getServerName()), conf);
+    logRoller.addWAL(wal3);
+
+    assertEquals(originalSize + 3, logRoller.getWalNeedsRoll().size());
+    assertTrue(logRoller.getWalNeedsRoll().containsKey(wal1));
+
+    wal1.close();
+    Thread.sleep(2 * logRollPeriod);
+
+    assertEquals(originalSize + 2, logRoller.getWalNeedsRoll().size());
+    assertFalse(logRoller.getWalNeedsRoll().containsKey(wal1));
+
+    wal2.close();
+    wal3.close();
+    Thread.sleep(2 * logRollPeriod);
+
+    assertEquals(originalSize, logRoller.getWalNeedsRoll().size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3e2cc737/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
index 5336963..aa0e6b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
@@ -473,4 +473,13 @@ public abstract class AbstractTestFSWAL {
       assertNull(key.getWriteEntry());
     }
   }
+
+  @Test(expected = WALClosedException.class)
+  public void testRollWriterForClosedWAL() throws IOException {
+    String testName = currentTest.getMethodName();
+    AbstractFSWAL<?> wal = newWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(), testName,
+      CONF, null, true, null, null);
+    wal.close();
+    wal.rollWriter();
+  }
 }


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

Branch: refs/heads/HBASE-19064
Commit: 2c4c1a81e531480426076cce3648d2a0475cc424
Parents: 30efc33
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Jan 13 18:55:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfigUtil.java  |  6 ++
 .../replication/ReplicationPeerConfig.java      | 20 ++++-
 .../ReplicationPeerConfigBuilder.java           |  7 ++
 .../src/main/protobuf/Replication.proto         |  1 +
 .../replication/ReplicationPeerManager.java     | 15 ++++
 .../replication/TestReplicationAdmin.java       | 77 ++++++++++++++++++++
 .../src/main/ruby/hbase/replication_admin.rb    | 14 ++--
 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, 186 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2c4c1a81/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 b1c1713..474ded3 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
@@ -319,6 +319,9 @@ public final class ReplicationPeerConfigUtil {
         excludeNamespacesList.stream().map(ByteString::toStringUtf8).collect(Collectors.toSet()));
     }
 
+    if (peer.hasRemoteWALDir()) {
+      builder.setRemoteWALDir(peer.getRemoteWALDir());
+    }
     return builder.build();
   }
 
@@ -376,6 +379,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/2c4c1a81/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 e0d9a4c..97abc74 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
@@ -47,6 +47,8 @@ public class ReplicationPeerConfig {
   private Set<String> excludeNamespaces = null;
   private long bandwidth = 0;
   private final boolean serial;
+  // Used by synchronous replication
+  private String remoteWALDir;
 
   private ReplicationPeerConfig(ReplicationPeerConfigBuilderImpl builder) {
     this.clusterKey = builder.clusterKey;
@@ -66,6 +68,7 @@ public class ReplicationPeerConfig {
             : null;
     this.bandwidth = builder.bandwidth;
     this.serial = builder.serial;
+    this.remoteWALDir = builder.remoteWALDir;
   }
 
   private Map<TableName, List<String>>
@@ -213,6 +216,10 @@ public class ReplicationPeerConfig {
     return this;
   }
 
+  public String getRemoteWALDir() {
+    return this.remoteWALDir;
+  }
+
   public static ReplicationPeerConfigBuilder newBuilder() {
     return new ReplicationPeerConfigBuilderImpl();
   }
@@ -230,7 +237,8 @@ public class ReplicationPeerConfig {
       .setReplicateAllUserTables(peerConfig.replicateAllUserTables())
       .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap())
       .setExcludeNamespaces(peerConfig.getExcludeNamespaces())
-      .setBandwidth(peerConfig.getBandwidth()).setSerial(peerConfig.isSerial());
+      .setBandwidth(peerConfig.getBandwidth()).setSerial(peerConfig.isSerial())
+      .setRemoteWALDir(peerConfig.getRemoteWALDir());
     return builder;
   }
 
@@ -259,6 +267,8 @@ public class ReplicationPeerConfig {
 
     private boolean serial = false;
 
+    private String remoteWALDir = null;
+
     @Override
     public ReplicationPeerConfigBuilder setClusterKey(String clusterKey) {
       this.clusterKey = clusterKey;
@@ -327,6 +337,11 @@ public class ReplicationPeerConfig {
       return this;
     }
 
+    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
@@ -357,6 +372,9 @@ public class ReplicationPeerConfig {
     }
     builder.append("bandwidth=").append(bandwidth).append(",");
     builder.append("serial=").append(serial);
+    if (this.remoteWALDir != null) {
+      builder.append(",remoteWALDir=").append(remoteWALDir);
+    }
     return builder.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c4c1a81/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 4c531c5..58ff220 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
@@ -150,6 +150,13 @@ public interface ReplicationPeerConfigBuilder {
   ReplicationPeerConfigBuilder setSerial(boolean serial);
 
   /**
+   * 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/2c4c1a81/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 557b87c..20dd049 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -49,6 +49,7 @@ message ReplicationPeer {
   repeated TableCF exclude_table_cfs = 9;
   repeated bytes exclude_namespaces = 10;
   optional bool serial = 11;
+  optional string remoteWALDir = 12;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c4c1a81/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 87d0111..05ecd61 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
@@ -149,6 +149,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.isNamespacesAndTableCFsEqual(oldPeerConfig, peerConfig)) {
+        throw new DoNotRetryIOException(
+          "Changing the replicated namespace/table config on a synchronous replication " +
+            "peer(peerId: " + peerId + ") is not allowed.");
+      }
+    }
     return desc;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c4c1a81/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/2c4c1a81/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 5b87595..d1f1344 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

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c4c1a81/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/2c4c1a81/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/2c4c1a81/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 eefcc42..f3ab749 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
                             SERIAL])
 
         peers.each do |peer|
@@ -53,8 +54,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, config.isSerial])
         end

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c4c1a81/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 29de710..5d04fbb 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"
 


[33/36] hbase git commit: HBASE-20426 Give up replicating anything in S state

Posted by zh...@apache.org.
HBASE-20426 Give up replicating anything in S state


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

Branch: refs/heads/HBASE-19064
Commit: c87a5481a547644cd253334517526a191cd59d14
Parents: f1e81b9
Author: zhangduo <zh...@apache.org>
Authored: Thu May 3 15:51:35 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu May 31 14:31:01 2018 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/MasterProcedure.proto     |  13 +-
 .../replication/AbstractPeerProcedure.java      |   4 +
 .../master/replication/ModifyPeerProcedure.java |   6 -
 .../replication/ReplicationPeerManager.java     |  13 +-
 ...ransitPeerSyncReplicationStateProcedure.java |  94 +++++++++++----
 .../hadoop/hbase/regionserver/LogRoller.java    |  11 +-
 .../regionserver/PeerProcedureHandlerImpl.java  |  63 ++++++++--
 .../regionserver/ReplicationSource.java         |   1 +
 .../regionserver/ReplicationSourceManager.java  | 118 ++++++++++++++++---
 .../TestDrainReplicationQueuesForStandBy.java   | 118 +++++++++++++++++++
 10 files changed, 379 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/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 d58608a..f58ad2e 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -394,11 +394,14 @@ enum PeerSyncReplicationStateTransitionState {
   SET_PEER_NEW_SYNC_REPLICATION_STATE = 2;
   REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_BEGIN = 3;
   REPLAY_REMOTE_WAL_IN_PEER = 4;
-  REOPEN_ALL_REGIONS_IN_PEER = 5;
-  TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE = 6;
-  REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END = 7;
-  CREATE_DIR_FOR_REMOTE_WAL = 8;
-  POST_PEER_SYNC_REPLICATION_STATE_TRANSITION = 9;
+  REMOVE_ALL_REPLICATION_QUEUES_IN_PEER = 5;
+  REOPEN_ALL_REGIONS_IN_PEER = 6;
+  TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE = 7;
+  REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END = 8;
+  SYNC_REPLICATION_SET_PEER_ENABLED = 9;
+  SYNC_REPLICATION_ENABLE_PEER_REFRESH_PEER_ON_RS = 10;
+  CREATE_DIR_FOR_REMOTE_WAL = 11;
+  POST_PEER_SYNC_REPLICATION_STATE_TRANSITION = 12;
 }
 
 message PeerModificationStateData {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
index 6679d78..458e073 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
@@ -106,4 +106,8 @@ public abstract class AbstractPeerProcedure<TState>
     throw new UnsupportedOperationException();
   }
 
+  protected final void refreshPeer(MasterProcedureEnv env, PeerOperationType type) {
+    addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
+      .map(sn -> new RefreshPeerProcedure(peerId, type, sn)).toArray(RefreshPeerProcedure[]::new));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index fc559b0..ad4df61 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -109,12 +109,6 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
     throw new UnsupportedOperationException();
   }
 
-  private void refreshPeer(MasterProcedureEnv env, PeerOperationType type) {
-    addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
-      .map(sn -> new RefreshPeerProcedure(peerId, type, sn))
-      .toArray(RefreshPeerProcedure[]::new));
-  }
-
   protected ReplicationPeerConfig getOldPeerConfig() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/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 229549e..e1d8b51 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
@@ -45,6 +45,7 @@ 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.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -192,9 +193,9 @@ public class ReplicationPeerManager {
   }
 
   /**
-   * @return the old state.
+   * @return the old state, and whether the peer is enabled.
    */
-  public SyncReplicationState preTransitPeerSyncReplicationState(String peerId,
+  Pair<SyncReplicationState, Boolean> preTransitPeerSyncReplicationState(String peerId,
       SyncReplicationState state) throws DoNotRetryIOException {
     ReplicationPeerDescription desc = checkPeerExists(peerId);
     SyncReplicationState fromState = desc.getSyncReplicationState();
@@ -203,7 +204,7 @@ public class ReplicationPeerManager {
       throw new DoNotRetryIOException("Can not transit current cluster state from " + fromState +
         " to " + state + " for peer id=" + peerId);
     }
-    return fromState;
+    return Pair.newPair(fromState, desc.isEnabled());
   }
 
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
@@ -303,7 +304,7 @@ public class ReplicationPeerManager {
     }
   }
 
-  public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
+  public void removeAllQueues(String peerId) throws ReplicationException {
     // Here we need two passes to address the problem of claimQueue. Maybe a claimQueue is still
     // on-going when the refresh peer config procedure is done, if a RS which has already been
     // scanned claims the queue of a RS which has not been scanned yet, we will miss that queue in
@@ -317,6 +318,10 @@ public class ReplicationPeerManager {
     // unless it has already been removed by others.
     ReplicationUtils.removeAllQueues(queueStorage, peerId);
     ReplicationUtils.removeAllQueues(queueStorage, peerId);
+  }
+
+  public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
+    removeAllQueues(peerId);
     queueStorage.removePeerFromHFileRefs(peerId);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/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 99fd615..0175296 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
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -54,6 +55,8 @@ public class TransitPeerSyncReplicationStateProcedure
 
   private SyncReplicationState toState;
 
+  private boolean enabled;
+
   public TransitPeerSyncReplicationStateProcedure() {
   }
 
@@ -110,7 +113,10 @@ public class TransitPeerSyncReplicationStateProcedure
     if (cpHost != null) {
       cpHost.preTransitReplicationPeerSyncReplicationState(peerId, toState);
     }
-    fromState = env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, toState);
+    Pair<SyncReplicationState, Boolean> pair =
+      env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, toState);
+    fromState = pair.getFirst();
+    enabled = pair.getSecond();
   }
 
   private void postTransit(MasterProcedureEnv env) throws IOException {
@@ -131,6 +137,21 @@ public class TransitPeerSyncReplicationStateProcedure
       .collect(Collectors.toList());
   }
 
+  private void createDirForRemoteWAL(MasterProcedureEnv env)
+      throws ProcedureYieldException, IOException {
+    MasterFileSystem mfs = env.getMasterFileSystem();
+    Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
+    Path remoteWALDirForPeer = ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId);
+    FileSystem walFs = mfs.getWALFileSystem();
+    if (walFs.exists(remoteWALDirForPeer)) {
+      LOG.warn("Wal dir {} already exists, usually this should not happen, continue anyway",
+        remoteWALDirForPeer);
+    } else if (!walFs.mkdirs(remoteWALDirForPeer)) {
+      LOG.warn("Can not create remote wal dir {}", remoteWALDirForPeer);
+      throw new ProcedureYieldException();
+    }
+  }
+
   @Override
   protected Flow executeFromState(MasterProcedureEnv env,
       PeerSyncReplicationStateTransitionState state)
@@ -151,6 +172,13 @@ public class TransitPeerSyncReplicationStateProcedure
       case SET_PEER_NEW_SYNC_REPLICATION_STATE:
         try {
           env.getReplicationPeerManager().setPeerNewSyncReplicationState(peerId, toState);
+          if (toState.equals(SyncReplicationState.STANDBY) && enabled) {
+            // disable the peer if we are going to transit to STANDBY state, as we need to remove
+            // all the pending replication files. If we do not disable the peer and delete the wal
+            // queues on zk directly, RS will get NoNode exception when updating the wal position
+            // and crash.
+            env.getReplicationPeerManager().disablePeer(peerId);
+          }
         } catch (ReplicationException e) {
           LOG.warn("Failed to update peer storage for peer {} when starting transiting sync " +
             "replication peer state from {} to {}, retry", peerId, fromState, toState, e);
@@ -163,16 +191,35 @@ public class TransitPeerSyncReplicationStateProcedure
         addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
           .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 0))
           .toArray(RefreshPeerProcedure[]::new));
-        if (fromState == SyncReplicationState.STANDBY &&
-          toState == SyncReplicationState.DOWNGRADE_ACTIVE) {
-          setNextState(PeerSyncReplicationStateTransitionState.REPLAY_REMOTE_WAL_IN_PEER);
+        if (fromState.equals(SyncReplicationState.ACTIVE)) {
+          setNextState(toState.equals(SyncReplicationState.STANDBY)
+            ? PeerSyncReplicationStateTransitionState.REMOVE_ALL_REPLICATION_QUEUES_IN_PEER
+            : PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
+        } else if (fromState.equals(SyncReplicationState.DOWNGRADE_ACTIVE)) {
+          setNextState(toState.equals(SyncReplicationState.STANDBY)
+            ? PeerSyncReplicationStateTransitionState.REMOVE_ALL_REPLICATION_QUEUES_IN_PEER
+            : PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
         } else {
-          setNextState(PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
+          assert toState.equals(SyncReplicationState.DOWNGRADE_ACTIVE);
+          setNextState(PeerSyncReplicationStateTransitionState.REPLAY_REMOTE_WAL_IN_PEER);
         }
         return Flow.HAS_MORE_STATE;
       case REPLAY_REMOTE_WAL_IN_PEER:
         addChildProcedure(new RecoverStandbyProcedure(peerId));
-        setNextState(PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER);
+        setNextState(
+          PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE);
+        return Flow.HAS_MORE_STATE;
+      case REMOVE_ALL_REPLICATION_QUEUES_IN_PEER:
+        try {
+          env.getReplicationPeerManager().removeAllQueues(peerId);
+        } catch (ReplicationException e) {
+          LOG.warn("Failed to remove all replication queues peer {} when starting transiting" +
+            " sync replication peer state from {} to {}, retry", peerId, fromState, toState, e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(fromState.equals(SyncReplicationState.ACTIVE)
+          ? PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER
+          : PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE);
         return Flow.HAS_MORE_STATE;
       case REOPEN_ALL_REGIONS_IN_PEER:
         try {
@@ -202,27 +249,35 @@ public class TransitPeerSyncReplicationStateProcedure
           .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 1))
           .toArray(RefreshPeerProcedure[]::new));
         if (toState == SyncReplicationState.STANDBY) {
-          setNextState(PeerSyncReplicationStateTransitionState.CREATE_DIR_FOR_REMOTE_WAL);
+          setNextState(
+            enabled ? PeerSyncReplicationStateTransitionState.SYNC_REPLICATION_SET_PEER_ENABLED
+              : PeerSyncReplicationStateTransitionState.CREATE_DIR_FOR_REMOTE_WAL);
         } else {
           setNextState(
             PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION);
         }
         return Flow.HAS_MORE_STATE;
+      case SYNC_REPLICATION_SET_PEER_ENABLED:
+        try {
+          env.getReplicationPeerManager().enablePeer(peerId);
+        } catch (ReplicationException e) {
+          LOG.warn("Failed to set peer enabled for peer {} when transiting sync replication peer " +
+            "state from {} to {}, retry", peerId, fromState, toState, e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(
+          PeerSyncReplicationStateTransitionState.SYNC_REPLICATION_ENABLE_PEER_REFRESH_PEER_ON_RS);
+        return Flow.HAS_MORE_STATE;
+      case SYNC_REPLICATION_ENABLE_PEER_REFRESH_PEER_ON_RS:
+        refreshPeer(env, PeerOperationType.ENABLE);
+        setNextState(PeerSyncReplicationStateTransitionState.CREATE_DIR_FOR_REMOTE_WAL);
+        return Flow.HAS_MORE_STATE;
       case CREATE_DIR_FOR_REMOTE_WAL:
-        MasterFileSystem mfs = env.getMasterFileSystem();
-        Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
-        Path remoteWALDirForPeer = ReplicationUtils.getRemoteWALDirForPeer(remoteWALDir, peerId);
-        FileSystem walFs = mfs.getWALFileSystem();
         try {
-          if (walFs.exists(remoteWALDirForPeer)) {
-            LOG.warn("Wal dir {} already exists, usually this should not happen, continue anyway",
-              remoteWALDirForPeer);
-          } else if (!walFs.mkdirs(remoteWALDirForPeer)) {
-            LOG.warn("Can not create remote wal dir {}", remoteWALDirForPeer);
-            throw new ProcedureYieldException();
-          }
+          createDirForRemoteWAL(env);
         } catch (IOException e) {
-          LOG.warn("Failed to create remote wal dir {}", remoteWALDirForPeer, e);
+          LOG.warn("Failed to create remote wal dir for peer {} when transiting sync replication " +
+            "peer state from {} to {}, retry", peerId, fromState, toState, e);
           throw new ProcedureYieldException();
         }
         setNextState(
@@ -242,5 +297,4 @@ public class TransitPeerSyncReplicationStateProcedure
         throw new UnsupportedOperationException("unhandled state=" + state);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/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 ab0083f..05a8fdf 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
@@ -244,10 +244,8 @@ public class LogRoller extends HasThread implements Closeable {
   }
 
   /**
-   * For testing only
    * @return true if all WAL roll finished
    */
-  @VisibleForTesting
   public boolean walRollFinished() {
     for (boolean needRoll : walNeedsRoll.values()) {
       if (needRoll) {
@@ -257,6 +255,15 @@ public class LogRoller extends HasThread implements Closeable {
     return true;
   }
 
+  /**
+   * Wait until all wals have been rolled after calling {@link #requestRollAll()}.
+   */
+  public void waitUntilWalRollFinished() throws InterruptedException {
+    while (!walRollFinished()) {
+      Thread.sleep(100);
+    }
+  }
+
   @Override
   public void close() {
     running = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
index 7fc9f53..d01b130 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -18,8 +18,10 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.concurrent.locks.Lock;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.LogRoller;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -154,24 +156,65 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
       if (!peer.getPeerConfig().isSyncReplication()) {
         throw new ReplicationException("Peer with id=" + peerId + " is not synchronous.");
       }
-      SyncReplicationState newState = peer.getNewSyncReplicationState();
+      SyncReplicationState newSyncReplicationState = peer.getNewSyncReplicationState();
       if (stage == 0) {
-        if (newState != SyncReplicationState.NONE) {
+        if (newSyncReplicationState != SyncReplicationState.NONE) {
           LOG.warn("The new sync replication state for peer {} has already been set to {}, " +
-            "this should be a retry, give up", peerId, newState);
+            "this should be a retry, give up", peerId, newSyncReplicationState);
           return;
         }
-        newState = replicationPeers.refreshPeerNewSyncReplicationState(peerId);
-        SyncReplicationState oldState = peer.getSyncReplicationState();
-        peerActionListener.peerSyncReplicationStateChange(peerId, oldState, newState, stage);
+        // refresh the peer state first, as when we transit to STANDBY, we may need to disable the
+        // peer before processing the sync replication state.
+        PeerState oldState = peer.getPeerState();
+        boolean success = false;
+        try {
+          PeerState newState = replicationPeers.refreshPeerState(peerId);
+          if (oldState.equals(PeerState.ENABLED) && newState.equals(PeerState.DISABLED)) {
+            replicationSourceManager.refreshSources(peerId);
+          }
+          success = true;
+        } finally {
+          if (!success) {
+            peer.setPeerState(oldState.equals(PeerState.ENABLED));
+          }
+        }
+        newSyncReplicationState = replicationPeers.refreshPeerNewSyncReplicationState(peerId);
+        SyncReplicationState oldSyncReplicationState = peer.getSyncReplicationState();
+        peerActionListener.peerSyncReplicationStateChange(peerId, oldSyncReplicationState,
+          newSyncReplicationState, stage);
       } else {
-        if (newState == SyncReplicationState.NONE) {
-          LOG.warn("The new sync replication state for peer {} has already been clear, and the " +
-            "current state is {}, this should be a retry, give up", peerId, newState);
+        if (newSyncReplicationState == SyncReplicationState.NONE) {
+          LOG.warn(
+            "The new sync replication state for peer {} has already been clear, and the " +
+              "current state is {}, this should be a retry, give up",
+            peerId, newSyncReplicationState);
           return;
         }
+        if (newSyncReplicationState == SyncReplicationState.STANDBY) {
+          replicationSourceManager.drainSources(peerId);
+          // Need to roll the wals and make the ReplicationSource for this peer track the new file.
+          // If we do not do this, there will be two problems that can not be addressed at the same
+          // time. First, if we just throw away the current wal file, and later when we transit the
+          // peer to DA, and the wal has not been rolled yet, then the new data written to the wal
+          // file will not be replicated and cause data inconsistency. But if we just track the
+          // current wal file without rolling, it may contains some data before we transit the peer
+          // to S, later if we transit the peer to DA, the data will also be replicated and cause
+          // data inconsistency. So here we need to roll the wal, and let the ReplicationSource
+          // track the new wal file, and throw the old wal files away.
+          LogRoller roller = rs.getWalRoller();
+          roller.requestRollAll();
+          try {
+            roller.waitUntilWalRollFinished();
+          } catch (InterruptedException e) {
+            // reset the interrupted flag
+            Thread.currentThread().interrupt();
+            throw (IOException) new InterruptedIOException(
+              "Interrupted while waiting for wal roll finish").initCause(e);
+          }
+        }
         SyncReplicationState oldState = peer.getSyncReplicationState();
-        peerActionListener.peerSyncReplicationStateChange(peerId, oldState, newState, stage);
+        peerActionListener.peerSyncReplicationStateChange(peerId, oldState, newSyncReplicationState,
+          stage);
         peer.transitSyncReplicationState();
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index c669622..1cac0c5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -500,6 +500,7 @@ public class ReplicationSource implements ReplicationSourceInterface {
     if (this.replicationEndpoint != null) {
       this.replicationEndpoint.stop();
     }
+    metrics.clear();
     if (join) {
       for (ReplicationSourceShipper worker : workers) {
         Threads.shutdown(worker, this.sleepForRetries);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 5015129..f25b073 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
@@ -391,11 +392,83 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
+   * <p>
+   * This is used when we transit a sync replication peer to {@link SyncReplicationState#STANDBY}.
+   * </p>
+   * <p>
+   * When transiting to {@link SyncReplicationState#STANDBY}, we can remove all the pending wal
+   * files for a replication peer as we do not need to replicate them any more. And this is
+   * necessary, otherwise when we transit back to {@link SyncReplicationState#DOWNGRADE_ACTIVE}
+   * later, the stale data will be replicated again and cause inconsistency.
+   * </p>
+   * <p>
+   * See HBASE-20426 for more details.
+   * </p>
+   * @param peerId the id of the sync replication peer
+   */
+  public void drainSources(String peerId) throws IOException, ReplicationException {
+    String terminateMessage = "Sync replication peer " + peerId +
+      " is transiting to STANDBY. Will close the previous replication source and open a new one";
+    ReplicationPeer peer = replicationPeers.getPeer(peerId);
+    assert peer.getPeerConfig().isSyncReplication();
+    ReplicationSourceInterface src = createSource(peerId, peer);
+    // synchronized here to avoid race with preLogRoll where we add new log to source and also
+    // walsById.
+    ReplicationSourceInterface toRemove;
+    Map<String, NavigableSet<String>> wals = new HashMap<>();
+    synchronized (latestPaths) {
+      toRemove = sources.put(peerId, src);
+      if (toRemove != null) {
+        LOG.info("Terminate replication source for " + toRemove.getPeerId());
+        toRemove.terminate(terminateMessage);
+        toRemove.getSourceMetrics().clear();
+      }
+      // Here we make a copy of all the remaining wal files and then delete them from the
+      // replication queue storage after releasing the lock. It is not safe to just remove the old
+      // map from walsById since later we may fail to delete them from the replication queue
+      // storage, and when we retry next time, we can not know the wal files that need to be deleted
+      // from the replication queue storage.
+      walsById.get(peerId).forEach((k, v) -> wals.put(k, new TreeSet<>(v)));
+    }
+    LOG.info("Startup replication source for " + src.getPeerId());
+    src.startup();
+    for (NavigableSet<String> walsByGroup : wals.values()) {
+      for (String wal : walsByGroup) {
+        queueStorage.removeWAL(server.getServerName(), peerId, wal);
+      }
+    }
+    synchronized (walsById) {
+      Map<String, NavigableSet<String>> oldWals = walsById.get(peerId);
+      wals.forEach((k, v) -> {
+        NavigableSet<String> walsByGroup = oldWals.get(k);
+        if (walsByGroup != null) {
+          walsByGroup.removeAll(v);
+        }
+      });
+    }
+    // synchronized on oldsources to avoid race with NodeFailoverWorker. Since NodeFailoverWorker is
+    // a background task, we will delete the file from replication queue storage under the lock to
+    // simplify the logic.
+    synchronized (this.oldsources) {
+      for (Iterator<ReplicationSourceInterface> iter = oldsources.iterator(); iter.hasNext();) {
+        ReplicationSourceInterface oldSource = iter.next();
+        if (oldSource.getPeerId().equals(peerId)) {
+          String queueId = oldSource.getQueueId();
+          oldSource.terminate(terminateMessage);
+          oldSource.getSourceMetrics().clear();
+          queueStorage.removeQueue(server.getServerName(), queueId);
+          walsByIdRecoveredQueues.remove(queueId);
+          iter.remove();
+        }
+      }
+    }
+  }
+
+  /**
    * Close the previous replication sources of this peer id and open new sources to trigger the new
    * replication state changes or new replication config changes. Here we don't need to change
    * replication queue storage and only to enqueue all logs to the new replication source
    * @param peerId the id of the replication peer
-   * @throws IOException
    */
   public void refreshSources(String peerId) throws IOException {
     String terminateMessage = "Peer " + peerId +
@@ -409,7 +482,7 @@ public class ReplicationSourceManager implements ReplicationListener {
         LOG.info("Terminate replication source for " + toRemove.getPeerId());
         toRemove.terminate(terminateMessage);
       }
-      for (SortedSet<String> walsByGroup : walsById.get(peerId).values()) {
+      for (NavigableSet<String> walsByGroup : walsById.get(peerId).values()) {
         walsByGroup.forEach(wal -> src.enqueueLog(new Path(this.logDir, wal)));
       }
     }
@@ -830,18 +903,6 @@ public class ReplicationSourceManager implements ReplicationListener {
               actualPeerId);
             continue;
           }
-          // track sources in walsByIdRecoveredQueues
-          Map<String, NavigableSet<String>> walsByGroup = new HashMap<>();
-          walsByIdRecoveredQueues.put(queueId, walsByGroup);
-          for (String wal : walsSet) {
-            String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal);
-            NavigableSet<String> wals = walsByGroup.get(walPrefix);
-            if (wals == null) {
-              wals = new TreeSet<>();
-              walsByGroup.put(walPrefix, wals);
-            }
-            wals.add(wal);
-          }
 
           ReplicationSourceInterface src = createSource(queueId, peer);
           // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
@@ -849,9 +910,36 @@ public class ReplicationSourceManager implements ReplicationListener {
             peer = replicationPeers.getPeer(src.getPeerId());
             if (peer == null || !isOldPeer(src.getPeerId(), peer)) {
               src.terminate("Recovered queue doesn't belong to any current peer");
-              removeRecoveredSource(src);
+              deleteQueue(queueId);
               continue;
             }
+            // Do not setup recovered queue if a sync replication peer is in STANDBY state, or is
+            // transiting to STANDBY state. The only exception is we are in STANDBY state and
+            // transiting to DA, under this state we will replay the remote WAL and they need to be
+            // replicated back.
+            if (peer.getPeerConfig().isSyncReplication()) {
+              Pair<SyncReplicationState, SyncReplicationState> stateAndNewState =
+                peer.getSyncReplicationStateAndNewState();
+              if ((stateAndNewState.getFirst().equals(SyncReplicationState.STANDBY) &&
+                stateAndNewState.getSecond().equals(SyncReplicationState.NONE)) ||
+                stateAndNewState.getSecond().equals(SyncReplicationState.STANDBY)) {
+                src.terminate("Sync replication peer is in STANDBY state");
+                deleteQueue(queueId);
+                continue;
+              }
+            }
+            // track sources in walsByIdRecoveredQueues
+            Map<String, NavigableSet<String>> walsByGroup = new HashMap<>();
+            walsByIdRecoveredQueues.put(queueId, walsByGroup);
+            for (String wal : walsSet) {
+              String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal);
+              NavigableSet<String> wals = walsByGroup.get(walPrefix);
+              if (wals == null) {
+                wals = new TreeSet<>();
+                walsByGroup.put(walPrefix, wals);
+              }
+              wals.add(wal);
+            }
             oldsources.add(src);
             for (String wal : walsSet) {
               src.enqueueLog(new Path(oldLogDir, wal));

http://git-wip-us.apache.org/repos/asf/hbase/blob/c87a5481/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDrainReplicationQueuesForStandBy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDrainReplicationQueuesForStandBy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDrainReplicationQueuesForStandBy.java
new file mode 100644
index 0000000..5da7870
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDrainReplicationQueuesForStandBy.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.replication.SyncReplicationTestBase;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestDrainReplicationQueuesForStandBy extends SyncReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestDrainReplicationQueuesForStandBy.class);
+
+  @Test
+  public void test() throws Exception {
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.ACTIVE);
+    UTIL1.getAdmin().disableReplicationPeer(PEER_ID);
+    write(UTIL1, 0, 100);
+
+    HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME);
+    String walGroupId = AbstractFSWALProvider.getWALPrefixFromWALName(
+      ((AbstractFSWAL<?>) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build()))
+        .getCurrentFileName().getName());
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    // transit cluster2 to DA and cluster 1 to S
+    verify(UTIL2, 0, 100);
+
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    // delete the original value, and then major compact
+    try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
+      for (int i = 0; i < 100; i++) {
+        table.delete(new Delete(Bytes.toBytes(i)));
+      }
+    }
+    UTIL2.flush(TABLE_NAME);
+    UTIL2.compact(TABLE_NAME, true);
+    // wait until the new values are replicated back to cluster1
+    HRegion region = rs.getRegions(TABLE_NAME).get(0);
+    UTIL1.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return region.get(new Get(Bytes.toBytes(99))).isEmpty();
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Replication has not been catched up yet";
+      }
+    });
+    // transit cluster1 to DA and cluster2 to S, then we will start replicating from cluster1 to
+    // cluster2
+    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
+      SyncReplicationState.STANDBY);
+    UTIL1.getAdmin().enableReplicationPeer(PEER_ID);
+
+    // confirm that we will not replicate the old data which causes inconsistency
+    ReplicationSource source = (ReplicationSource) ((Replication) rs.getReplicationSourceService())
+      .getReplicationManager().getSource(PEER_ID);
+    UTIL1.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return !source.workerThreads.containsKey(walGroupId);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Replication has not been catched up yet";
+      }
+    });
+    HRegion region2 = UTIL2.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    for (int i = 0; i < 100; i++) {
+      assertTrue(region2.get(new Get(Bytes.toBytes(i))).isEmpty());
+    }
+  }
+}
\ No newline at end of file