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/04/11 07:27:13 UTC

[01/30] hbase git commit: HBASE-20361 Non-successive TableInputSplits may wrongly be merged by auto balancing feature [Forced Update!]

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


HBASE-20361 Non-successive TableInputSplits may wrongly be merged by auto balancing feature

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

Branch: refs/heads/HBASE-19064
Commit: c7593d14a937509ee5d6a4dcb8322651edd9c6fd
Parents: f663430
Author: Yuki Tawara <yk...@gmail.com>
Authored: Sun Apr 8 01:46:52 2018 +0900
Committer: tedyu <yu...@gmail.com>
Committed: Sun Apr 8 10:58:23 2018 -0700

----------------------------------------------------------------------
 .../hbase/mapreduce/TableInputFormatBase.java   |  12 +-
 .../mapreduce/TestTableInputFormatBase.java     | 232 +++++++++++++++++++
 2 files changed, 242 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c7593d14/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
index d8031d9..24973c9 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * A base for {@link TableInputFormat}s. Receives a {@link Connection}, a {@link TableName},
@@ -291,7 +292,7 @@ public abstract class TableInputFormatBase
    */
   private List<InputSplit> oneInputSplitPerRegion() throws IOException {
     RegionSizeCalculator sizeCalculator =
-        new RegionSizeCalculator(getRegionLocator(), getAdmin());
+        createRegionSizeCalculator(getRegionLocator(), getAdmin());
 
     TableName tableName = getTable().getName();
 
@@ -478,7 +479,8 @@ public abstract class TableInputFormatBase
         while (j < splits.size()) {
           TableSplit nextRegion = (TableSplit) splits.get(j);
           long nextRegionSize = nextRegion.getLength();
-          if (totalSize + nextRegionSize <= averageRegionSize) {
+          if (totalSize + nextRegionSize <= averageRegionSize
+              && Bytes.equals(splitEndKey, nextRegion.getStartRow())) {
             totalSize = totalSize + nextRegionSize;
             splitEndKey = nextRegion.getEndRow();
             j++;
@@ -586,6 +588,12 @@ public abstract class TableInputFormatBase
     this.connection = connection;
   }
 
+  @VisibleForTesting
+  protected RegionSizeCalculator createRegionSizeCalculator(RegionLocator locator, Admin admin)
+      throws IOException {
+    return new RegionSizeCalculator(locator, admin);
+  }
+
   /**
    * Gets the scan defining the actual details like columns etc.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/c7593d14/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
index 5fa4b54..29a92ee 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
@@ -18,15 +18,45 @@
 package org.apache.hadoop.hbase.mapreduce;
 
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyBoolean;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.net.Inet6Address;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutorService;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.BufferedMutatorParams;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+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.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableBuilder;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.mapreduce.JobContext;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @Category({SmallTests.class})
 public class TestTableInputFormatBase {
@@ -55,4 +85,206 @@ public class TestTableInputFormatBase {
     assertEquals("Should retrun the hostname for this host. Expected : " +
         localhost + " Actual : " + actualHostName, localhost, actualHostName);
   }
+
+  @Test
+  public void testNonSuccessiveSplitsAreNotMerged() throws IOException {
+    JobContext context = mock(JobContext.class);
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
+        ConnectionForMergeTesting.class.getName());
+    conf.set(TableInputFormat.INPUT_TABLE, "testTable");
+    conf.setBoolean(TableInputFormatBase.MAPREDUCE_INPUT_AUTOBALANCE, true);
+    when(context.getConfiguration()).thenReturn(conf);
+
+    TableInputFormat tifExclude = new TableInputFormatForMergeTesting();
+    tifExclude.setConf(conf);
+    // split["b", "c"] is excluded, split["o", "p"] and split["p", "q"] are merged,
+    // but split["a", "b"] and split["c", "d"] are not merged.
+    assertEquals(ConnectionForMergeTesting.START_KEYS.length - 1 - 1,
+        tifExclude.getSplits(context).size());
+  }
+
+  /**
+   * Subclass of {@link TableInputFormat} to use in {@link #testNonSuccessiveSplitsAreNotMerged}.
+   * This class overrides {@link TableInputFormatBase#includeRegionInSplit}
+   * to exclude specific splits.
+   */
+  private static class TableInputFormatForMergeTesting extends TableInputFormat {
+    private byte[] prefixStartKey = Bytes.toBytes("b");
+    private byte[] prefixEndKey = Bytes.toBytes("c");
+    private RegionSizeCalculator sizeCalculator;
+
+    /**
+     * Exclude regions which contain rows starting with "b".
+     */
+    @Override
+    protected boolean includeRegionInSplit(final byte[] startKey, final byte [] endKey) {
+      if (Bytes.compareTo(startKey, prefixEndKey) < 0
+          && (Bytes.compareTo(prefixStartKey, endKey) < 0
+              || Bytes.equals(endKey, HConstants.EMPTY_END_ROW))) {
+        return false;
+      } else {
+        return true;
+      }
+    }
+
+    @Override
+    protected void initializeTable(Connection connection, TableName tableName) throws IOException {
+      super.initializeTable(connection, tableName);
+      ConnectionForMergeTesting cft = (ConnectionForMergeTesting) connection;
+      sizeCalculator = cft.getRegionSizeCalculator();
+    }
+
+    @Override
+    protected RegionSizeCalculator createRegionSizeCalculator(RegionLocator locator, Admin admin)
+      throws IOException {
+      return sizeCalculator;
+    }
+  }
+
+  /**
+   * Connection class to use in {@link #testNonSuccessiveSplitsAreNotMerged}.
+   * This class returns mocked {@link Table}, {@link RegionLocator}, {@link RegionSizeCalculator},
+   * and {@link Admin}.
+   */
+  private static class ConnectionForMergeTesting implements Connection {
+    public static final byte[][] SPLITS = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"), Bytes.toBytes("d"),
+      Bytes.toBytes("e"), Bytes.toBytes("f"), Bytes.toBytes("g"), Bytes.toBytes("h"),
+      Bytes.toBytes("i"), Bytes.toBytes("j"), Bytes.toBytes("k"), Bytes.toBytes("l"),
+      Bytes.toBytes("m"), Bytes.toBytes("n"), Bytes.toBytes("o"), Bytes.toBytes("p"),
+      Bytes.toBytes("q"), Bytes.toBytes("r"), Bytes.toBytes("s"), Bytes.toBytes("t"),
+      Bytes.toBytes("u"), Bytes.toBytes("v"), Bytes.toBytes("w"), Bytes.toBytes("x"),
+      Bytes.toBytes("y"), Bytes.toBytes("z")
+    };
+
+    public static final byte[][] START_KEYS;
+    public static final byte[][] END_KEYS;
+    static {
+      START_KEYS = new byte[SPLITS.length + 1][];
+      START_KEYS[0] = HConstants.EMPTY_BYTE_ARRAY;
+      for (int i = 0; i < SPLITS.length; i++) {
+        START_KEYS[i + 1] = SPLITS[i];
+      }
+
+      END_KEYS = new byte[SPLITS.length + 1][];
+      for (int i = 0; i < SPLITS.length; i++) {
+        END_KEYS[i] = SPLITS[i];
+      }
+      END_KEYS[SPLITS.length] = HConstants.EMPTY_BYTE_ARRAY;
+    }
+
+    public static final Map<byte[], Long> SIZE_MAP = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    static {
+      for (byte[] startKey : START_KEYS) {
+        SIZE_MAP.put(startKey, 1024L * 1024L * 1024L);
+      }
+      SIZE_MAP.put(Bytes.toBytes("a"), 200L * 1024L * 1024L);
+      SIZE_MAP.put(Bytes.toBytes("b"), 200L * 1024L * 1024L);
+      SIZE_MAP.put(Bytes.toBytes("c"), 200L * 1024L * 1024L);
+      SIZE_MAP.put(Bytes.toBytes("o"), 200L * 1024L * 1024L);
+      SIZE_MAP.put(Bytes.toBytes("p"), 200L * 1024L * 1024L);
+    }
+
+    ConnectionForMergeTesting(Configuration conf, ExecutorService pool, User user)
+        throws IOException {
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+    @Override
+    public Configuration getConfiguration() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Table getTable(TableName tableName) throws IOException {
+      Table table = mock(Table.class);
+      when(table.getName()).thenReturn(tableName);
+      return table;
+    }
+
+    @Override
+    public Table getTable(TableName tableName, ExecutorService pool) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public RegionLocator getRegionLocator(TableName tableName) throws IOException {
+      final Map<byte[], HRegionLocation> locationMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+      for (byte[] startKey : START_KEYS) {
+        HRegionLocation hrl = new HRegionLocation(
+            RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).build(),
+            ServerName.valueOf("localhost", 0, 0));
+        locationMap.put(startKey, hrl);
+      }
+
+      RegionLocator locator = mock(RegionLocator.class);
+      when(locator.getRegionLocation(any(byte [].class), anyBoolean())).
+        thenAnswer(new Answer<HRegionLocation>() {
+          @Override
+          public HRegionLocation answer(InvocationOnMock invocationOnMock) throws Throwable {
+            Object [] args = invocationOnMock.getArguments();
+            byte [] key = (byte [])args[0];
+            return locationMap.get(key);
+          }
+        });
+      when(locator.getStartEndKeys()).
+        thenReturn(new Pair<byte[][], byte[][]>(START_KEYS, END_KEYS));
+      return locator;
+    }
+
+    public RegionSizeCalculator getRegionSizeCalculator() {
+      RegionSizeCalculator sizeCalculator = mock(RegionSizeCalculator.class);
+      when(sizeCalculator.getRegionSize(any(byte[].class))).
+        thenAnswer(new Answer<Long>() {
+          @Override
+          public Long answer(InvocationOnMock invocationOnMock) throws Throwable {
+            Object [] args = invocationOnMock.getArguments();
+            byte [] regionId = (byte [])args[0];
+            byte[] startKey = RegionInfo.getStartKey(regionId);
+            return SIZE_MAP.get(startKey);
+          }
+        });
+      return sizeCalculator;
+    }
+
+    @Override
+    public Admin getAdmin() throws IOException {
+      Admin admin = mock(Admin.class);
+      // return non-null admin to pass null checks
+      return admin;
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public boolean isClosed() {
+      return false;
+    }
+
+    @Override
+    public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) {
+      throw new UnsupportedOperationException();
+    }
+  }
 }


[26/30] 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/a3383847/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/a3383847/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/a3383847/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 2e228f5..a2c974a 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -390,6 +390,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/a3383847/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/a3383847/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/a3383847/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 }


[04/30] hbase git commit: HBASE-20365 add 2.0 docs to menu.

Posted by zh...@apache.org.
HBASE-20365 add 2.0 docs to menu.

Signed-off-by: Mike Drob <md...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: 116a8085178d9d5bc55ab48a7e48a3969a0fd787
Parents: ee87de9
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Apr 9 01:07:02 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Apr 9 12:01:12 2018 -0500

----------------------------------------------------------------------
 src/site/site.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/116a8085/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index b5436d4..ce7f8ae 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -110,6 +110,10 @@
       <item name="Bulk Loads" href="book.html#arch.bulk.load" target="_blank" />
       <item name="Metrics" href="metrics.html" target="_blank" />
       <item name="Cluster replication" href="book.html#replication" target="_blank" />
+      <item name="2.0 Documentation">
+        <item name="API" href="2.0/apidocs/index.html" target="_blank" />
+        <item name="Ref Guide (single-page)" href="2.0/book.html" target="_blank" />
+      </item>
       <item name="1.2 Documentation">
         <item name="API" href="1.2/apidocs/index.html" target="_blank" />
         <item name="X-Ref" href="1.2/xref/index.html" target="_blank" />


[11/30] hbase git commit: HBASE-20335 nightly job bash cleanup.

Posted by zh...@apache.org.
HBASE-20335 nightly job bash cleanup.

* Ensure Jenkins steps that invoke bash inline set -e
* machine stats script should check that passed directory will work

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: a2c1be9a7674e5d18892b72b2984a524c7dc64c9
Parents: 83ee82d
Author: Sean Busbey <bu...@apache.org>
Authored: Tue Apr 3 12:08:05 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Apr 11 01:33:14 2018 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile                   | 11 +++++++++++
 dev-support/gather_machine_environment.sh |  5 +++++
 2 files changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a2c1be9a/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index e8bf8c9..3f3066b 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -63,6 +63,7 @@ pipeline {
     stage ('yetus install') {
       steps {
         sh  '''#!/usr/bin/env bash
+set -e
 echo "Ensure we have a copy of Apache Yetus."
 if [[ true !=  "${USE_YETUS_PRERELEASE}" ]]; then
   YETUS_DIR="${WORKSPACE}/yetus-${YETUS_RELEASE}"
@@ -101,6 +102,7 @@ fi
         // Set up the file we need at PERSONALITY_FILE location
         dir ("tools") {
           sh """#!/usr/bin/env bash
+set -e
 echo "Downloading Project personality."
 curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
           """
@@ -144,12 +146,14 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
               checkout scm
             }
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "${OUTPUT_DIR}" && mkdir "${OUTPUT_DIR}"
               rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
               "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_DIR_RELATIVE}/machine"
 '''
             // TODO roll this into the hbase_nightly_yetus script
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "${OUTPUT_DIR}/commentfile}"
               declare -i status=0
               if "${BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
@@ -202,11 +206,13 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
               checkout scm
             }
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "${OUTPUT_DIR}" && mkdir "${OUTPUT_DIR}"
               rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
               "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_DIR_RELATIVE}/machine"
 '''
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "${OUTPUT_DIR}/commentfile}"
               declare -i status=0
               if "${BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
@@ -273,11 +279,13 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
               checkout scm
             }
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "${OUTPUT_DIR}" && mkdir "${OUTPUT_DIR}"
               rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
               "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_DIR_RELATIVE}/machine"
 '''
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "${OUTPUT_DIR}/commentfile}"
               declare -i status=0
               if "${BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
@@ -351,11 +359,13 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
               checkout scm
             }
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "${OUTPUT_DIR}" && mkdir "${OUTPUT_DIR}"
               rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
               "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_DIR_RELATIVE}/machine"
 '''
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "${OUTPUT_DIR}/commentfile}"
               declare -i status=0
               if "${BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
@@ -425,6 +435,7 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
               checkout scm
             }
             sh '''#!/usr/bin/env bash
+              set -e
               rm -rf "output-srctarball/machine" && mkdir "output-srctarball/machine"
               "${BASEDIR}/dev-support/gather_machine_environment.sh" "output-srctarball/machine"
 '''

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2c1be9a/dev-support/gather_machine_environment.sh
----------------------------------------------------------------------
diff --git a/dev-support/gather_machine_environment.sh b/dev-support/gather_machine_environment.sh
index 589dffe..fa99d3f 100755
--- a/dev-support/gather_machine_environment.sh
+++ b/dev-support/gather_machine_environment.sh
@@ -33,6 +33,11 @@ fi
 
 declare output=$1
 
+if [ ! -d "${output}" ] || [ ! -w "${output}" ]; then
+  echo "Specified output directory must exist and be writable." >&2
+  exit 1
+fi
+
 echo "getting machine specs, find in ${BUILD_URL}/artifact/${output}/"
 echo "JAVA_HOME: ${JAVA_HOME}" >"${output}/java_home" 2>&1 || true
 ls -l "${JAVA_HOME}" >"${output}/java_home_ls" 2>&1 || true


[07/30] hbase git commit: HBASE-20371 note hbasecon cfp on landing page.

Posted by zh...@apache.org.
HBASE-20371 note hbasecon cfp on landing page.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: 17f930c4d6cce0b36edd43a8e2ff02b5f146a7f8
Parents: a3d5bd5
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Apr 9 11:31:03 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Apr 9 12:19:07 2018 -0500

----------------------------------------------------------------------
 src/site/xdoc/index.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/17f930c4/src/site/xdoc/index.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/index.xml b/src/site/xdoc/index.xml
index b402e64..49cb6f7 100644
--- a/src/site/xdoc/index.xml
+++ b/src/site/xdoc/index.xml
@@ -83,7 +83,7 @@ Apache HBase is an open-source, distributed, versioned, non-relational database
 </section>
 
      <section name="News">
-       <p>June 18th, 2018 <a href="https://hbase.apache.org/hbasecon-2018">HBaseCon 2018</a> @ San Jose Convention Center, San Jose, CA, USA</p>
+       <p>June 18th, 2018 <a href="https://hbase.apache.org/hbasecon-2018">HBaseCon 2018</a> @ San Jose Convention Center, San Jose, CA, USA. CFP open, see site for details!</p>
        <p>August 4th, 2017 <a href="https://easychair.org/cfp/HBaseConAsia2017">HBaseCon Asia 2017</a> @ the Huawei Campus in Shenzhen, China</p>
        <p>June 12th, 2017 <a href="https://easychair.org/cfp/hbasecon2017">HBaseCon2017</a> at the Crittenden Buildings on the Google Mountain View Campus</p>
        <p>April 25th, 2017 <a href="https://www.meetup.com/hbaseusergroup/events/239291716/">Meetup</a> @ Visa in Palo Alto</p>


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

Branch: refs/heads/HBASE-19064
Commit: 1e7c341cbafb7ab80f48c39c59ca912211f35553
Parents: 4ced805
Author: zhangduo <zh...@apache.org>
Authored: Tue Apr 10 22:35:19 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:58:35 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/1e7c341c/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/1e7c341c/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/1e7c341c/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/1e7c341c/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 19b2ab1..98a736e 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,8 +20,10 @@
 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.hbase.Cell;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -30,7 +32,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;
@@ -163,4 +167,23 @@ public class WALUtil {
     }
     return walKey;
   }
+
+  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/1e7c341c/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/1e7c341c/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/1e7c341c/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/1e7c341c/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/1e7c341c/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/1e7c341c/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/1e7c341c/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 44cac1a..253017e 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
@@ -133,7 +133,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/1e7c341c/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/1e7c341c/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/1e7c341c/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/1e7c341c/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/1e7c341c/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/1e7c341c/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);
+  }
+}


[08/30] hbase git commit: HBASE-17918 document serial replication

Posted by zh...@apache.org.
HBASE-17918 document serial 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/93498ddc
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/93498ddc
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/93498ddc

Branch: refs/heads/HBASE-19064
Commit: 93498ddc599fe15f4357acf742a2dcb8b2c84b9c
Parents: 17f930c
Author: meiyi <me...@xiaomi.com>
Authored: Mon Apr 9 20:57:18 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Apr 10 09:44:32 2018 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/ops_mgt.adoc | 34 ++++++++++++++++++++++++++-
 1 file changed, 33 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/93498ddc/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 6d332af..816773e 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -1407,9 +1407,12 @@ If a slave cluster does run out of room, or is inaccessible for other reasons, i
 .Consistency Across Replicated Clusters
 [WARNING]
 ====
-How your application builds on top of the HBase API matters when replication is in play. HBase's replication system provides at-least-once delivery of client edits for an enabled column family to each configured destination cluster. In the event of failure to reach a given destination, the replication system will retry sending edits in a way that might repeat a given message. Further more, there is not a guaranteed order of delivery for client edits. In the event of a RegionServer failing, recovery of the replication queue happens independent of recovery of the individual regions that server was previously handling. This means that it is possible for the not-yet-replicated edits to be serviced by a RegionServer that is currently slower to replicate than the one that handles edits from after the failure.
+How your application builds on top of the HBase API matters when replication is in play. HBase's replication system provides at-least-once delivery of client edits for an enabled column family to each configured destination cluster. In the event of failure to reach a given destination, the replication system will retry sending edits in a way that might repeat a given message. HBase provides two ways of replication, one is the original replication and the other is serial replication. In the previous way of replication, there is not a guaranteed order of delivery for client edits. In the event of a RegionServer failing, recovery of the replication queue happens independent of recovery of the individual regions that server was previously handling. This means that it is possible for the not-yet-replicated edits to be serviced by a RegionServer that is currently slower to replicate than the one that handles edits from after the failure.
 
 The combination of these two properties (at-least-once delivery and the lack of message ordering) means that some destination clusters may end up in a different state if your application makes use of operations that are not idempotent, e.g. Increments.
+
+To solve the problem, HBase now supports serial replication, which sends edits to destination cluster as the order of requests from client. See <<Serial Replication,Serial Replication>>.
+
 ====
 
 .Terminology Changes
@@ -1450,6 +1453,9 @@ Instead of SQL statements, entire WALEdits (consisting of multiple cell inserts
 LOG.info("Replicating "+clusterId + " -> " + peerClusterId);
 ----
 
+.Serial Replication Configuration
+See <<Serial Replication,Serial Replication>>
+
 .Cluster Management Commands
 add_peer <ID> <CLUSTER_KEY>::
   Adds a replication relationship between two clusters. +
@@ -1471,6 +1477,32 @@ enable_table_replication <TABLE_NAME>::
 disable_table_replication <TABLE_NAME>::
   Disable the table replication switch for all its column families.
 
+=== Serial Replication
+
+Note: this feature is introduced in HBase 2.1
+
+.Function of serial replication
+
+Serial replication supports to push logs to the destination cluster in the same order as logs reach to the source cluster.
+
+.Why need serial replication?
+In replication of HBase, we push mutations to destination cluster by reading WAL in each region server. We have a queue for WAL files so we can read them in order of creation time. However, when region-move or RS failure occurs in source cluster, the hlog entries that are not pushed before region-move or RS-failure will be pushed by original RS(for region move) or another RS which takes over the remained hlog of dead RS(for RS failure), and the new entries for the same region(s) will be pushed by the RS which now serves the region(s), but they push the hlog entries of a same region concurrently without coordination.
+
+This treatment can possibly lead to data inconsistency between source and destination clusters:
+
+1. there are put and then delete written to source cluster.
+
+2. due to region-move / RS-failure, they are pushed by different replication-source threads to peer cluster.
+
+3. if delete is pushed to peer cluster before put, and flush and major-compact occurs in peer cluster before put is pushed to peer cluster, the delete is collected and the put remains in peer cluster, but in source cluster the put is masked by the delete, hence data inconsistency between source and destination clusters.
+
+
+.Serial replication configuration
+
+. Set the serial flag to true for a repliation peer. You can either set it to true when creating a replication peer, or change it to true later.
+
+The serial replication feature had been done firstly in link:https://issues.apache.org/jira/browse/HBASE-9465[HBASE-9465] and then reverted and redone in link:https://issues.apache.org/jira/browse/HBASE-20046[HBASE-20046]. You can find more details in these issues.
+
 === Verifying Replicated Data
 
 The `VerifyReplication` MapReduce job, which is included in HBase, performs a systematic comparison of replicated data between two different clusters. Run the VerifyReplication job on the master cluster, supplying it with the peer ID and table name to use for validation. You can limit the verification further by specifying a time range or specific families. The job's short name is `verifyrep`. To run the job, use a command like the following:


[12/30] hbase git commit: HBASE-20068 personality tests for Apache Yetus should use the maven plugin to exec maven.

Posted by zh...@apache.org.
HBASE-20068 personality tests for Apache Yetus should use the maven plugin to exec maven.

Signed-off-by: Josh Elser <el...@apache.org>
Signed-off-by: Reid Chan <re...@outlook.com>


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

Branch: refs/heads/HBASE-19064
Commit: 83ee82d6a02f2b4b0f8e490483fd7a7b0ace73d8
Parents: 1efd3ac
Author: Sean Busbey <bu...@apache.org>
Authored: Sun Apr 8 21:27:58 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Apr 11 01:33:14 2018 -0500

----------------------------------------------------------------------
 dev-support/hbase-personality.sh | 16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/83ee82d6/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index d34f761..90786f2 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -277,8 +277,10 @@ function refguide_rebuild
 
   start_clock
 
+  # disabled because "maven_executor" needs to return both command and args
+  # shellcheck disable=2046
   echo_and_redirect "${logfile}" \
-    "${MAVEN}" "${MAVEN_ARGS[@]}" clean site --batch-mode \
+    $(maven_executor) clean site --batch-mode \
       -pl . \
       -Dtest=NoUnitTests -DHBasePatchProcess -Prelease \
       -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dfindbugs.skip=true
@@ -352,8 +354,10 @@ function shadedjars_rebuild
 
   start_clock
 
+  # disabled because "maven_executor" needs to return both command and args
+  # shellcheck disable=2046
   echo_and_redirect "${logfile}" \
-    "${MAVEN}" "${MAVEN_ARGS[@]}" clean verify -fae --batch-mode \
+    $(maven_executor) clean verify -fae --batch-mode \
       -pl hbase-shaded/hbase-shaded-check-invariants -am \
       -Dtest=NoUnitTests -DHBasePatchProcess -Prelease \
       -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dfindbugs.skip=true
@@ -459,8 +463,10 @@ function hadoopcheck_rebuild
   export MAVEN_OPTS="${MAVEN_OPTS}"
   for hadoopver in ${hbase_hadoop2_versions}; do
     logfile="${PATCH_DIR}/patch-javac-${hadoopver}.txt"
+    # disabled because "maven_executor" needs to return both command and args
+    # shellcheck disable=2046
     echo_and_redirect "${logfile}" \
-      "${MAVEN}" clean install \
+      $(maven_executor) clean install \
         -DskipTests -DHBasePatchProcess \
         -Dhadoop-two.version="${hadoopver}"
     count=$(${GREP} -c '\[ERROR\]' "${logfile}")
@@ -473,8 +479,10 @@ function hadoopcheck_rebuild
 
   for hadoopver in ${hbase_hadoop3_versions}; do
     logfile="${PATCH_DIR}/patch-javac-${hadoopver}.txt"
+    # disabled because "maven_executor" needs to return both command and args
+    # shellcheck disable=2046
     echo_and_redirect "${logfile}" \
-      "${MAVEN}" clean install \
+      $(maven_executor) clean install \
         -DskipTests -DHBasePatchProcess \
         -Dhadoop-three.version="${hadoopver}" \
         -Dhadoop.profile=3.0


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

Branch: refs/heads/HBASE-19064
Commit: cfb572eb06405023e1b8441285ca916ca02f54e3
Parents: a338384
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 25 20:09:00 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/cfb572eb/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/cfb572eb/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/cfb572eb/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index eacb2f4..f120dbc 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -129,7 +129,8 @@ public class ReplicationPeers {
   private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
     ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
     boolean enabled = peerStorage.isPeerEnabled(peerId);
+    SyncReplicationState syncReplicationState = peerStorage.getPeerSyncReplicationState(peerId);
     return new ReplicationPeerImpl(ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf),
-        peerId, enabled, peerConfig);
+        peerId, peerConfig, enabled, syncReplicationState);
   }
 }

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

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


[14/30] hbase git commit: HBASE-20182 Addendum throw IOException instead of NoServerForRegionException because it is a DoNotRetryRegionException

Posted by zh...@apache.org.
HBASE-20182 Addendum throw IOException instead of NoServerForRegionException because it is a DoNotRetryRegionException


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

Branch: refs/heads/HBASE-19064
Commit: a2b91727715e10afa29c759cef20540f29a79228
Parents: a2c1be9
Author: zhangduo <zh...@apache.org>
Authored: Wed Apr 11 14:37:56 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:47:06 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HRegionInfo.java    |  8 ++++---
 .../hbase/client/AsyncNonMetaRegionLocator.java | 23 ++++++++++----------
 .../hbase/client/ConnectionImplementation.java  |  4 ++--
 .../hadoop/hbase/client/RegionInfoBuilder.java  |  8 ++++---
 4 files changed, 24 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a2b91727/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index e2982bd..fc03926 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -159,9 +159,11 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
   }
 
   private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
-  // This flag is in the parent of a split while the parent is still referenced
-  // by daughter regions.  We USED to set this flag when we disabled a table
-  // but now table state is kept up in zookeeper as of 0.90.0 HBase.
+  // This flag is in the parent of a split while the parent is still referenced by daughter regions.
+  // We USED to set this flag when we disabled a table but now table state is kept up in zookeeper
+  // as of 0.90.0 HBase. And now in DisableTableProcedure, finally we will create bunch of
+  // UnassignProcedures and at the last of the procedure we will set the region state to CLOSED, and
+  // will not change the offLine flag.
   private boolean offLine = false;
   private long regionId = -1;
   private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2b91727/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index c30de9a..7634b10 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
@@ -269,12 +269,7 @@ class AsyncNonMetaRegionLocator {
   }
 
   // return whether we should stop the scan
-  private boolean onScanNext(TableName tableName, LocateRequest req, Result result,
-      Throwable error) {
-    if (error != null) {
-      complete(tableName, req, null, error);
-      return true;
-    }
+  private boolean onScanNext(TableName tableName, LocateRequest req, Result result) {
     RegionLocations locs = MetaTableAccessor.getRegionLocations(result);
     LOG.debug("The fetched location of '{}', row='{}', locateType={} is {}", tableName,
       Bytes.toStringBinary(req.row), req.locateType, locs);
@@ -298,7 +293,7 @@ class AsyncNonMetaRegionLocator {
     }
     if (loc.getServerName() == null) {
       complete(tableName, req, null,
-        new NoServerForRegionException(
+        new IOException(
             String.format("No server address listed for region '%s', row='%s', locateType=%s",
               info.getRegionNameAsString(), Bytes.toStringBinary(req.row), req.locateType)));
       return true;
@@ -370,22 +365,28 @@ class AsyncNonMetaRegionLocator {
 
           private boolean completeNormally = false;
 
+          private boolean tableNotFound = true;
+
           @Override
           public void onError(Throwable error) {
-            onScanNext(tableName, req, null, error);
+            complete(tableName, req, null, error);
           }
 
           @Override
           public void onComplete() {
-            if (!completeNormally) {
-              onScanNext(tableName, req, null, new TableNotFoundException(tableName));
+            if (tableNotFound) {
+              complete(tableName, req, null, new TableNotFoundException(tableName));
+            } else if (!completeNormally) {
+              complete(tableName, req, null, new IOException(
+                "Unable to find region for " + Bytes.toStringBinary(req.row) + " in " + tableName));
             }
           }
 
           @Override
           public void onNext(Result[] results, ScanController controller) {
             for (Result result : results) {
-              if (onScanNext(tableName, req, result, null)) {
+              tableNotFound = false;
+              if (onScanNext(tableName, req, result)) {
                 completeNormally = true;
                 controller.terminate();
                 return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2b91727/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 a272ffb..53e4b7f 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
@@ -836,7 +836,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
               if (tableNotFound) {
                 throw new TableNotFoundException(tableName);
               } else {
-                throw new NoServerForRegionException(
+                throw new IOException(
                   "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName);
               }
             }
@@ -864,7 +864,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
             // the parent in the above condition, so we may have already reached a region which does
             // not contains us.
             if (!regionInfo.containsRow(row)) {
-              throw new NoServerForRegionException(
+              throw new IOException(
                 "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName);
             }
             ServerName serverName = locations.getRegionLocation(replicaId).getServerName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2b91727/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
index fc35afb..3de9860 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
@@ -149,9 +149,11 @@ public class RegionInfoBuilder {
      * old region name format.
      */
 
-    // This flag is in the parent of a split while the parent is still referenced
-    // by daughter regions.  We USED to set this flag when we disabled a table
-    // but now table state is kept up in zookeeper as of 0.90.0 HBase.
+    // This flag is in the parent of a split while the parent is still referenced by daughter
+    // regions. We USED to set this flag when we disabled a table but now table state is kept up in
+    // zookeeper as of 0.90.0 HBase. And now in DisableTableProcedure, finally we will create bunch
+    // of UnassignProcedures and at the last of the procedure we will set the region state to
+    // CLOSED, and will not change the offLine flag.
     private boolean offLine = false;
     private boolean split = false;
     private final long regionId;


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

Branch: refs/heads/HBASE-19064
Commit: 4bb4d5213062a34c1daf7222c374ffa58d64065f
Parents: d35ba3f
Author: huzheng <op...@gmail.com>
Authored: Thu Mar 1 18:34:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/4bb4d521/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/4bb4d521/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/4bb4d521/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/4bb4d521/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);


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

Branch: refs/heads/HBASE-19064
Commit: 9114d68c12ef71a64dc68a146fc439733e7cba07
Parents: a2b9172
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 11 21:08:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/9114d68c/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 e34818f..0bee9d6 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,12 +607,16 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  @Override
-  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+  protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException {
     return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup,
       channelClass);
   }
 
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    return createAsyncWriter(fs, path);
+  }
+
   private void waitForSafePoint() {
     consumeLock.lock();
     try {
@@ -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/9114d68c/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/9114d68c/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/9114d68c/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/9114d68c/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/9114d68c/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/9114d68c/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/9114d68c/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/9114d68c/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());
+    }
+  }
+}


[10/30] hbase git commit: HBASE-15291 FileSystem not closed in secure bulkLoad

Posted by zh...@apache.org.
HBASE-15291 FileSystem not closed in secure bulkLoad

Signed-off-by: Ashish Singhi <as...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: 828a1c76c71b0179bd9709e3da5d988b18fea631
Parents: 95ca38a
Author: Ashish Singhi <as...@apache.org>
Authored: Wed Apr 11 12:01:28 2018 +0530
Committer: Ashish Singhi <as...@apache.org>
Committed: Wed Apr 11 12:01:28 2018 +0530

----------------------------------------------------------------------
 .../regionserver/SecureBulkLoadManager.java     | 82 +++++++++++++-------
 1 file changed, 54 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/828a1c76/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
index 264d985..a4ee517 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
@@ -145,15 +145,26 @@ public class SecureBulkLoadManager {
 
   public void cleanupBulkLoad(final HRegion region, final CleanupBulkLoadRequest request)
       throws IOException {
-    region.getCoprocessorHost().preCleanupBulkLoad(getActiveUser());
+    try {
+      region.getCoprocessorHost().preCleanupBulkLoad(getActiveUser());
 
-    Path path = new Path(request.getBulkToken());
-    if (!fs.delete(path, true)) {
-      if (fs.exists(path)) {
-        throw new IOException("Failed to clean up " + path);
+      Path path = new Path(request.getBulkToken());
+      if (!fs.delete(path, true)) {
+        if (fs.exists(path)) {
+          throw new IOException("Failed to clean up " + path);
+        }
+      }
+      LOG.info("Cleaned up " + path + " successfully.");
+    } finally {
+      UserGroupInformation ugi = getActiveUser().getUGI();
+      try {
+        if (!UserGroupInformation.getLoginUser().equals(ugi)) {
+          FileSystem.closeAllForUGI(ugi);
+        }
+      } catch (IOException e) {
+        LOG.error("Failed to close FileSystem for: " + ugi, e);
       }
     }
-    LOG.info("Cleaned up " + path + " successfully.");
   }
 
   public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region,
@@ -304,7 +315,7 @@ public class SecureBulkLoadManager {
       }
 
       if (srcFs == null) {
-        srcFs = FileSystem.get(p.toUri(), conf);
+        srcFs = FileSystem.newInstance(p.toUri(), conf);
       }
 
       if(!isFile(p)) {
@@ -334,34 +345,49 @@ public class SecureBulkLoadManager {
     @Override
     public void doneBulkLoad(byte[] family, String srcPath) throws IOException {
       LOG.debug("Bulk Load done for: " + srcPath);
+      closeSrcFs();
+    }
+
+    private void closeSrcFs() throws IOException {
+      if (srcFs != null) {
+        srcFs.close();
+        srcFs = null;
+      }
     }
 
     @Override
     public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
-      if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
-        // files are copied so no need to move them back
-        return;
-      }
-      Path p = new Path(srcPath);
-      Path stageP = new Path(stagingDir,
-          new Path(Bytes.toString(family), p.getName()));
+      try {
+        Path p = new Path(srcPath);
+        if (srcFs == null) {
+          srcFs = FileSystem.newInstance(p.toUri(), conf);
+        }
+        if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
+          // files are copied so no need to move them back
+          return;
+        }
+        Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
 
-      // In case of Replication for bulk load files, hfiles are not renamed by end point during
-      // prepare stage, so no need of rename here again
-      if (p.equals(stageP)) {
-        LOG.debug(p.getName() + " is already available in source directory. Skipping rename.");
-        return;
-      }
+        // In case of Replication for bulk load files, hfiles are not renamed by end point during
+        // prepare stage, so no need of rename here again
+        if (p.equals(stageP)) {
+          LOG.debug(p.getName() + " is already available in source directory. Skipping rename.");
+          return;
+        }
 
-      LOG.debug("Moving " + stageP + " back to " + p);
-      if(!fs.rename(stageP, p))
-        throw new IOException("Failed to move HFile: " + stageP + " to " + p);
+        LOG.debug("Moving " + stageP + " back to " + p);
+        if (!fs.rename(stageP, p)) {
+          throw new IOException("Failed to move HFile: " + stageP + " to " + p);
+        }
 
-      // restore original permission
-      if (origPermissions.containsKey(srcPath)) {
-        fs.setPermission(p, origPermissions.get(srcPath));
-      } else {
-        LOG.warn("Can't find previous permission for path=" + srcPath);
+        // restore original permission
+        if (origPermissions.containsKey(srcPath)) {
+          fs.setPermission(p, origPermissions.get(srcPath));
+        } else {
+          LOG.warn("Can't find previous permission for path=" + srcPath);
+        }
+      } finally {
+        closeSrcFs();
       }
     }
 


[03/30] hbase git commit: HBASE-15466 test personality should recognize docs-only patch.

Posted by zh...@apache.org.
HBASE-15466 test personality should recognize docs-only patch.

* add a new test to build the refguide specifically instead of site
* check for changes to src/main/asciidoc or src/main/xslt and run that test and only that test
* check for changes to the hbase-default.xml file and build the refguide if found (but maybe other tests too)
* fallback to relying on the yetus default for other changes
* fix some missing start_clock entries that cause longer-than-actual reported test time.

Signed-off-by: Mike Drob <md...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: ee87de9bfd37499cd850c5ac214f327d616db2fe
Parents: bc35de5
Author: Sean Busbey <bu...@apache.org>
Authored: Thu Apr 5 15:22:41 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Apr 9 10:37:28 2018 -0500

----------------------------------------------------------------------
 dev-support/hbase-personality.sh | 92 +++++++++++++++++++++++++++++++++++
 1 file changed, 92 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ee87de9b/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index b67777a..d34f761 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -180,6 +180,26 @@ function personality_modules
   done
 }
 
+## @description places where we override the built in assumptions about what tests to run
+## @audience    private
+## @stability   evolving
+## @param       filename of changed file
+function personality_file_tests
+{
+  local filename=$1
+  # If the change is to the refguide, then we don't need any builtin yetus tests
+  # the refguide test (below) will suffice for coverage.
+  if [[ ${filename} =~ src/main/asciidoc ]] ||
+     [[ ${filename} =~ src/main/xslt ]]; then
+    yetus_debug "Skipping builtin yetus checks for ${filename}. refguide test should pick it up."
+  # fallback to checking which tests based on what yetus would do by default
+  elif declare -f "${BUILDTOOL}_builtin_personality_file_tests" >/dev/null; then
+    "${BUILDTOOL}_builtin_personality_file_tests" "${filename}"
+  elif declare -f builtin_personality_file_tests >/dev/null; then
+    builtin_personality_file_tests "${filename}"
+  fi
+}
+
 ## @description  Uses relevant include/exclude env variable to fetch list of included/excluded
 #                tests and sets given variable to arguments to be passes to maven command.
 ## @audience     private
@@ -225,6 +245,74 @@ function get_include_exclude_tests_arg
 
 ###################################################
 
+add_test_type refguide
+
+function refguide_initialize
+{
+  maven_add_install refguide
+}
+
+function refguide_filefilter
+{
+  local filename=$1
+
+  if [[ ${filename} =~ src/main/asciidoc ]] ||
+     [[ ${filename} =~ src/main/xslt ]] ||
+     [[ ${filename} =~ hbase-common/src/main/resources/hbase-default.xml ]]; then
+    add_test refguide
+  fi
+}
+
+function refguide_rebuild
+{
+  local repostatus=$1
+  local logfile="${PATCH_DIR}/${repostatus}-refguide.log"
+  declare -i count
+
+  if ! verify_needed_test refguide; then
+    return 0
+  fi
+
+  big_console_header "Checking we can create the ref guide on ${repostatus}"
+
+  start_clock
+
+  echo_and_redirect "${logfile}" \
+    "${MAVEN}" "${MAVEN_ARGS[@]}" clean site --batch-mode \
+      -pl . \
+      -Dtest=NoUnitTests -DHBasePatchProcess -Prelease \
+      -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dfindbugs.skip=true
+
+  count=$(${GREP} -c '\[ERROR\]' "${logfile}")
+  if [[ ${count} -gt 0 ]]; then
+    add_vote_table -1 refguide "${repostatus} has ${count} errors when building the reference guide."
+    add_footer_table refguide "@@BASE@@/${repostatus}-refguide.log"
+    return 1
+  fi
+
+  if ! mv target/site "${PATCH_DIR}/${repostatus}-site"; then
+    add_vote_table -1 refguide "${repostatus} failed to produce a site directory."
+    add_footer_table refguide "@@BASE@@/${repostatus}-refguide.log"
+    return 1
+  fi
+
+  if [[ ! -f "${PATCH_DIR}/${repostatus}-site/book.html" ]]; then
+    add_vote_table -1 refguide "${repostatus} failed to produce the html version of the reference guide."
+    add_footer_table refguide "@@BASE@@/${repostatus}-refguide.log"
+    return 1
+  fi
+
+  if [[ ! -f "${PATCH_DIR}/${repostatus}-site/apache_hbase_reference_guide.pdf" ]]; then
+    add_vote_table -1 refguide "${repostatus} failed to produce the pdf version of the reference guide."
+    add_footer_table refguide "@@BASE@@/${repostatus}-refguide.log"
+    return 1
+  fi
+
+  add_vote_table 0 refguide "${repostatus} has no errors when building the reference guide. See footer for rendered docs, which you should manually inspect."
+  add_footer_table refguide "@@BASE@@/${repostatus}-site/book.html"
+  return 0
+}
+
 add_test_type shadedjars
 
 
@@ -262,6 +350,8 @@ function shadedjars_rebuild
 
   big_console_header "Checking shaded client builds on ${repostatus}"
 
+  start_clock
+
   echo_and_redirect "${logfile}" \
     "${MAVEN}" "${MAVEN_ARGS[@]}" clean verify -fae --batch-mode \
       -pl hbase-shaded/hbase-shaded-check-invariants -am \
@@ -343,6 +433,8 @@ function hadoopcheck_rebuild
 
   big_console_header "Compiling against various Hadoop versions"
 
+  start_clock
+
   # 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"


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

Branch: refs/heads/HBASE-19064
Commit: 74211ac22d9c3f46d0ff27b4fcb64b0f70288d53
Parents: c5692a9
Author: zhangduo <zh...@apache.org>
Authored: Mon Feb 12 18:20:18 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/74211ac2/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/74211ac2/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 d1003b9..bd4b320 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -392,7 +392,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/74211ac2/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/74211ac2/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/74211ac2/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 eccb67e..9f9911d 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
@@ -4314,12 +4314,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/74211ac2/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 ddd6a06..af7b1e8 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
@@ -1805,7 +1805,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/74211ac2/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 922fa86..9555a71 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,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/74211ac2/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/74211ac2/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/74211ac2/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/74211ac2/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/74211ac2/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/74211ac2/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 202b584..1b8f52e 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;
@@ -160,7 +160,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/74211ac2/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/74211ac2/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());


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

Branch: refs/heads/HBASE-19064
Commit: 4ced805ab2f7fbcd6eb64b37232ce58dbf44b7b9
Parents: d5858f6
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Mar 9 11:30:25 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/4ced805a/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/4ced805a/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 f8e2105..25bc6be 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,10 +1804,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);
@@ -1926,11 +1924,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/4ced805a/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/4ced805a/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 1b8f52e..44cac1a 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;
@@ -149,6 +148,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);
@@ -160,7 +174,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/4ced805a/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/4ced805a/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 817f03d..01d0348 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.master.replication.ReplaySyncReplicationWALManage
 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.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/4ced805a/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());


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

Branch: refs/heads/HBASE-19064
Commit: 21d0a0b8af382c15877486b30aa3226056055e56
Parents: 9114d68
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Jan 13 18:55:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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/21d0a0b8/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"
 


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

Branch: refs/heads/HBASE-19064
Commit: c5692a90c1021e75d7e967cca3aff2220aec2a71
Parents: deac48d
Author: zhangduo <zh...@apache.org>
Authored: Fri Feb 9 18:33:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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 acc2cbd..d1003b9 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -384,6 +384,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;
 }
@@ -394,18 +405,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 {
@@ -433,5 +449,7 @@ message DisablePeerStateData {
 }
 
 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/c5692a90/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/c5692a90/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/c5692a90/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 f120dbc..d2a5599 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
@@ -79,8 +79,8 @@ public class ReplicationPeers {
     return true;
   }
 
-  public void removePeer(String peerId) {
-    peerCache.remove(peerId);
+  public ReplicationPeerImpl removePeer(String peerId) {
+    return peerCache.remove(peerId);
   }
 
   /**
@@ -105,22 +105,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
@@ -130,7 +137,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/c5692a90/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/c5692a90/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 8d2b55f..ba340cb 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
@@ -1236,7 +1236,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,
@@ -1247,11 +1247,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/c5692a90/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 e90c43b..2809efa 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;
@@ -3405,7 +3405,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/c5692a90/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 cc008bd..158a1d0 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
@@ -1531,22 +1531,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/c5692a90/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/c5692a90/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 8bedeff..45531f0 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
@@ -302,17 +302,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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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 240de85..ddd6a06 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
@@ -1802,21 +1802,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;
   }
 
@@ -2940,11 +2946,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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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 3ecc50a..471d9b5 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.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@@ -134,6 +135,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
@@ -168,9 +171,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;
@@ -183,10 +185,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.
@@ -247,8 +250,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 {
@@ -267,13 +273,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
@@ -304,7 +313,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));
   }
@@ -356,6 +368,10 @@ public class ReplicationSourceManager implements ReplicationListener {
         }
       }
     }
+    ReplicationPeerConfig peerConfig = peer.getPeerConfig();
+    if (peerConfig.isSyncReplication()) {
+      syncReplicationPeerMappingManager.add(peer.getId(), peerConfig);
+    }
     src.startup();
     return src;
   }
@@ -435,6 +451,7 @@ public class ReplicationSourceManager implements ReplicationListener {
     // Delete queue from storage and memory
     deleteQueue(src.getQueueId());
     this.walsById.remove(src.getQueueId());
+
   }
 
   /**


[09/30] hbase git commit: HBASE-20149 Purge dev javadoc from bin tarball (or make a separate tarball of javadoc)

Posted by zh...@apache.org.
HBASE-20149 Purge dev javadoc from bin tarball (or make a separate tarball of javadoc)

Instead point back to the website where version-specific dev apis are
hosted.


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

Branch: refs/heads/HBASE-19064
Commit: 95ca38a539e983a59deeee08187bf36e7fadbbf0
Parents: 93498dd
Author: Michael Stack <st...@apache.org>
Authored: Fri Apr 6 14:25:07 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Tue Apr 10 10:04:22 2018 -0700

----------------------------------------------------------------------
 hbase-assembly/src/main/assembly/components.xml | 5 +++++
 src/site/site.xml                               | 4 ++--
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/95ca38a5/hbase-assembly/src/main/assembly/components.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/components.xml b/hbase-assembly/src/main/assembly/components.xml
index e6c45e0..59502d6 100644
--- a/hbase-assembly/src/main/assembly/components.xml
+++ b/hbase-assembly/src/main/assembly/components.xml
@@ -26,6 +26,11 @@
     <fileSet>
       <directory>${project.basedir}/../target/site</directory>
       <outputDirectory>docs</outputDirectory>
+      <!--Don't include dev apidocs. They bloat the tgz by almost 50%.
+           See HBASE-20149-->
+      <excludes>
+        <exclude>**/*devapidocs/**</exclude>
+      </excludes>
     </fileSet>
     <!-- Include top level text files-->
     <fileSet>

http://git-wip-us.apache.org/repos/asf/hbase/blob/95ca38a5/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index ce7f8ae..808801b 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -100,8 +100,8 @@
       <item name="Getting Started" href="book.html#quickstart" target="_blank" />
       <item name="User API" href="apidocs/index.html" target="_blank" />
       <item name="User API (Test)" href="testapidocs/index.html" target="_blank" />
-      <item name="Developer API" href="devapidocs/index.html" target="_blank" />
-      <item name="Developer API (Test)" href="testdevapidocs/index.html" target="_blank" />
+      <item name="Developer API" href="https://hbase.apache.org/2.0/devapidocs/index.html" target="_blank" />
+      <item name="Developer API (Test)" href="https://hbase.apache.org/2.0/testdevapidocs/index.html" target="_blank" />
       <item name="中文参考指南(单页)" href="http://abloz.com/hbase/book.html" target="_blank" />
       <item name="FAQ" href="book.html#faq" target="_blank" />
       <item name="Videos/Presentations" href="book.html#other.info" target="_blank" />


[02/30] hbase git commit: HBASE-20363 Addendum wait until region is splittable

Posted by zh...@apache.org.
HBASE-20363 Addendum wait until region is splittable


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

Branch: refs/heads/HBASE-19064
Commit: bc35de597d07b41d7456a38fc2cb771c9292429a
Parents: c7593d1
Author: zhangduo <zh...@apache.org>
Authored: Mon Apr 9 09:04:32 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 09:07:06 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/namespace/TestNamespaceAuditor.java      | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bc35de59/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 40aba85..1b4957a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -361,8 +361,14 @@ public class TestNamespaceAuditor {
     hris = ADMIN.getRegions(tableTwo);
     assertEquals(initialRegions - 1, hris.size());
     Collections.sort(hris, RegionInfo.COMPARATOR);
-    UTIL.compact(tableTwo, true);
-    ADMIN.splitRegionAsync(hris.get(0).getRegionName(), Bytes.toBytes("3")).get(10,
+    byte[] splitKey = Bytes.toBytes("3");
+    HRegion regionToSplit = UTIL.getMiniHBaseCluster().getRegions(tableTwo).stream()
+      .filter(r -> r.getRegionInfo().containsRow(splitKey)).findFirst().get();
+    regionToSplit.compact(true);
+    // the above compact may quit immediately if there is a compaction ongoing, so here we need to
+    // wait a while to let the ongoing compaction finish.
+    UTIL.waitFor(10000, regionToSplit::isSplittable);
+    ADMIN.splitRegionAsync(regionToSplit.getRegionInfo().getRegionName(), splitKey).get(10,
       TimeUnit.SECONDS);
     hris = ADMIN.getRegions(tableTwo);
     assertEquals(initialRegions, hris.size());


[06/30] hbase git commit: Revert "HBASE-20371 note hbasecon cfp on landing page."

Posted by zh...@apache.org.
Revert "HBASE-20371 note hbasecon cfp on landing page."

This reverts commit 7fc6968f7d93bda884abd61b72e69b629dce7a1c.

missing sign-off


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

Branch: refs/heads/HBASE-19064
Commit: a3d5bd55860010c45cf0c7221e42e78507709ed0
Parents: 7fc6968
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Apr 9 12:18:27 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Apr 9 12:18:38 2018 -0500

----------------------------------------------------------------------
 src/site/xdoc/index.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3d5bd55/src/site/xdoc/index.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/index.xml b/src/site/xdoc/index.xml
index 49cb6f7..b402e64 100644
--- a/src/site/xdoc/index.xml
+++ b/src/site/xdoc/index.xml
@@ -83,7 +83,7 @@ Apache HBase is an open-source, distributed, versioned, non-relational database
 </section>
 
      <section name="News">
-       <p>June 18th, 2018 <a href="https://hbase.apache.org/hbasecon-2018">HBaseCon 2018</a> @ San Jose Convention Center, San Jose, CA, USA. CFP open, see site for details!</p>
+       <p>June 18th, 2018 <a href="https://hbase.apache.org/hbasecon-2018">HBaseCon 2018</a> @ San Jose Convention Center, San Jose, CA, USA</p>
        <p>August 4th, 2017 <a href="https://easychair.org/cfp/HBaseConAsia2017">HBaseCon Asia 2017</a> @ the Huawei Campus in Shenzhen, China</p>
        <p>June 12th, 2017 <a href="https://easychair.org/cfp/hbasecon2017">HBaseCon2017</a> at the Crittenden Buildings on the Google Mountain View Campus</p>
        <p>April 25th, 2017 <a href="https://www.meetup.com/hbaseusergroup/events/239291716/">Meetup</a> @ Visa in Palo Alto</p>


[15/30] 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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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/c5692a90/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 06999ea..202b584 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;
@@ -143,18 +143,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
@@ -171,7 +159,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.");
@@ -181,26 +175,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.
@@ -248,8 +222,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/c5692a90/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/c5692a90/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/c5692a90/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());
   }
 }


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

Branch: refs/heads/HBASE-19064
Commit: a3383847452134b3672b9dec1d0ed79cbc94cb97
Parents: 6c9e1f7
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Jan 22 11:44:49 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/a3383847/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 a6a4224..3fd541b 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/a3383847/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 35cdd3f..895e7ff 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.RpcChannel;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -600,6 +602,35 @@ public interface AsyncAdmin {
       ReplicationPeerConfig peerConfig);
 
   /**
+   * Transit current cluster to a new state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @param state a new state of current cluster
+   */
+  CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
+      SyncReplicationState state);
+
+  /**
+   * Get the current cluster state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @return the current cluster state wrapped by a {@link CompletableFuture}.
+   */
+  default CompletableFuture<SyncReplicationState>
+      getReplicationPeerSyncReplicationState(String peerId) {
+    CompletableFuture<SyncReplicationState> future = new CompletableFuture<>();
+    listReplicationPeers(Pattern.compile(peerId)).whenComplete((peers, error) -> {
+      if (error != null) {
+        future.completeExceptionally(error);
+      } else if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) {
+        future.completeExceptionally(
+          new IOException("Replication peer " + peerId + " does not exist"));
+      } else {
+        future.complete(peers.get(0).getSyncReplicationState());
+      }
+    });
+    return future;
+  }
+
+  /**
    * Append the replicable table-cf config of the specified peer
    * @param peerId a short that identifies the cluster
    * @param tableCfs A map from tableName to column family names

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3383847/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 9b2390c..44771fd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -414,6 +415,12 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
+      SyncReplicationState clusterState) {
+    return wrap(rawAdmin.transitReplicationPeerSyncReplicationState(peerId, clusterState));
+  }
+
+  @Override
   public CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
       Map<TableName, List<String>> tableCfs) {
     return wrap(rawAdmin.appendReplicationPeerTableCFs(peerId, tableCfs));

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3383847/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/a3383847/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/a3383847/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/a3383847/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/a3383847/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 8ce2f1b..4e67b78 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/a3383847/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/a3383847/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 b37557c..acc2cbd 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -431,3 +431,7 @@ message EnablePeerStateData {
 
 message DisablePeerStateData {
 }
+
+message TransitPeerSyncReplicationStateStateData {
+  required SyncReplicationState syncReplicationState = 1;
+}

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3383847/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 9dd685d..e90c43b 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;
@@ -3474,6 +3476,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/a3383847/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 8c8c02c..cc008bd 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;
@@ -1530,6 +1531,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/a3383847/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/a3383847/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/a3383847/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/a3383847/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/a3383847/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/a3383847/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 bebf16c..52de755 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;
@@ -2505,6 +2506,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/a3383847/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/a3383847/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/a3383847/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/a3383847/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/a3383847/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/a3383847/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 2e9be30..ac88ff5 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;
@@ -2939,6 +2940,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


[05/30] hbase git commit: HBASE-20371 note hbasecon cfp on landing page.

Posted by zh...@apache.org.
HBASE-20371 note hbasecon cfp on landing page.


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

Branch: refs/heads/HBASE-19064
Commit: 7fc6968f7d93bda884abd61b72e69b629dce7a1c
Parents: 116a808
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Apr 9 11:31:03 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Apr 9 12:15:52 2018 -0500

----------------------------------------------------------------------
 src/site/xdoc/index.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7fc6968f/src/site/xdoc/index.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/index.xml b/src/site/xdoc/index.xml
index b402e64..49cb6f7 100644
--- a/src/site/xdoc/index.xml
+++ b/src/site/xdoc/index.xml
@@ -83,7 +83,7 @@ Apache HBase is an open-source, distributed, versioned, non-relational database
 </section>
 
      <section name="News">
-       <p>June 18th, 2018 <a href="https://hbase.apache.org/hbasecon-2018">HBaseCon 2018</a> @ San Jose Convention Center, San Jose, CA, USA</p>
+       <p>June 18th, 2018 <a href="https://hbase.apache.org/hbasecon-2018">HBaseCon 2018</a> @ San Jose Convention Center, San Jose, CA, USA. CFP open, see site for details!</p>
        <p>August 4th, 2017 <a href="https://easychair.org/cfp/HBaseConAsia2017">HBaseCon Asia 2017</a> @ the Huawei Campus in Shenzhen, China</p>
        <p>June 12th, 2017 <a href="https://easychair.org/cfp/hbasecon2017">HBaseCon2017</a> at the Crittenden Buildings on the Google Mountain View Campus</p>
        <p>April 25th, 2017 <a href="https://www.meetup.com/hbaseusergroup/events/239291716/">Meetup</a> @ Visa in Palo Alto</p>


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

Branch: refs/heads/HBASE-19064
Commit: d5858f6bbfec890b1879dbf49d4d5f7750bf5d13
Parents: 4bb4d52
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Mar 2 18:43:25 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/d5858f6b/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 bd4b320..24ffc78 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -454,3 +454,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/d5858f6b/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/d5858f6b/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/d5858f6b/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 2809efa..4d3310c 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();
@@ -3654,4 +3658,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/d5858f6b/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/d5858f6b/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/d5858f6b/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/d5858f6b/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/d5858f6b/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/d5858f6b/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/d5858f6b/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 af7b1e8..f8e2105 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
@@ -1927,6 +1927,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);
@@ -2874,14 +2879,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/d5858f6b/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/d5858f6b/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/d5858f6b/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 b106a31..178447e 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
@@ -937,6 +937,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/d5858f6b/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/d5858f6b/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..817f03d
--- /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.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);
+        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);
+  }
+}


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

Branch: refs/heads/HBASE-19064
Commit: a87d6138c1a1dc8c497d0bec7fa122755f7e316f
Parents: cfb572e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 26 16:50:48 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/a87d6138/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/a87d6138/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/a87d6138/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 4e67b78..8b7bc38 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/a87d6138/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/a87d6138/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/a87d6138/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/a87d6138/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/a87d6138/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);
 


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

Branch: refs/heads/HBASE-19064
Commit: d35ba3f1913299129c02d9f91441a6fd5ce40a8a
Parents: 74211ac
Author: zhangduo <zh...@apache.org>
Authored: Wed Feb 14 16:01:16 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/d35ba3f1/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/d35ba3f1/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/d35ba3f1/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/d35ba3f1/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/d35ba3f1/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/d35ba3f1/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,


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

Branch: refs/heads/HBASE-19064
Commit: deac48d9e19e9a565f71b965580ead1a6661602b
Parents: a87d613
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Feb 6 16:00:59 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/deac48d9/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/deac48d9/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/deac48d9/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/deac48d9/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/deac48d9/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";


[13/30] hbase git commit: HBASE-20372 [website] move stuff from more than 2 years ago to old news

Posted by zh...@apache.org.
HBASE-20372 [website] move stuff from more than 2 years ago to old news

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: 1efd3ac2402233d5cb1d36ef764c34d2dba1f1a5
Parents: 828a1c7
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Apr 9 11:34:57 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Apr 11 01:33:14 2018 -0500

----------------------------------------------------------------------
 src/site/xdoc/index.xml    | 13 -------------
 src/site/xdoc/old_news.xml | 13 +++++++++++++
 2 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1efd3ac2/src/site/xdoc/index.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/index.xml b/src/site/xdoc/index.xml
index 49cb6f7..e1440ab 100644
--- a/src/site/xdoc/index.xml
+++ b/src/site/xdoc/index.xml
@@ -90,19 +90,6 @@ Apache HBase is an open-source, distributed, versioned, non-relational database
         <p>December 8th, 2016 <a href="https://www.meetup.com/hbaseusergroup/events/235542241/">Meetup@Splice</a> in San Francisco</p>
        <p>September 26th, 2016 <a href="http://www.meetup.com/HBase-NYC/events/233024937/">HBaseConEast2016</a> at Google in Chelsea, NYC</p>
          <p>May 24th, 2016 <a href="http://www.hbasecon.com/">HBaseCon2016</a> at The Village, 969 Market, San Francisco</p>
-       <p>June 25th, 2015 <a href="http://www.zusaar.com/event/14057003">HBase Summer Meetup 2015</a> in Tokyo</p>
-       <p>May 7th, 2015 <a href="http://hbasecon.com/">HBaseCon2015</a> in San Francisco</p>
-       <p>February 17th, 2015 <a href="http://www.meetup.com/hbaseusergroup/events/219260093/">HBase meetup around Strata+Hadoop World</a> in San Jose</p>
-       <p>January 15th, 2015 <a href="http://www.meetup.com/hbaseusergroup/events/218744798/">HBase meetup @ AppDynamics</a> in San Francisco</p>
-       <p>November 20th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/205219992/">HBase meetup @ WANdisco</a> in San Ramon</p>
-       <p>October 27th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/207386102/">HBase Meetup @ Apple</a> in Cupertino</p>
-       <p>October 15th, 2014 <a href="http://www.meetup.com/HBase-NYC/events/207655552/">HBase Meetup @ Google</a> on the night before Strata/HW in NYC</p>
-       <p>September 25th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/203173692/">HBase Meetup @ Continuuity</a> in Palo Alto</p>
-         <p>August 28th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/197773762/">HBase Meetup @ Sift Science</a> in San Francisco</p>
-         <p>July 17th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/190994082/">HBase Meetup @ HP</a> in Sunnyvale</p>
-         <p>June 5th, 2014 <a href="http://www.meetup.com/Hadoop-Summit-Community-San-Jose/events/179081342/">HBase BOF at Hadoop Summit</a>, San Jose Convention Center</p>
-         <p>May 5th, 2014 <a href="http://www.hbasecon.com/">HBaseCon2014</a> at the Hilton San Francisco on Union Square</p>
-         <p>March 12th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/160757912/">HBase Meetup @ Ancestry.com</a> in San Francisco</p>
       <p><small><a href="old_news.html">Old News</a></small></p>
     </section>
   </body>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1efd3ac2/src/site/xdoc/old_news.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/old_news.xml b/src/site/xdoc/old_news.xml
index 705d69d..2203e4e 100644
--- a/src/site/xdoc/old_news.xml
+++ b/src/site/xdoc/old_news.xml
@@ -31,6 +31,19 @@ under the License.
   </properties>
   <body>
   <section name="Old News">
+       <p>June 25th, 2015 <a href="http://www.zusaar.com/event/14057003">HBase Summer Meetup 2015</a> in Tokyo</p>
+       <p>May 7th, 2015 <a href="http://hbasecon.com/">HBaseCon2015</a> in San Francisco</p>
+       <p>February 17th, 2015 <a href="http://www.meetup.com/hbaseusergroup/events/219260093/">HBase meetup around Strata+Hadoop World</a> in San Jose</p>
+       <p>January 15th, 2015 <a href="http://www.meetup.com/hbaseusergroup/events/218744798/">HBase meetup @ AppDynamics</a> in San Francisco</p>
+       <p>November 20th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/205219992/">HBase meetup @ WANdisco</a> in San Ramon</p>
+       <p>October 27th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/207386102/">HBase Meetup @ Apple</a> in Cupertino</p>
+       <p>October 15th, 2014 <a href="http://www.meetup.com/HBase-NYC/events/207655552/">HBase Meetup @ Google</a> on the night before Strata/HW in NYC</p>
+       <p>September 25th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/203173692/">HBase Meetup @ Continuuity</a> in Palo Alto</p>
+         <p>August 28th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/197773762/">HBase Meetup @ Sift Science</a> in San Francisco</p>
+         <p>July 17th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/190994082/">HBase Meetup @ HP</a> in Sunnyvale</p>
+         <p>June 5th, 2014 <a href="http://www.meetup.com/Hadoop-Summit-Community-San-Jose/events/179081342/">HBase BOF at Hadoop Summit</a>, San Jose Convention Center</p>
+         <p>May 5th, 2014 <a href="http://www.hbasecon.com/">HBaseCon2014</a> at the Hilton San Francisco on Union Square</p>
+         <p>March 12th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/160757912/">HBase Meetup @ Ancestry.com</a> in San Francisco</p>
          <p>February 10th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/163139322/">HBase Meetup @ Continuuity</a> in Palo Alto</p>
          <p>January 30th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/158491762/">HBase Meetup @ Apple</a> in Cupertino</p>
          <p>January 30th, 2014 <a href="http://www.meetup.com/Los-Angeles-HBase-User-group/events/160560282/">Los Angeles HBase User Group</a> in El Segundo</p>


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

Branch: refs/heads/HBASE-19064
Commit: 6c9e1f7f446bd5de3a1e7ec9069b0e970879f8b4
Parents: 21d0a0b
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 19 18:38:39 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 11 14:56:43 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/6c9e1f7f/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 ce8dafa..4816d77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -430,6 +430,13 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     this.implClassName = getClass().getSimpleName();
   }
 
+  /**
+   * Used to initialize the WAL. Usually just call rollWriter to create the first log writer.
+   */
+  public void init() throws IOException {
+    rollWriter();
+  }
+
   @Override
   public void registerWALActionsListener(WALActionsListener listener) {
     this.listeners.add(listener);

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c9e1f7f/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/6c9e1f7f/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/6c9e1f7f/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 2d4b385..7529f90 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/6c9e1f7f/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/6c9e1f7f/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/6c9e1f7f/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/6c9e1f7f/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/6c9e1f7f/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/6c9e1f7f/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/6c9e1f7f/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 e54f1f8..ee50564 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -104,8 +104,6 @@ public class IOTestProvider implements WALProvider {
     this.factory = factory;
     this.conf = conf;
     this.providerId = providerId != null ? providerId : DEFAULT_PROVIDER_ID;
-
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c9e1f7f/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());
+  }
+}