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

[01/50] [abbrv] hbase git commit: HBASE-19998 Flakey TestVisibilityLabelsWithDefaultVisLabelService [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19064 33d0606a2 -> 0ac769246 (forced update)


HBASE-19998 Flakey TestVisibilityLabelsWithDefaultVisLabelService

Only call server.checkIfShouldMoveSystemRegionAsync if a node has been
added. Do not call it if only one regionserver in cluster. Make it
so ServerCrashProcedure runs before it. Add logging if
server.checkIfShouldMoveSystemRegionAsync was responsible for
MOVE (Previous was a mystery when it cut in).

Previous we'd call it when there was a nodeChildrenChanged. These
happen before nodeDeleted. If a server crashed,
checkIfShouldMoveSystemRegionAsync could run first, find the
server that had not yet registered as crashed, find system
tables on it and then try to move them. It would fail because
server would not respond to RPC. The region move would then
be waiting on the servercrashprocedure to wake it up when
done processing but this move had locked the region so
SCP couldn't run....


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

Branch: refs/heads/HBASE-19064
Commit: 50c705dad9825d3095352b997be35a2568bd6190
Parents: 816d860
Author: Michael Stack <st...@apache.org>
Authored: Wed Feb 14 22:32:29 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 15 06:08:55 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/MasterServices.java     |  5 +++++
 .../hbase/master/RegionServerTracker.java       | 16 +++++++++++---
 .../master/assignment/AssignmentManager.java    | 23 ++++++++++++++++++--
 .../master/assignment/MoveRegionProcedure.java  |  1 -
 4 files changed, 39 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/50c705da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 9d371bd..0e552d6 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
@@ -486,6 +486,11 @@ public interface MasterServices extends Server {
 
   public String getRegionServerVersion(final ServerName sn);
 
+  /**
+   * Called when a new RegionServer is added to the cluster.
+   * Checks if new server has a newer version than any existing server and will move system tables
+   * there if so.
+   */
   public void checkIfShouldMoveSystemRegionAsync();
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/50c705da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
index 29218e2..81fc589 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
@@ -104,9 +104,6 @@ public class RegionServerTracker extends ZKListener {
         }
       }
     }
-    if (server.isInitialized()) {
-      server.checkIfShouldMoveSystemRegionAsync();
-    }
   }
 
   private void remove(final ServerName sn) {
@@ -116,6 +113,19 @@ public class RegionServerTracker extends ZKListener {
   }
 
   @Override
+  public void nodeCreated(String path) {
+    if (path.startsWith(watcher.znodePaths.rsZNode)) {
+      String serverName = ZKUtil.getNodeName(path);
+      LOG.info("RegionServer ephemeral node created, adding [" + serverName + "]");
+      if (server.isInitialized()) {
+        // Only call the check to move servers if a RegionServer was added to the cluster; in this
+        // case it could be a server with a new version so it makes sense to run the check.
+        server.checkIfShouldMoveSystemRegionAsync();
+      }
+    }
+  }
+
+  @Override
   public void nodeDeleted(String path) {
     if (path.startsWith(watcher.znodePaths.rsZNode)) {
       String serverName = ZKUtil.getNodeName(path);

http://git-wip-us.apache.org/repos/asf/hbase/blob/50c705da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 8b3dc61..97d8258 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -455,12 +455,27 @@ public class AssignmentManager implements ServerListener {
    * If so, move all system table regions to RS with the highest version to keep compatibility.
    * The reason is, RS in new version may not be able to access RS in old version when there are
    * some incompatible changes.
+   * <p>This method is called when a new RegionServer is added to cluster only.</p>
    */
   public void checkIfShouldMoveSystemRegionAsync() {
+    // TODO: Fix this thread. If a server is killed and a new one started, this thread thinks that
+    // it should 'move' the system tables from the old server to the new server but
+    // ServerCrashProcedure is on it; and it will take care of the assign without dataloss.
+    if (this.master.getServerManager().countOfRegionServers() <= 1) {
+      return;
+    }
+    // This thread used to run whenever there was a change in the cluster. The ZooKeeper
+    // childrenChanged notification came in before the nodeDeleted message and so this method
+    // cold run before a ServerCrashProcedure could run. That meant that this thread could see
+    // a Crashed Server before ServerCrashProcedure and it could find system regions on the
+    // crashed server and go move them before ServerCrashProcedure had a chance; could be
+    // dataloss too if WALs were not recovered.
     new Thread(() -> {
       try {
         synchronized (checkIfShouldMoveSystemRegionLock) {
           List<RegionPlan> plans = new ArrayList<>();
+          // TODO: I don't think this code does a good job if all servers in cluster have same
+          // version. It looks like it will schedule unnecessary moves.
           for (ServerName server : getExcludedServersForSystemTable()) {
             if (master.getServerManager().isServerDead(server)) {
               // TODO: See HBASE-18494 and HBASE-18495. Though getExcludedServersForSystemTable()
@@ -471,13 +486,15 @@ public class AssignmentManager implements ServerListener {
               // handling.
               continue;
             }
-            List<RegionInfo> regionsShouldMove = getCarryingSystemTables(server);
+            List<RegionInfo> regionsShouldMove = getSystemTables(server);
             if (!regionsShouldMove.isEmpty()) {
               for (RegionInfo regionInfo : regionsShouldMove) {
                 // null value for dest forces destination server to be selected by balancer
                 RegionPlan plan = new RegionPlan(regionInfo, server, null);
                 if (regionInfo.isMetaRegion()) {
                   // Must move meta region first.
+                  LOG.info("Async MOVE of {} to newer Server={}",
+                      regionInfo.getEncodedName(), server);
                   moveAsync(plan);
                 } else {
                   plans.add(plan);
@@ -485,6 +502,8 @@ public class AssignmentManager implements ServerListener {
               }
             }
             for (RegionPlan plan : plans) {
+              LOG.info("Async MOVE of {} to newer Server={}",
+                  plan.getRegionInfo().getEncodedName(), server);
               moveAsync(plan);
             }
           }
@@ -495,7 +514,7 @@ public class AssignmentManager implements ServerListener {
     }).start();
   }
 
-  private List<RegionInfo> getCarryingSystemTables(ServerName serverName) {
+  private List<RegionInfo> getSystemTables(ServerName serverName) {
     Set<RegionStateNode> regions = this.getRegionStates().getServerNode(serverName).getRegions();
     if (regions == null) {
       return new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/50c705da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
index 4e7cde6..a29bfee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
@@ -54,7 +54,6 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
   public MoveRegionProcedure(final MasterProcedureEnv env, final RegionPlan plan) {
     super(env, plan.getRegionInfo());
     this.plan = plan;
-    LOG.info("REMOVE", new Throwable("REMOVE: Just to see who is calling Move!!!"));
   }
 
   @Override


[02/50] [abbrv] hbase git commit: HBASE-19903 Split TestShell so it will not time out

Posted by zh...@apache.org.
HBASE-19903 Split TestShell so it will not time out


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

Branch: refs/heads/HBASE-19064
Commit: 01a2869363d45c7692f3984f31ab8c7a50d117fc
Parents: 50c705d
Author: zhangduo <zh...@apache.org>
Authored: Thu Feb 15 21:11:14 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 16 00:17:19 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/TestAdminShell.java     | 42 ++++++++++++++++++++
 .../hadoop/hbase/client/TestQuotasShell.java    | 42 ++++++++++++++++++++
 .../apache/hadoop/hbase/client/TestShell.java   |  6 +--
 .../hadoop/hbase/client/TestShellNoCluster.java |  4 +-
 .../hadoop/hbase/client/TestTableShell.java     | 42 ++++++++++++++++++++
 5 files changed, 131 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/01a28693/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestAdminShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestAdminShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestAdminShell.java
new file mode 100644
index 0000000..1835d88
--- /dev/null
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestAdminShell.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.jruby.embed.PathType;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, LargeTests.class })
+public class TestAdminShell extends AbstractTestShell {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAdminShell.class);
+
+  @Test
+  public void testRunShellTests() throws IOException {
+    System.setProperty("shell.test.include", "admin_test.rb");
+    // Start all ruby tests
+    jruby.runScriptlet(PathType.ABSOLUTE, "src/test/ruby/tests_runner.rb");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/01a28693/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestQuotasShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestQuotasShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestQuotasShell.java
new file mode 100644
index 0000000..482bf0f
--- /dev/null
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestQuotasShell.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.jruby.embed.PathType;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, LargeTests.class })
+public class TestQuotasShell extends AbstractTestShell {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestQuotasShell.class);
+
+  @Test
+  public void testRunShellTests() throws IOException {
+    System.setProperty("shell.test.include", "quotas_test.rb");
+    // Start all ruby tests
+    jruby.runScriptlet(PathType.ABSOLUTE, "src/test/ruby/tests_runner.rb");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/01a28693/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java
index e3de94a..ab36edc 100644
--- a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java
@@ -30,12 +30,12 @@ import org.junit.experimental.categories.Category;
 public class TestShell extends AbstractTestShell {
 
   @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestShell.class);
+  public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestShell.class);
 
   @Test
   public void testRunShellTests() throws IOException {
-    System.setProperty("shell.test.exclude", "replication_admin_test.rb,rsgroup_shell_test.rb");
+    System.setProperty("shell.test.exclude", "replication_admin_test.rb,rsgroup_shell_test.rb," +
+      "admin_test.rb,table_test.rb,quotas_test.rb");
     // Start all ruby tests
     jruby.runScriptlet(PathType.ABSOLUTE, "src/test/ruby/tests_runner.rb");
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/01a28693/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java
index 3dfd991..3172e97 100644
--- a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShellNoCluster.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.jruby.embed.PathType;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -30,7 +30,7 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-@Category({ ClientTests.class, LargeTests.class })
+@Category({ ClientTests.class, MediumTests.class })
 public class TestShellNoCluster extends AbstractTestShell {
 
   @ClassRule

http://git-wip-us.apache.org/repos/asf/hbase/blob/01a28693/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestTableShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestTableShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestTableShell.java
new file mode 100644
index 0000000..e2fdcaa
--- /dev/null
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestTableShell.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.jruby.embed.PathType;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, MediumTests.class })
+public class TestTableShell extends AbstractTestShell {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestTableShell.class);
+
+  @Test
+  public void testRunShellTests() throws IOException {
+    System.setProperty("shell.test.include", "table_test.rb");
+    // Start all ruby tests
+    jruby.runScriptlet(PathType.ABSOLUTE, "src/test/ruby/tests_runner.rb");
+  }
+}


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

Branch: refs/heads/HBASE-19064
Commit: fbab0eb9a8b58d3c4c6f0b8890413b47ab4708e1
Parents: c457396
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 25 20:09:00 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 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/fbab0eb9/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/fbab0eb9/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/fbab0eb9/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/fbab0eb9/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/fbab0eb9/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/fbab0eb9/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/fbab0eb9/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/fbab0eb9/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/fbab0eb9/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/fbab0eb9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index db62b49..1204c9b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -172,6 +172,9 @@ public abstract class TestReplicationSourceManager {
     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state");
     ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state",
       ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
+    ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/sync-rep-state");
+    ZKUtil.setData(zkw, "/hbase/replication/peers/1/sync-rep-state",
+      Bytes.toBytes(SyncReplicationState.NONE.ordinal()));
     ZKUtil.createWithParents(zkw, "/hbase/replication/state");
     ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
 

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


[10/50] [abbrv] hbase git commit: HBASE-20013 TestZKPermissionWatcher is flakey

Posted by zh...@apache.org.
HBASE-20013 TestZKPermissionWatcher is flakey


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

Branch: refs/heads/HBASE-19064
Commit: 68d509bc1ff7a3bf69a596aed49f238b42ee0679
Parents: 9698951
Author: Michael Stack <st...@apache.org>
Authored: Fri Feb 16 20:11:03 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Feb 16 23:22:23 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/68d509bc/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 02fbc02..b949aa9 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
@@ -2718,11 +2718,21 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  void checkInitialized()
-      throws PleaseHoldException, ServerNotRunningYetException, MasterNotRunningException {
+  public static class MasterStoppedException extends DoNotRetryIOException {
+    MasterStoppedException() {
+      super();
+    }
+  }
+
+  void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException,
+      MasterNotRunningException, MasterStoppedException {
     checkServiceStarted();
-    if (!isInitialized()) throw new PleaseHoldException("Master is initializing");
-    if (isStopped()) throw new MasterNotRunningException();
+    if (!isInitialized()) {
+      throw new PleaseHoldException("Master is initializing");
+    }
+    if (isStopped()) {
+      throw new MasterStoppedException();
+    }
   }
 
   /**


[25/50] [abbrv] hbase git commit: HBASE-19506: The CellChunkMap index chunks are usually small, so in order to prevent memory underutilization, HBASE-19506 presents small chunks preallocated in a small pool

Posted by zh...@apache.org.
HBASE-19506: The CellChunkMap index chunks are usually small, so in order to prevent memory underutilization, HBASE-19506 presents small chunks preallocated in a small pool


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

Branch: refs/heads/HBASE-19064
Commit: 92d04d57516391376d34aaca6307e5906ffa4ae8
Parents: 79d9403
Author: anastas <an...@yahoo-inc.com>
Authored: Wed Feb 21 10:12:25 2018 +0200
Committer: anastas <an...@yahoo-inc.com>
Committed: Wed Feb 21 10:12:25 2018 +0200

----------------------------------------------------------------------
 .../regionserver/CellChunkImmutableSegment.java |  76 +++--
 .../hadoop/hbase/regionserver/CellChunkMap.java |  17 +-
 .../apache/hadoop/hbase/regionserver/Chunk.java |   4 +
 .../hadoop/hbase/regionserver/ChunkCreator.java | 324 ++++++++++++++-----
 .../regionserver/ImmutableMemStoreLAB.java      |  28 +-
 .../hadoop/hbase/regionserver/MemStoreLAB.java  |  22 +-
 .../hbase/regionserver/MemStoreLABImpl.java     |  74 +++--
 .../regionserver/TestCompactingMemStore.java    |   3 +-
 .../regionserver/TestMemStoreChunkPool.java     |   6 +-
 .../hbase/regionserver/TestMemStoreLAB.java     |   8 +-
 .../TestMemstoreLABWithoutPool.java             |   2 +-
 11 files changed, 377 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
index 53458f1..0eebfb5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
 
 
-
 /**
  * CellChunkImmutableSegment extends the API supported by a {@link Segment},
  * and {@link ImmutableSegment}. This immutable segment is working with CellSet with
@@ -43,6 +42,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
 
   public static final long DEEP_OVERHEAD_CCM =
       ImmutableSegment.DEEP_OVERHEAD + ClassSize.CELL_CHUNK_MAP;
+  public static final float INDEX_CHUNK_UNUSED_SPACE_PRECENTAGE = 0.1f;
 
   /////////////////////  CONSTRUCTORS  /////////////////////
   /**------------------------------------------------------------------------
@@ -135,20 +135,12 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
   private void initializeCellSet(int numOfCells, MemStoreSegmentsIterator iterator,
       MemStoreCompactionStrategy.Action action) {
 
-    // calculate how many chunks we will need for index
-    int chunkSize = ChunkCreator.getInstance().getChunkSize();
-    int numOfCellsInChunk = CellChunkMap.NUM_OF_CELL_REPS_IN_CHUNK;
-    int numberOfChunks = calculateNumberOfChunks(numOfCells, numOfCellsInChunk);
     int numOfCellsAfterCompaction = 0;
     int currentChunkIdx = 0;
     int offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER;
     int numUniqueKeys=0;
     Cell prev = null;
-    // all index Chunks are allocated from ChunkCreator
-    Chunk[] chunks = new Chunk[numberOfChunks];
-    for (int i=0; i < numberOfChunks; i++) {
-      chunks[i] = this.getMemStoreLAB().getNewExternalChunk();
-    }
+    Chunk[] chunks = allocIndexChunks(numOfCells);
     while (iterator.hasNext()) {        // the iterator hides the elimination logic for compaction
       boolean alreadyCopied = false;
       Cell c = iterator.next();
@@ -161,7 +153,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
         c = copyCellIntoMSLAB(c);
         alreadyCopied = true;
       }
-      if (offsetInCurentChunk + ClassSize.CELL_CHUNK_MAP_ENTRY > chunkSize) {
+      if (offsetInCurentChunk + ClassSize.CELL_CHUNK_MAP_ENTRY > chunks[currentChunkIdx].size) {
         currentChunkIdx++;              // continue to the next index chunk
         offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER;
       }
@@ -207,15 +199,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
       int numOfCells, KeyValueScanner segmentScanner, CellSet oldCellSet,
       MemStoreCompactionStrategy.Action action) {
     Cell curCell;
-    // calculate how many chunks we will need for metadata
-    int chunkSize = ChunkCreator.getInstance().getChunkSize();
-    int numOfCellsInChunk = CellChunkMap.NUM_OF_CELL_REPS_IN_CHUNK;
-    int numberOfChunks = calculateNumberOfChunks(numOfCells, numOfCellsInChunk);
-    // all index Chunks are allocated from ChunkCreator
-    Chunk[] chunks = new Chunk[numberOfChunks];
-    for (int i=0; i < numberOfChunks; i++) {
-      chunks[i] = this.getMemStoreLAB().getNewExternalChunk();
-    }
+    Chunk[] chunks = allocIndexChunks(numOfCells);
 
     int currentChunkIdx = 0;
     int offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER;
@@ -231,7 +215,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
           // are copied into MSLAB here.
           curCell = copyCellIntoMSLAB(curCell);
         }
-        if (offsetInCurentChunk + ClassSize.CELL_CHUNK_MAP_ENTRY > chunkSize) {
+        if (offsetInCurentChunk + ClassSize.CELL_CHUNK_MAP_ENTRY > chunks[currentChunkIdx].size) {
           // continue to the next metadata chunk
           currentChunkIdx++;
           offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER;
@@ -279,14 +263,58 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
     return offset;
   }
 
-  private int calculateNumberOfChunks(int numOfCells, int numOfCellsInChunk) {
-    int numberOfChunks = numOfCells/numOfCellsInChunk;
-    if(numOfCells%numOfCellsInChunk!=0) { // if cells cannot be divided evenly between chunks
+  private int calculateNumberOfChunks(int numOfCells, int chunkSize) {
+    int numOfCellsInChunk = calcNumOfCellsInChunk(chunkSize);
+    int numberOfChunks = numOfCells / numOfCellsInChunk;
+    if(numOfCells % numOfCellsInChunk != 0) { // if cells cannot be divided evenly between chunks
       numberOfChunks++;                   // add one additional chunk
     }
     return numberOfChunks;
   }
 
+  // Assuming we are going to use regular data chunks as index chunks,
+  // we check here how much free space will remain in the last allocated chunk
+  // (the least occupied one).
+  // If the percentage of its remaining free space is above the INDEX_CHUNK_UNUSED_SPACE
+  // threshold, then we will use index chunks (which are smaller) instead.
+  private ChunkCreator.ChunkType useIndexChunks(int numOfCells) {
+    int dataChunkSize = ChunkCreator.getInstance().getChunkSize();
+    int numOfCellsInChunk = calcNumOfCellsInChunk(dataChunkSize);
+    int cellsInLastChunk = numOfCells % numOfCellsInChunk;
+    if (cellsInLastChunk == 0) { // There is no free space in the last chunk and thus,
+      return ChunkCreator.ChunkType.DATA_CHUNK;               // no need to use index chunks.
+    } else {
+      int chunkSpace = dataChunkSize - ChunkCreator.SIZEOF_CHUNK_HEADER;
+      int freeSpaceInLastChunk = chunkSpace - cellsInLastChunk * ClassSize.CELL_CHUNK_MAP_ENTRY;
+      if (freeSpaceInLastChunk > INDEX_CHUNK_UNUSED_SPACE_PRECENTAGE * chunkSpace) {
+        return ChunkCreator.ChunkType.INDEX_CHUNK;
+      }
+      return ChunkCreator.ChunkType.DATA_CHUNK;
+    }
+  }
+
+  private int calcNumOfCellsInChunk(int chunkSize) {
+    int chunkSpace = chunkSize - ChunkCreator.SIZEOF_CHUNK_HEADER;
+    int numOfCellsInChunk = chunkSpace / ClassSize.CELL_CHUNK_MAP_ENTRY;
+    return numOfCellsInChunk;
+  }
+
+  private Chunk[] allocIndexChunks(int numOfCells) {
+    // Decide whether to use regular or small chunks and then
+    // calculate how many chunks we will need for index
+
+    ChunkCreator.ChunkType chunkType = useIndexChunks(numOfCells);
+    int chunkSize = ChunkCreator.getInstance().getChunkSize(chunkType);
+    int numberOfChunks = calculateNumberOfChunks(numOfCells, chunkSize);
+    // all index Chunks are allocated from ChunkCreator
+    Chunk[] chunks = new Chunk[numberOfChunks];
+    // all index Chunks are allocated from ChunkCreator
+    for (int i = 0; i < numberOfChunks; i++) {
+      chunks[i] = this.getMemStoreLAB().getNewExternalChunk(chunkType);
+    }
+    return chunks;
+  }
+
   private Cell copyCellIntoMSLAB(Cell cell) {
     // Take care for a special case when a cell is copied from on-heap to (probably off-heap) MSLAB.
     // The cell allocated as an on-heap JVM object (byte array) occupies slightly different

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
index a3b45c3..a2c1820 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
@@ -57,11 +57,10 @@ public class CellChunkMap extends CellFlatMap {
 
   private final Chunk[] chunks;             // the array of chunks, on which the index is based
 
-  // constant number of cell-representations in a chunk
+  // number of cell-representations in a chunk
+  // depends on the size of the chunks (may be index chunks or regular data chunks)
   // each chunk starts with its own ID following the cells data
-  public static final int NUM_OF_CELL_REPS_IN_CHUNK =
-      (ChunkCreator.getInstance().getChunkSize() - ChunkCreator.SIZEOF_CHUNK_HEADER) /
-          ClassSize.CELL_CHUNK_MAP_ENTRY;
+  private final int numOfCellRepsInChunk;
 
   /**
    * C-tor for creating CellChunkMap from existing Chunk array, which must be ordered
@@ -76,6 +75,12 @@ public class CellChunkMap extends CellFlatMap {
       Chunk[] chunks, int min, int max, boolean descending) {
     super(comparator, min, max, descending);
     this.chunks = chunks;
+    if (chunks != null && chunks.length != 0 && chunks[0] != null) {
+      this.numOfCellRepsInChunk = (chunks[0].size - ChunkCreator.SIZEOF_CHUNK_HEADER) /
+              ClassSize.CELL_CHUNK_MAP_ENTRY;
+    } else { // In case the chunks array was not allocated
+      this.numOfCellRepsInChunk = 0;
+    }
   }
 
   /* To be used by base (CellFlatMap) class only to create a sub-CellFlatMap
@@ -89,9 +94,9 @@ public class CellChunkMap extends CellFlatMap {
   @Override
   protected Cell getCell(int i) {
     // get the index of the relevant chunk inside chunk array
-    int chunkIndex = (i / NUM_OF_CELL_REPS_IN_CHUNK);
+    int chunkIndex = (i / numOfCellRepsInChunk);
     ByteBuffer block = chunks[chunkIndex].getData();// get the ByteBuffer of the relevant chunk
-    int j = i - chunkIndex * NUM_OF_CELL_REPS_IN_CHUNK; // get the index of the cell-representation
+    int j = i - chunkIndex * numOfCellRepsInChunk; // get the index of the cell-representation
 
     // find inside the offset inside the chunk holding the index, skip bytes for chunk id
     int offsetInBytes = ChunkCreator.SIZEOF_CHUNK_HEADER + j* ClassSize.CELL_CHUNK_MAP_ENTRY;

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
index 199298b..136efee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
@@ -89,6 +89,10 @@ public abstract class Chunk {
     return size > ChunkCreator.getInstance().getChunkSize();
   }
 
+  boolean isIndexChunk() {
+    return size == ChunkCreator.getInstance().getChunkSize(ChunkCreator.ChunkType.INDEX_CHUNK);
+  }
+
   /**
    * Actually claim the memory for this chunk. This should only be called from the thread that
    * constructed the chunk. It is thread-safe against other threads calling alloc(), who will block

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
index b2d4ba8..e62303e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
@@ -56,27 +56,58 @@ public class ChunkCreator {
   // the header size need to be changed in case chunk id size is changed
   public static final int SIZEOF_CHUNK_HEADER = Bytes.SIZEOF_INT;
 
+  /**
+   * Types of chunks, based on their sizes
+   */
+  public enum ChunkType {
+    // An index chunk is a small chunk, allocated from the index chunks pool.
+    // Its size is fixed and is 10% of the size of a data chunk.
+    INDEX_CHUNK,
+    // A data chunk is a regular chunk, allocated from the data chunks pool.
+    // Its size is fixed and given as input to the ChunkCreator c'tor.
+    DATA_CHUNK,
+    // A jumbo chunk isn't allocated from pool. Its size is bigger than the size of a
+    // data chunk, and is determined per chunk (meaning, there is no fixed jumbo size).
+    JUMBO_CHUNK
+  }
+
   // mapping from chunk IDs to chunks
   private Map<Integer, Chunk> chunkIdMap = new ConcurrentHashMap<Integer, Chunk>();
 
-  private final int chunkSize;
   private final boolean offheap;
   @VisibleForTesting
-  static ChunkCreator INSTANCE;
+  static ChunkCreator instance;
   @VisibleForTesting
   static boolean chunkPoolDisabled = false;
-  private MemStoreChunkPool pool;
+  private MemStoreChunkPool dataChunksPool;
+  private int chunkSize;
+  private MemStoreChunkPool indexChunksPool;
 
   @VisibleForTesting
   ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage,
-      float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
-    this.chunkSize = chunkSize;
+               float initialCountPercentage, HeapMemoryManager heapMemoryManager,
+               float indexChunkSizePercentage) {
     this.offheap = offheap;
-    this.pool = initializePool(globalMemStoreSize, poolSizePercentage, initialCountPercentage);
-    if (heapMemoryManager != null && this.pool != null) {
-      // Register with Heap Memory manager
-      heapMemoryManager.registerTuneObserver(this.pool);
-    }
+    this.chunkSize = chunkSize; // in case pools are not allocated
+    initializePools(chunkSize, globalMemStoreSize, poolSizePercentage, indexChunkSizePercentage,
+            initialCountPercentage, heapMemoryManager);
+  }
+
+  @VisibleForTesting
+  private void initializePools(int chunkSize, long globalMemStoreSize,
+                               float poolSizePercentage, float indexChunkSizePercentage,
+                               float initialCountPercentage,
+                               HeapMemoryManager heapMemoryManager) {
+    this.dataChunksPool = initializePool(globalMemStoreSize,
+            (1 - indexChunkSizePercentage) * poolSizePercentage,
+            initialCountPercentage, chunkSize, heapMemoryManager);
+    // The index chunks pool is needed only when the index type is CCM.
+    // Since the pools are not created at all when the index type isn't CCM,
+    // we don't need to check it here.
+    this.indexChunksPool = initializePool(globalMemStoreSize,
+            indexChunkSizePercentage * poolSizePercentage,
+            initialCountPercentage, (int) (indexChunkSizePercentage * chunkSize),
+            heapMemoryManager);
   }
 
   /**
@@ -90,18 +121,30 @@ public class ChunkCreator {
    * @return singleton MSLABChunkCreator
    */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
-      justification = "Method is called by single thread at the starting of RS")
+          justification = "Method is called by single thread at the starting of RS")
   @VisibleForTesting
   public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize,
-      float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
-    if (INSTANCE != null) return INSTANCE;
-    INSTANCE = new ChunkCreator(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
-        initialCountPercentage, heapMemoryManager);
-    return INSTANCE;
+                                        float poolSizePercentage, float initialCountPercentage,
+                                        HeapMemoryManager heapMemoryManager) {
+    if (instance != null) {
+      return instance;
+    }
+    instance = new ChunkCreator(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
+            initialCountPercentage, heapMemoryManager,
+            MemStoreLABImpl.INDEX_CHUNK_PERCENTAGE_DEFAULT);
+    return instance;
   }
 
   static ChunkCreator getInstance() {
-    return INSTANCE;
+    return instance;
+  }
+
+  /**
+   * Creates and inits a chunk. The default implementation for a specific chunk size.
+   * @return the chunk that was initialized
+   */
+  Chunk getChunk(ChunkType chunkType) {
+    return getChunk(CompactingMemStore.IndexType.ARRAY_MAP, chunkType);
   }
 
   /**
@@ -109,15 +152,37 @@ public class ChunkCreator {
    * @return the chunk that was initialized
    */
   Chunk getChunk() {
-    return getChunk(CompactingMemStore.IndexType.ARRAY_MAP, chunkSize);
+    return getChunk(CompactingMemStore.IndexType.ARRAY_MAP, ChunkType.DATA_CHUNK);
   }
 
   /**
-   * Creates and inits a chunk. The default implementation for specific index type.
+   * Creates and inits a chunk. The default implementation for a specific index type.
    * @return the chunk that was initialized
    */
   Chunk getChunk(CompactingMemStore.IndexType chunkIndexType) {
-    return getChunk(chunkIndexType, chunkSize);
+    return getChunk(chunkIndexType, ChunkType.DATA_CHUNK);
+  }
+
+  /**
+   * Creates and inits a chunk with specific index type and type.
+   * @return the chunk that was initialized
+   */
+  Chunk getChunk(CompactingMemStore.IndexType chunkIndexType, ChunkType chunkType) {
+    switch (chunkType) {
+      case INDEX_CHUNK:
+        if (indexChunksPool != null) {
+          return getChunk(chunkIndexType, indexChunksPool.getChunkSize());
+        }
+      case DATA_CHUNK:
+        if (dataChunksPool == null) {
+          return getChunk(chunkIndexType, chunkSize);
+        } else {
+          return getChunk(chunkIndexType, dataChunksPool.getChunkSize());
+        }
+      default:
+        throw new IllegalArgumentException(
+                "chunkType must either be INDEX_CHUNK or DATA_CHUNK");
+    }
   }
 
   /**
@@ -128,18 +193,28 @@ public class ChunkCreator {
    */
   Chunk getChunk(CompactingMemStore.IndexType chunkIndexType, int size) {
     Chunk chunk = null;
-    // if we have pool and this is not jumbo chunk (when size != chunkSize this is jumbo chunk)
-    if ((pool != null) && (size == chunkSize)) {
+    MemStoreChunkPool pool = null;
+
+    // if the size is suitable for one of the pools
+    if (dataChunksPool != null && size == dataChunksPool.getChunkSize()) {
+      pool = dataChunksPool;
+    } else if (indexChunksPool != null && size == indexChunksPool.getChunkSize()) {
+      pool = indexChunksPool;
+    }
+
+    // if we have a pool
+    if (pool != null) {
       //  the pool creates the chunk internally. The chunk#init() call happens here
-      chunk = this.pool.getChunk();
+      chunk = pool.getChunk();
       // the pool has run out of maxCount
       if (chunk == null) {
         if (LOG.isTraceEnabled()) {
-          LOG.trace("Chunk pool full (maxCount={}); creating chunk offheap.",
-              this.pool.getMaxCount());
+          LOG.trace("The chunk pool is full. Reached maxCount= " + pool.getMaxCount()
+                  + ". Creating chunk onheap.");
         }
       }
     }
+
     if (chunk == null) {
       // the second parameter explains whether CellChunkMap index is requested,
       // in that case, put allocated on demand chunk mapping into chunkIdMap
@@ -156,18 +231,18 @@ public class ChunkCreator {
    * Creates and inits a chunk of a special size, bigger than a regular chunk size.
    * Such a chunk will never come from pool and will always be on demand allocated.
    * @return the chunk that was initialized
-   * @param chunkIndexType whether the requested chunk is going to be used with CellChunkMap index
    * @param jumboSize the special size to be used
    */
-  Chunk getJumboChunk(CompactingMemStore.IndexType chunkIndexType, int jumboSize) {
-    if (jumboSize <= chunkSize) {
-      LOG.warn("Jumbo chunk size=" + jumboSize + " must be more than regular chunk size="
-          + chunkSize + "; converting to regular chunk.");
-      return getChunk(chunkIndexType,chunkSize);
+  Chunk getJumboChunk(int jumboSize) {
+    int allocSize = jumboSize + SIZEOF_CHUNK_HEADER;
+    if (allocSize <= dataChunksPool.getChunkSize()) {
+      LOG.warn("Jumbo chunk size " + jumboSize + " must be more than regular chunk size "
+              + dataChunksPool.getChunkSize() + ". Converting to regular chunk.");
+      return getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
     }
-    // the size of the allocation includes
-    // both the size requested and a place for the Chunk's header
-    return getChunk(chunkIndexType, jumboSize + SIZEOF_CHUNK_HEADER);
+    // the new chunk is going to hold the jumbo cell data and needs to be referenced by
+    // a strong map. Therefore the CCM index type
+    return getChunk(CompactingMemStore.IndexType.CHUNK_MAP, allocSize);
   }
 
   /**
@@ -196,21 +271,21 @@ public class ChunkCreator {
 
   // Chunks from pool are created covered with strong references anyway
   // TODO: change to CHUNK_MAP if it is generally defined
-  private Chunk createChunkForPool() {
-    return createChunk(true, CompactingMemStore.IndexType.ARRAY_MAP, chunkSize);
+  private Chunk createChunkForPool(CompactingMemStore.IndexType chunkIndexType, int chunkSize) {
+    if (chunkSize != dataChunksPool.getChunkSize() &&
+            chunkSize != indexChunksPool.getChunkSize()) {
+      return null;
+    }
+    return createChunk(true, chunkIndexType, chunkSize);
   }
 
   @VisibleForTesting
-  // Used to translate the ChunkID into a chunk ref
+    // Used to translate the ChunkID into a chunk ref
   Chunk getChunk(int id) {
     // can return null if chunk was never mapped
     return chunkIdMap.get(id);
   }
 
-  int getChunkSize() {
-    return this.chunkSize;
-  }
-
   boolean isOffheap() {
     return this.offheap;
   }
@@ -224,8 +299,8 @@ public class ChunkCreator {
   }
 
   @VisibleForTesting
-  // the chunks in the chunkIdMap may already be released so we shouldn't relay
-  // on this counting for strong correctness. This method is used only in testing.
+    // the chunks in the chunkIdMap may already be released so we shouldn't relay
+    // on this counting for strong correctness. This method is used only in testing.
   int numberOfMappedChunks() {
     return this.chunkIdMap.size();
   }
@@ -243,6 +318,7 @@ public class ChunkCreator {
    * collection on JVM.
    */
   private  class MemStoreChunkPool implements HeapMemoryTuneObserver {
+    private final int chunkSize;
     private int maxCount;
 
     // A queue of reclaimed chunks
@@ -256,21 +332,22 @@ public class ChunkCreator {
     private final AtomicLong chunkCount = new AtomicLong();
     private final LongAdder reusedChunkCount = new LongAdder();
 
-    MemStoreChunkPool(int maxCount, int initialCount, float poolSizePercentage) {
+    MemStoreChunkPool(int chunkSize, int maxCount, int initialCount, float poolSizePercentage) {
+      this.chunkSize = chunkSize;
       this.maxCount = maxCount;
       this.poolSizePercentage = poolSizePercentage;
       this.reclaimedChunks = new LinkedBlockingQueue<>();
       for (int i = 0; i < initialCount; i++) {
-        Chunk chunk = createChunkForPool();
+        Chunk chunk = createChunk(true, CompactingMemStore.IndexType.ARRAY_MAP, chunkSize);
         chunk.init();
         reclaimedChunks.add(chunk);
       }
       chunkCount.set(initialCount);
       final String n = Thread.currentThread().getName();
       scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
-          .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
+              .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
       this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
-          statThreadPeriod, TimeUnit.SECONDS);
+              statThreadPeriod, TimeUnit.SECONDS);
     }
 
     /**
@@ -283,6 +360,10 @@ public class ChunkCreator {
      * @see #putbackChunks(Chunk)
      */
     Chunk getChunk() {
+      return getChunk(CompactingMemStore.IndexType.ARRAY_MAP);
+    }
+
+    Chunk getChunk(CompactingMemStore.IndexType chunkIndexType) {
       Chunk chunk = reclaimedChunks.poll();
       if (chunk != null) {
         chunk.reset();
@@ -293,7 +374,7 @@ public class ChunkCreator {
           long created = this.chunkCount.get();
           if (created < this.maxCount) {
             if (this.chunkCount.compareAndSet(created, created + 1)) {
-              chunk = createChunkForPool();
+              chunk = createChunkForPool(chunkIndexType, chunkSize);
               break;
             }
           } else {
@@ -304,6 +385,10 @@ public class ChunkCreator {
       return chunk;
     }
 
+    int getChunkSize() {
+      return chunkSize;
+    }
+
     /**
      * Add the chunks to the pool, when the pool achieves the max size, it will skip the remaining
      * chunks
@@ -311,7 +396,7 @@ public class ChunkCreator {
      */
     private void putbackChunks(Chunk c) {
       int toAdd = this.maxCount - reclaimedChunks.size();
-      if (c.isFromPool() && toAdd > 0) {
+      if (c.isFromPool() && c.size == chunkSize && toAdd > 0) {
         reclaimedChunks.add(c);
       } else {
         // remove the chunk (that is not going to pool)
@@ -336,10 +421,11 @@ public class ChunkCreator {
         long created = chunkCount.get();
         long reused = reusedChunkCount.sum();
         long total = created + reused;
-        LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
-            + ",created chunk count=" + created
-            + ",reused chunk count=" + reused
-            + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
+        LOG.debug("Stats (chunk size=" + chunkSize + "): "
+                + "current pool size=" + reclaimedChunks.size()
+                + ",created chunk count=" + created
+                + ",reused chunk count=" + reused
+                + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
                 (float) reused / (float) total, 2)));
       }
     }
@@ -356,7 +442,7 @@ public class ChunkCreator {
         return;
       }
       int newMaxCount =
-          (int) (newMemstoreSize * poolSizePercentage / getChunkSize());
+              (int) (newMemstoreSize * poolSizePercentage / getChunkSize());
       if (newMaxCount != this.maxCount) {
         // We need an adjustment in the chunks numbers
         if (newMaxCount > this.maxCount) {
@@ -387,7 +473,8 @@ public class ChunkCreator {
   }
 
   private MemStoreChunkPool initializePool(long globalMemStoreSize, float poolSizePercentage,
-      float initialCountPercentage) {
+                                           float initialCountPercentage, int chunkSize,
+                                           HeapMemoryManager heapMemoryManager) {
     if (poolSizePercentage <= 0) {
       LOG.info("PoolSizePercentage is less than 0. So not using pool");
       return null;
@@ -397,47 +484,90 @@ public class ChunkCreator {
     }
     if (poolSizePercentage > 1.0) {
       throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
+              MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
     }
-    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / getChunkSize());
+    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / chunkSize);
     if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
       throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
+              MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
     }
     int initialCount = (int) (initialCountPercentage * maxCount);
-    LOG.info("Allocating MemStoreChunkPool with chunk size="
-        + StringUtils.byteDesc(getChunkSize()) + ", max count=" + maxCount
-        + ", initial count=" + initialCount);
-    return new MemStoreChunkPool(maxCount, initialCount, poolSizePercentage);
+    LOG.info("Allocating MemStoreChunkPool with chunk size "
+            + StringUtils.byteDesc(chunkSize) + ", max count " + maxCount
+            + ", initial count " + initialCount);
+    MemStoreChunkPool memStoreChunkPool = new MemStoreChunkPool(chunkSize, maxCount,
+            initialCount, poolSizePercentage);
+    if (heapMemoryManager != null && memStoreChunkPool != null) {
+      // Register with Heap Memory manager
+      heapMemoryManager.registerTuneObserver(memStoreChunkPool);
+    }
+    return memStoreChunkPool;
   }
 
   @VisibleForTesting
   int getMaxCount() {
-    if (pool != null) {
-      return pool.getMaxCount();
+    return getMaxCount(ChunkType.DATA_CHUNK);
+  }
+
+  @VisibleForTesting
+  int getMaxCount(ChunkType chunkType) {
+    switch (chunkType) {
+      case INDEX_CHUNK:
+        if (indexChunksPool != null) {
+          return indexChunksPool.getMaxCount();
+        }
+        break;
+      case DATA_CHUNK:
+        if (dataChunksPool != null) {
+          return dataChunksPool.getMaxCount();
+        }
+        break;
+      default:
+        throw new IllegalArgumentException(
+                "chunkType must either be INDEX_CHUNK or DATA_CHUNK");
     }
+
     return 0;
   }
 
   @VisibleForTesting
   int getPoolSize() {
-    if (pool != null) {
-      return pool.reclaimedChunks.size();
+    return getPoolSize(ChunkType.DATA_CHUNK);
+  }
+
+  @VisibleForTesting
+  int getPoolSize(ChunkType chunkType) {
+    switch (chunkType) {
+      case INDEX_CHUNK:
+        if (indexChunksPool != null) {
+          return indexChunksPool.reclaimedChunks.size();
+        }
+        break;
+      case DATA_CHUNK:
+        if (dataChunksPool != null) {
+          return dataChunksPool.reclaimedChunks.size();
+        }
+        break;
+      default:
+        throw new IllegalArgumentException(
+                "chunkType must either be INDEX_CHUNK or DATA_CHUNK");
     }
     return 0;
   }
 
   @VisibleForTesting
   boolean isChunkInPool(int chunkId) {
-    if (pool != null) {
-      // chunks that are from pool will return true chunk reference not null
-      Chunk c = getChunk(chunkId);
-      if (c==null) {
-        return false;
-      }
-      return pool.reclaimedChunks.contains(c);
+    Chunk c = getChunk(chunkId);
+    if (c==null) {
+      return false;
     }
 
+    // chunks that are from pool will return true chunk reference not null
+    if (dataChunksPool != null && dataChunksPool.reclaimedChunks.contains(c)) {
+      return true;
+    } else if (indexChunksPool != null && indexChunksPool.reclaimedChunks.contains(c)) {
+      return true;
+    }
     return false;
   }
 
@@ -446,31 +576,58 @@ public class ChunkCreator {
    */
   @VisibleForTesting
   void clearChunksInPool() {
-    if (pool != null) {
-      pool.reclaimedChunks.clear();
+    if (dataChunksPool != null) {
+      dataChunksPool.reclaimedChunks.clear();
+    }
+    if (indexChunksPool != null) {
+      indexChunksPool.reclaimedChunks.clear();
+    }
+  }
+
+  int getChunkSize() {
+    return getChunkSize(ChunkType.DATA_CHUNK);
+  }
+
+  int getChunkSize(ChunkType chunkType) {
+    switch (chunkType) {
+      case INDEX_CHUNK:
+        if (indexChunksPool != null) {
+          return indexChunksPool.getChunkSize();
+        }
+      case DATA_CHUNK:
+        if (dataChunksPool != null) {
+          return dataChunksPool.getChunkSize();
+        } else { // When pools are empty
+          return chunkSize;
+        }
+      default:
+        throw new IllegalArgumentException(
+                "chunkType must either be INDEX_CHUNK or DATA_CHUNK");
     }
   }
 
   synchronized void putbackChunks(Set<Integer> chunks) {
     // if there is no pool just try to clear the chunkIdMap in case there is something
-    if ( pool == null ) {
+    if (dataChunksPool == null && indexChunksPool == null) {
       this.removeChunks(chunks);
       return;
     }
 
-    // if there is pool, go over all chunk IDs that came back, the chunks may be from pool or not
+    // if there is a pool, go over all chunk IDs that came back, the chunks may be from pool or not
     for (int chunkID : chunks) {
       // translate chunk ID to chunk, if chunk initially wasn't in pool
       // this translation will (most likely) return null
       Chunk chunk = ChunkCreator.this.getChunk(chunkID);
       if (chunk != null) {
-        // Jumbo chunks are covered with chunkIdMap, but are not from pool, so such a chunk should
-        // be released here without going to pool.
-        // Removing them from chunkIdMap will cause their removal by the GC.
-        if (chunk.isJumbo()) {
-          this.removeChunk(chunkID);
+        if (chunk.isFromPool() && chunk.isIndexChunk()) {
+          indexChunksPool.putbackChunks(chunk);
+        } else if (chunk.isFromPool() && chunk.size == dataChunksPool.getChunkSize()) {
+          dataChunksPool.putbackChunks(chunk);
         } else {
-          pool.putbackChunks(chunk);
+          // chunks which are not from one of the pools
+          // should be released without going to the pools.
+          // Removing them from chunkIdMap will cause their removal by the GC.
+          this.removeChunk(chunkID);
         }
       }
       // if chunk is null, it was never covered by the chunkIdMap (and so wasn't in pool also),
@@ -480,3 +637,4 @@ public class ChunkCreator {
   }
 
 }
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java
index 71648a0..0ce52b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java
@@ -61,32 +61,26 @@ public class ImmutableMemStoreLAB implements MemStoreLAB {
     return mslab.forceCopyOfBigCellInto(cell);
   }
 
-  /* Creating chunk to be used as index chunk in CellChunkMap, part of the chunks array.
-  ** Returning a new chunk, without replacing current chunk,
+  /* Returning a new pool chunk, without replacing current chunk,
   ** meaning MSLABImpl does not make the returned chunk as CurChunk.
   ** The space on this chunk will be allocated externally.
-  ** The interface is only for external callers
+  ** The interface is only for external callers.
   */
   @Override
-  public Chunk getNewExternalChunk() {
+  public Chunk getNewExternalChunk(ChunkCreator.ChunkType chunkType) {
     MemStoreLAB mslab = this.mslabs.get(0);
-    return mslab.getNewExternalChunk();
+    return mslab.getNewExternalChunk(chunkType);
   }
 
-  /* Creating chunk to be used as data chunk in CellChunkMap.
-  ** This chunk is bigger the normal constant chunk size, and thus called JumboChunk it is used for
-  ** jumbo cells (which size is bigger than normal chunks).
-  ** Jumbo Chunks are needed only for CCM and thus are created only in
-  ** CompactingMemStore.IndexType.CHUNK_MAP type.
-  ** Returning a new chunk, without replacing current chunk,
-  ** meaning MSLABImpl does not make the returned chunk as CurChunk.
-  ** The space on this chunk will be allocated externally.
-  ** The interface is only for external callers
-  */
+  /* Returning a new chunk, without replacing current chunk,
+   ** meaning MSLABImpl does not make the returned chunk as CurChunk.
+   ** The space on this chunk will be allocated externally.
+   ** The interface is only for external callers.
+   */
   @Override
-  public Chunk getNewExternalJumboChunk(int size) {
+  public Chunk getNewExternalChunk(int size) {
     MemStoreLAB mslab = this.mslabs.get(0);
-    return mslab.getNewExternalJumboChunk(size);
+    return mslab.getNewExternalChunk(size);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
index 8b77981..90cf932 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
@@ -52,6 +52,8 @@ public interface MemStoreLAB {
 
   String CHUNK_SIZE_KEY = "hbase.hregion.memstore.mslab.chunksize";
   int CHUNK_SIZE_DEFAULT = 2048 * 1024;
+  String INDEX_CHUNK_PERCENTAGE_KEY = "hbase.hregion.memstore.mslab.indexchunksize";
+  float INDEX_CHUNK_PERCENTAGE_DEFAULT = 0.1f;
   String MAX_ALLOC_KEY = "hbase.hregion.memstore.mslab.max.allocation";
   int MAX_ALLOC_DEFAULT = 256 * 1024; // allocs bigger than this don't go through
                                                    // allocator
@@ -94,25 +96,19 @@ public interface MemStoreLAB {
    */
   void decScannerCount();
 
-  /* Creating chunk to be used as index chunk in CellChunkMap, part of the chunks array.
-  ** Returning a new chunk, without replacing current chunk,
+  /* Returning a new pool chunk, without replacing current chunk,
   ** meaning MSLABImpl does not make the returned chunk as CurChunk.
   ** The space on this chunk will be allocated externally.
-  ** The interface is only for external callers
+  ** The interface is only for external callers.
   */
-  Chunk getNewExternalChunk();
-
-  /* Creating chunk to be used as data chunk in CellChunkMap.
-  ** This chunk is bigger than normal constant chunk size, and thus called JumboChunk it is used for
-  ** jumbo cells (which size is bigger than normal chunks).
-  ** Jumbo Chunks are needed only for CCM and thus are created only in
-  ** CompactingMemStore.IndexType.CHUNK_MAP type.
-  ** Returning a new chunk, without replacing current chunk,
+  Chunk getNewExternalChunk(ChunkCreator.ChunkType chunkType);
+
+  /* Returning a new chunk, without replacing current chunk,
   ** meaning MSLABImpl does not make the returned chunk as CurChunk.
   ** The space on this chunk will be allocated externally.
-  ** The interface is only for external callers
+  ** The interface is only for external callers.
   */
-  Chunk getNewExternalJumboChunk(int size);
+  Chunk getNewExternalChunk(int size);
 
   static MemStoreLAB newInstance(Configuration conf) {
     MemStoreLAB memStoreLAB = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index 4ff0480..0db0fd9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -67,14 +67,14 @@ public class MemStoreLABImpl implements MemStoreLAB {
 
   static final Logger LOG = LoggerFactory.getLogger(MemStoreLABImpl.class);
 
-  private AtomicReference<Chunk> curChunk = new AtomicReference<>();
+  private AtomicReference<Chunk> currChunk = new AtomicReference<>();
   // Lock to manage multiple handlers requesting for a chunk
   private ReentrantLock lock = new ReentrantLock();
 
   // A set of chunks contained by this memstore LAB
   @VisibleForTesting
   Set<Integer> chunks = new ConcurrentSkipListSet<Integer>();
-  private final int chunkSize;
+  private final int dataChunkSize;
   private final int maxAlloc;
   private final ChunkCreator chunkCreator;
   private final CompactingMemStore.IndexType idxType; // what index is used for corresponding segment
@@ -94,11 +94,11 @@ public class MemStoreLABImpl implements MemStoreLAB {
   }
 
   public MemStoreLABImpl(Configuration conf) {
-    chunkSize = conf.getInt(CHUNK_SIZE_KEY, CHUNK_SIZE_DEFAULT);
+    dataChunkSize = conf.getInt(CHUNK_SIZE_KEY, CHUNK_SIZE_DEFAULT);
     maxAlloc = conf.getInt(MAX_ALLOC_KEY, MAX_ALLOC_DEFAULT);
     this.chunkCreator = ChunkCreator.getInstance();
     // if we don't exclude allocations >CHUNK_SIZE, we'd infiniteloop on one!
-    Preconditions.checkArgument(maxAlloc <= chunkSize,
+    Preconditions.checkArgument(maxAlloc <= dataChunkSize,
         MAX_ALLOC_KEY + " must be less than " + CHUNK_SIZE_KEY);
 
     // if user requested to work with MSLABs (whether on- or off-heap), then the
@@ -120,14 +120,14 @@ public class MemStoreLABImpl implements MemStoreLAB {
    */
   @Override
   public Cell forceCopyOfBigCellInto(Cell cell) {
-    int size = KeyValueUtil.length(cell);
+    int size = KeyValueUtil.length(cell) + ChunkCreator.SIZEOF_CHUNK_HEADER;
     Preconditions.checkArgument(size >= 0, "negative size");
-    if (size <= chunkSize) {
+    if (size <= dataChunkSize) {
       // Using copyCellInto for cells which are bigger than the original maxAlloc
-      Cell newCell = copyCellInto(cell, chunkSize);
+      Cell newCell = copyCellInto(cell, dataChunkSize);
       return newCell;
     } else {
-      Chunk c = getNewExternalJumboChunk(size);
+      Chunk c = getNewExternalChunk(size);
       int allocOffset = c.alloc(size);
       return copyToChunkCell(cell, c.getData(), allocOffset, size);
     }
@@ -240,7 +240,7 @@ public class MemStoreLABImpl implements MemStoreLAB {
    * @return true if we won the race to retire the chunk
    */
   private void tryRetireChunk(Chunk c) {
-    curChunk.compareAndSet(c, null);
+    currChunk.compareAndSet(c, null);
     // If the CAS succeeds, that means that we won the race
     // to retire the chunk. We could use this opportunity to
     // update metrics on external fragmentation.
@@ -255,7 +255,8 @@ public class MemStoreLABImpl implements MemStoreLAB {
    */
   private Chunk getOrMakeChunk() {
     // Try to get the chunk
-    Chunk c = curChunk.get();
+    Chunk c;
+    c = currChunk.get();
     if (c != null) {
       return c;
     }
@@ -265,14 +266,14 @@ public class MemStoreLABImpl implements MemStoreLAB {
     if (lock.tryLock()) {
       try {
         // once again check inside the lock
-        c = curChunk.get();
+        c = currChunk.get();
         if (c != null) {
           return c;
         }
         c = this.chunkCreator.getChunk(idxType);
         if (c != null) {
           // set the curChunk. No need of CAS as only one thread will be here
-          curChunk.set(c);
+          currChunk.set(c);
           chunks.add(c.getId());
           return c;
         }
@@ -283,38 +284,41 @@ public class MemStoreLABImpl implements MemStoreLAB {
     return null;
   }
 
-  /* Creating chunk to be used as index chunk in CellChunkMap, part of the chunks array.
-  ** Returning a new chunk, without replacing current chunk,
+  /* Returning a new pool chunk, without replacing current chunk,
   ** meaning MSLABImpl does not make the returned chunk as CurChunk.
   ** The space on this chunk will be allocated externally.
-  ** The interface is only for external callers
+  ** The interface is only for external callers.
   */
   @Override
-  public Chunk getNewExternalChunk() {
-    // the new chunk is going to be part of the chunk array and will always be referenced
-    Chunk c = this.chunkCreator.getChunk();
-    chunks.add(c.getId());
-    return c;
+  public Chunk getNewExternalChunk(ChunkCreator.ChunkType chunkType) {
+    switch (chunkType) {
+      case INDEX_CHUNK:
+      case DATA_CHUNK:
+        Chunk c = this.chunkCreator.getChunk(chunkType);
+        chunks.add(c.getId());
+        return c;
+      case JUMBO_CHUNK: // a jumbo chunk doesn't have a fixed size
+      default:
+        return null;
+    }
   }
 
-  /* Creating chunk to be used as data chunk in CellChunkMap.
-  ** This chunk is bigger than normal constant chunk size, and thus called JumboChunk.
-  ** JumboChunk is used for jumbo cell (which size is bigger than normal chunk). It is allocated
-  ** once per cell. So even if there is space this is not reused.
-  ** Jumbo Chunks are used only for CCM and thus are created only in
-  ** CompactingMemStore.IndexType.CHUNK_MAP type.
-  ** Returning a new chunk, without replacing current chunk,
+  /* Returning a new chunk, without replacing current chunk,
   ** meaning MSLABImpl does not make the returned chunk as CurChunk.
   ** The space on this chunk will be allocated externally.
-  ** The interface is only for external callers
+  ** The interface is only for external callers.
+  ** Chunks from pools are not allocated from here, since they have fixed sizes
   */
   @Override
-  public Chunk getNewExternalJumboChunk(int size) {
-    // the new chunk is going to hold the jumbo cell data and need to be referenced by a strong map
-    // thus giving the CCM index type
-    Chunk c = this.chunkCreator.getJumboChunk(CompactingMemStore.IndexType.CHUNK_MAP, size);
-    chunks.add(c.getId());
-    return c;
+  public Chunk getNewExternalChunk(int size) {
+    int allocSize = size + ChunkCreator.getInstance().SIZEOF_CHUNK_HEADER;
+    if (allocSize <= ChunkCreator.getInstance().getChunkSize()) {
+      return getNewExternalChunk(ChunkCreator.ChunkType.DATA_CHUNK);
+    } else {
+      Chunk c = this.chunkCreator.getJumboChunk(size);
+      chunks.add(c.getId());
+      return c;
+    }
   }
 
   @Override
@@ -329,7 +333,7 @@ public class MemStoreLABImpl implements MemStoreLAB {
 
   @VisibleForTesting
   Chunk getCurrentChunk() {
-    return this.curChunk.get();
+    return currChunk.get();
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index 505c2f0..5cbfff9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -118,7 +118,8 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
     chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
-        globalMemStoreLimit, 0.4f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+        globalMemStoreLimit, 0.4f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT,
+            null);
     assertTrue(chunkCreator != null);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
index 891e835..4f3de36 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
@@ -233,10 +233,10 @@ public class TestMemStoreChunkPool {
     final int chunkSize = 40;
     final int valSize = 7;
     ChunkCreator oldCreator = ChunkCreator.getInstance();
-    ChunkCreator newCreator = new ChunkCreator(chunkSize, false, 400, 1, 0.5f, null);
+    ChunkCreator newCreator = new ChunkCreator(chunkSize, false, 400, 1, 0.5f, null, 0);
     assertEquals(initialCount, newCreator.getPoolSize());
     assertEquals(maxCount, newCreator.getMaxCount());
-    ChunkCreator.INSTANCE = newCreator;// Replace the global ref with the new one we created.
+    ChunkCreator.instance = newCreator;// Replace the global ref with the new one we created.
                                              // Used it for the testing. Later in finally we put
                                              // back the original
     final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
@@ -265,7 +265,7 @@ public class TestMemStoreChunkPool {
       t3.join();
       assertTrue(newCreator.getPoolSize() <= maxCount);
     } finally {
-      ChunkCreator.INSTANCE = oldCreator;
+      ChunkCreator.instance = oldCreator;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
index 52491e7..ef4ad69 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
@@ -206,8 +206,8 @@ public class TestMemStoreLAB {
       MemStoreLABImpl mslab = new MemStoreLABImpl();
       // by default setting, there should be no chunks initialized in the pool
       assertTrue(mslab.getPooledChunks().isEmpty());
-      oldInstance = ChunkCreator.INSTANCE;
-      ChunkCreator.INSTANCE = null;
+      oldInstance = ChunkCreator.instance;
+      ChunkCreator.instance = null;
       // reset mslab with chunk pool
       Configuration conf = HBaseConfiguration.create();
       conf.setDouble(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.1);
@@ -251,7 +251,7 @@ public class TestMemStoreLAB {
       }
       // none of the chunkIds would have been returned back
       assertTrue("All the chunks must have been cleared",
-          ChunkCreator.INSTANCE.numberOfMappedChunks() != 0);
+          ChunkCreator.instance.numberOfMappedChunks() != 0);
       int pooledChunksNum = mslab.getPooledChunks().size();
       // close the mslab
       mslab.close();
@@ -261,7 +261,7 @@ public class TestMemStoreLAB {
           + " after mslab closed but actually: " + (pooledChunksNum-queueLength),
           pooledChunksNum-queueLength == 0);
     } finally {
-      ChunkCreator.INSTANCE = oldInstance;
+      ChunkCreator.instance = oldInstance;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
index 0813a8c..e2da5d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
@@ -148,7 +148,7 @@ public class TestMemstoreLABWithoutPool {
     }
     // all of the chunkIds would have been returned back
     assertTrue("All the chunks must have been cleared",
-        ChunkCreator.INSTANCE.numberOfMappedChunks() == 0);
+        ChunkCreator.instance.numberOfMappedChunks() == 0);
   }
 
   private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String threadName,


[03/50] [abbrv] hbase git commit: HBASE-20006 TestRestoreSnapshotFromClientWithRegionReplicas is flakey Remove assert in splittableregionprocedure. It was in the prepare. Was causing fail in legit case where a region split follows a table split BEFORE th

Posted by zh...@apache.org.
HBASE-20006 TestRestoreSnapshotFromClientWithRegionReplicas is flakey
Remove assert in splittableregionprocedure. It was in the prepare.
Was causing fail in legit case where a region split follows a
table split BEFORE the parent has been GC'd. The region split
finds the parent in SPLIT state which is right. The assert was
having us fail. No need.

Also disabled TestHTrace since not supported in 2.0.0 and flakey.


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

Branch: refs/heads/HBASE-19064
Commit: 40f8d20cf7b297a9324319190d03d93563230d6e
Parents: 01a2869
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 15 20:43:47 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 15 20:46:32 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/assignment/SplitTableRegionProcedure.java  | 1 -
 .../main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java | 2 +-
 .../test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java   | 2 ++
 3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/40f8d20c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index be0741d..c139566 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -225,7 +225,6 @@ public class SplitTableRegionProcedure
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION);
           break;
         } else {
-          assert isFailed() : "split region should have an exception here";
           return Flow.NO_MORE_STATE;
         }
       case SPLIT_TABLE_REGION_PRE_OPERATION:

http://git-wip-us.apache.org/repos/asf/hbase/blob/40f8d20c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
index a32a493..58f8f81 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
@@ -120,7 +120,7 @@ public class StoreUtils {
     if (comparator.compareRows(midKey, firstKey) == 0 ||
         comparator.compareRows(midKey, lastKey) == 0) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("cannot split because midkey is the same as first or last row");
+        LOG.debug("cannot split {} because midkey is the same as first or last row", file);
       }
       return Optional.empty();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/40f8d20c/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
index c39afdd..4513dd2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
@@ -38,6 +38,7 @@ import org.apache.htrace.core.TraceScope;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -45,6 +46,7 @@ import org.junit.rules.TestName;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
+@Ignore // We don't support htrace in hbase-2.0.0 and this flakey is a little flakey.
 @Category({MiscTests.class, MediumTests.class})
 public class TestHTraceHooks {
 


[29/50] [abbrv] hbase git commit: HBASE-20039 MR tests out to hbase-mapreduce mobile

Posted by zh...@apache.org.
HBASE-20039 MR tests out to hbase-mapreduce mobile


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

Branch: refs/heads/HBASE-19064
Commit: 5d994a24fc570ee5156df0d1356cb7b12305fb78
Parents: 401227b
Author: Mike Drob <md...@apache.org>
Authored: Wed Feb 21 10:53:42 2018 -0600
Committer: Mike Drob <md...@apache.org>
Committed: Wed Feb 21 14:54:30 2018 -0600

----------------------------------------------------------------------
 .../mapreduce/TestHBaseMRTestingUtility.java    | 73 ++++++++++++++++++++
 .../hadoop/hbase/TestHBaseTestingUtility.java   | 32 ---------
 2 files changed, 73 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5d994a24/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java
new file mode 100644
index 0000000..2467dca
--- /dev/null
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java
@@ -0,0 +1,73 @@
+/**
+ * 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.mapreduce;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MapReduceTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({MapReduceTests.class, LargeTests.class})
+public class TestHBaseMRTestingUtility {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestHBaseMRTestingUtility.class);
+
+  @Test
+  public void testMRYarnConfigsPopulation() throws IOException {
+    Map<String, String> dummyProps = new HashMap<>();
+    dummyProps.put("mapreduce.jobtracker.address", "dummyhost:11234");
+    dummyProps.put("yarn.resourcemanager.address", "dummyhost:11235");
+    dummyProps.put("mapreduce.jobhistory.address", "dummyhost:11236");
+    dummyProps.put("yarn.resourcemanager.scheduler.address", "dummyhost:11237");
+    dummyProps.put("mapreduce.jobhistory.webapp.address", "dummyhost:11238");
+    dummyProps.put("yarn.resourcemanager.webapp.address", "dummyhost:11239");
+
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+
+    // populate the mr props to the Configuration instance
+    for (Map.Entry<String, String> entry : dummyProps.entrySet()) {
+      hbt.getConfiguration().set(entry.getKey(), entry.getValue());
+    }
+
+    for (Map.Entry<String,String> entry : dummyProps.entrySet()) {
+      assertTrue("The Configuration for key " + entry.getKey() +" and value: " + entry.getValue() +
+          " is not populated correctly", hbt.getConfiguration().get(entry.getKey()).equals(entry.getValue()));
+    }
+
+    hbt.startMiniMapReduceCluster();
+
+    // Confirm that MiniMapReduceCluster overwrites the mr properties and updates the Configuration
+    for (Map.Entry<String,String> entry : dummyProps.entrySet()) {
+      assertFalse("The MR prop: " + entry.getValue() + " is not overwritten when map reduce mini"+
+          "cluster is started", hbt.getConfiguration().get(entry.getKey()).equals(entry.getValue()));
+    }
+
+    hbt.shutdownMiniMapReduceCluster();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d994a24/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
index 4e09e1e..f49171c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
@@ -458,36 +458,4 @@ public class TestHBaseTestingUtility {
     assertEquals(nonDefaultRegionServerPort
             , htu.getConfiguration().getInt(HConstants.REGIONSERVER_PORT, 0));
   }
-
-  @Test public void testMRYarnConfigsPopulation() throws IOException {
-    Map<String, String> dummyProps = new HashMap<>();
-    dummyProps.put("mapreduce.jobtracker.address", "dummyhost:11234");
-    dummyProps.put("yarn.resourcemanager.address", "dummyhost:11235");
-    dummyProps.put("mapreduce.jobhistory.address", "dummyhost:11236");
-    dummyProps.put("yarn.resourcemanager.scheduler.address", "dummyhost:11237");
-    dummyProps.put("mapreduce.jobhistory.webapp.address", "dummyhost:11238");
-    dummyProps.put("yarn.resourcemanager.webapp.address", "dummyhost:11239");
-
-    HBaseTestingUtility hbt = new HBaseTestingUtility();
-
-    // populate the mr props to the Configuration instance
-    for (Entry<String, String> entry : dummyProps.entrySet()) {
-      hbt.getConfiguration().set(entry.getKey(), entry.getValue());
-    }
-
-    for (Entry<String,String> entry : dummyProps.entrySet()) {
-      assertTrue("The Configuration for key " + entry.getKey() +" and value: " + entry.getValue() +
-                 " is not populated correctly", hbt.getConfiguration().get(entry.getKey()).equals(entry.getValue()));
-    }
-
-    hbt.startMiniMapReduceCluster();
-
-    // Confirm that MiniMapReduceCluster overwrites the mr properties and updates the Configuration
-    for (Entry<String,String> entry : dummyProps.entrySet()) {
-      assertFalse("The MR prop: " + entry.getValue() + " is not overwritten when map reduce mini"+
-                  "cluster is started", hbt.getConfiguration().get(entry.getKey()).equals(entry.getValue()));
-    }
-
-    hbt.shutdownMiniMapReduceCluster();
-  }
 }


[06/50] [abbrv] hbase git commit: HBASE-19920 Lazy init for ProtobufUtil classloader

Posted by zh...@apache.org.
HBASE-19920 Lazy init for ProtobufUtil classloader


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

Branch: refs/heads/HBASE-19064
Commit: 8cf7a9d51ee92444e5e2c730045d2289ea97e82a
Parents: 9497eac
Author: Mike Drob <md...@apache.org>
Authored: Thu Feb 8 16:12:46 2018 -0600
Committer: Mike Drob <md...@apache.org>
Committed: Fri Feb 16 09:12:15 2018 -0600

----------------------------------------------------------------------
 .../hbase/ipc/RemoteWithExtrasException.java    | 23 ++++--
 .../hadoop/hbase/protobuf/ProtobufUtil.java     | 30 ++++----
 .../hbase/shaded/protobuf/ProtobufUtil.java     | 35 ++++++---
 .../hadoop/hbase/security/token/TokenUtil.java  | 13 +++-
 .../hbase/security/token/TestTokenUtil.java     | 80 ++++++++++++++++++++
 5 files changed, 146 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8cf7a9d5/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
index 1374ab0..3a7540e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.ipc;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -36,19 +38,24 @@ import org.apache.hadoop.ipc.RemoteException;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@edu.umd.cs.findbugs.annotations.SuppressWarnings(
-    value = "DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED", justification = "None. Address sometime.")
 public class RemoteWithExtrasException extends RemoteException {
   private final String hostname;
   private final int port;
   private final boolean doNotRetry;
 
-  private final static ClassLoader CLASS_LOADER;
+  /**
+   * Dynamic class loader to load filter/comparators
+   */
+  private final static class ClassLoaderHolder {
+    private final static ClassLoader CLASS_LOADER;
 
-  static {
-    ClassLoader parent = RemoteWithExtrasException.class.getClassLoader();
-    Configuration conf = HBaseConfiguration.create();
-    CLASS_LOADER = new DynamicClassLoader(conf, parent);
+    static {
+      ClassLoader parent = RemoteWithExtrasException.class.getClassLoader();
+      Configuration conf = HBaseConfiguration.create();
+      CLASS_LOADER = AccessController.doPrivileged((PrivilegedAction<ClassLoader>)
+        () -> new DynamicClassLoader(conf, parent)
+      );
+    }
   }
 
   public RemoteWithExtrasException(String className, String msg, final boolean doNotRetry) {
@@ -69,7 +76,7 @@ public class RemoteWithExtrasException extends RemoteException {
     try {
       // try to load a exception class from where the HBase classes are loaded or from Dynamic
       // classloader.
-      realClass = Class.forName(getClassName(), false, CLASS_LOADER);
+      realClass = Class.forName(getClassName(), false, ClassLoaderHolder.CLASS_LOADER);
     } catch (ClassNotFoundException cnfe) {
       try {
         // cause could be a hadoop exception, try to load from hadoop classpath

http://git-wip-us.apache.org/repos/asf/hbase/blob/8cf7a9d5/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 29ff2a2..3c01fd6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -33,6 +33,8 @@ import com.google.protobuf.TextFormat;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -114,8 +116,6 @@ import org.apache.yetus.audience.InterfaceAudience;
  * @see ProtobufUtil
  */
 // TODO: Generate this class from the shaded version.
-@edu.umd.cs.findbugs.annotations.SuppressWarnings(
-  value="DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED", justification="None. Address sometime.")
 @InterfaceAudience.Private // TODO: some clients (Hive, etc) use this class.
 public final class ProtobufUtil {
   private ProtobufUtil() {
@@ -169,14 +169,18 @@ public final class ProtobufUtil {
   }
 
   /**
-   * Dynamic class loader to load filter/comparators
-   */
-  private final static ClassLoader CLASS_LOADER;
+    * Dynamic class loader to load filter/comparators
+    */
+  private final static class ClassLoaderHolder {
+    private final static ClassLoader CLASS_LOADER;
 
-  static {
-    ClassLoader parent = ProtobufUtil.class.getClassLoader();
-    Configuration conf = HBaseConfiguration.create();
-    CLASS_LOADER = new DynamicClassLoader(conf, parent);
+    static {
+      ClassLoader parent = ProtobufUtil.class.getClassLoader();
+      Configuration conf = HBaseConfiguration.create();
+      CLASS_LOADER = AccessController.doPrivileged((PrivilegedAction<ClassLoader>)
+        () -> new DynamicClassLoader(conf, parent)
+      );
+    }
   }
 
   /**
@@ -1431,8 +1435,7 @@ public final class ProtobufUtil {
     String funcName = "parseFrom";
     byte [] value = proto.getSerializedComparator().toByteArray();
     try {
-      Class<? extends ByteArrayComparable> c =
-        (Class<? extends ByteArrayComparable>)Class.forName(type, true, CLASS_LOADER);
+      Class<?> c = Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER);
       Method parseFrom = c.getMethod(funcName, byte[].class);
       if (parseFrom == null) {
         throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
@@ -1455,8 +1458,7 @@ public final class ProtobufUtil {
     final byte [] value = proto.getSerializedFilter().toByteArray();
     String funcName = "parseFrom";
     try {
-      Class<? extends Filter> c =
-        (Class<? extends Filter>)Class.forName(type, true, CLASS_LOADER);
+      Class<?> c = Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER);
       Method parseFrom = c.getMethod(funcName, byte[].class);
       if (parseFrom == null) {
         throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
@@ -1542,7 +1544,7 @@ public final class ProtobufUtil {
     String type = parameter.getName();
     try {
       Class<? extends Throwable> c =
-        (Class<? extends Throwable>)Class.forName(type, true, CLASS_LOADER);
+        (Class<? extends Throwable>)Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER);
       Constructor<? extends Throwable> cn = null;
       try {
         cn = c.getDeclaredConstructor(String.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8cf7a9d5/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 5bb3b4b..2ce8d7b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -25,6 +25,8 @@ import java.io.InputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -106,6 +108,7 @@ import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams;
 import org.apache.hbase.thirdparty.com.google.gson.JsonArray;
 import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
@@ -192,8 +195,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
  * @see ProtobufUtil
  */
 // TODO: Generate the non-shaded protobufutil from this one.
-@edu.umd.cs.findbugs.annotations.SuppressWarnings(
-  value="DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED", justification="None. Address sometime.")
 @InterfaceAudience.Private // TODO: some clients (Hive, etc) use this class
 public final class ProtobufUtil {
   private ProtobufUtil() {
@@ -245,15 +246,27 @@ public final class ProtobufUtil {
     EMPTY_RESULT_PB_STALE = builder.build();
   }
 
+  private static volatile boolean classLoaderLoaded = false;
+
   /**
    * Dynamic class loader to load filter/comparators
    */
-  private final static ClassLoader CLASS_LOADER;
+  private final static class ClassLoaderHolder {
+    private final static ClassLoader CLASS_LOADER;
 
-  static {
-    ClassLoader parent = ProtobufUtil.class.getClassLoader();
-    Configuration conf = HBaseConfiguration.create();
-    CLASS_LOADER = new DynamicClassLoader(conf, parent);
+    static {
+      ClassLoader parent = ProtobufUtil.class.getClassLoader();
+      Configuration conf = HBaseConfiguration.create();
+      CLASS_LOADER = AccessController.doPrivileged((PrivilegedAction<ClassLoader>)
+        () -> new DynamicClassLoader(conf, parent)
+      );
+      classLoaderLoaded = true;
+    }
+  }
+
+  @VisibleForTesting
+  public static boolean isClassLoaderLoaded() {
+    return classLoaderLoaded;
   }
 
   /**
@@ -1588,8 +1601,7 @@ public final class ProtobufUtil {
     String funcName = "parseFrom";
     byte [] value = proto.getSerializedComparator().toByteArray();
     try {
-      Class<? extends ByteArrayComparable> c =
-        (Class<? extends ByteArrayComparable>)Class.forName(type, true, CLASS_LOADER);
+      Class<?> c = Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER);
       Method parseFrom = c.getMethod(funcName, byte[].class);
       if (parseFrom == null) {
         throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
@@ -1612,8 +1624,7 @@ public final class ProtobufUtil {
     final byte [] value = proto.getSerializedFilter().toByteArray();
     String funcName = "parseFrom";
     try {
-      Class<? extends Filter> c =
-        (Class<? extends Filter>)Class.forName(type, true, CLASS_LOADER);
+      Class<?> c = Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER);
       Method parseFrom = c.getMethod(funcName, byte[].class);
       if (parseFrom == null) {
         throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
@@ -1699,7 +1710,7 @@ public final class ProtobufUtil {
     String type = parameter.getName();
     try {
       Class<? extends Throwable> c =
-        (Class<? extends Throwable>)Class.forName(type, true, CLASS_LOADER);
+        (Class<? extends Throwable>)Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER);
       Constructor<? extends Throwable> cn = null;
       try {
         cn = c.getDeclaredConstructor(String.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8cf7a9d5/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
index 5461760..c54d905 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
@@ -26,7 +26,6 @@ import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
 
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
@@ -41,6 +40,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.security.token.Token;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,6 +53,15 @@ public class TokenUtil {
   // This class is referenced indirectly by User out in common; instances are created by reflection
   private static final Logger LOG = LoggerFactory.getLogger(TokenUtil.class);
 
+  // Set in TestTokenUtil via reflection
+  private static ServiceException injectedException;
+
+  private static void injectFault() throws ServiceException {
+    if (injectedException != null) {
+      throw injectedException;
+    }
+  }
+
   /**
    * Obtain and return an authentication token for the current user.
    * @param conn The HBase cluster connection
@@ -63,6 +72,8 @@ public class TokenUtil {
       Connection conn) throws IOException {
     Table meta = null;
     try {
+      injectFault();
+
       meta = conn.getTable(TableName.META_TABLE_NAME);
       CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
       AuthenticationProtos.AuthenticationService.BlockingInterface service =

http://git-wip-us.apache.org/repos/asf/hbase/blob/8cf7a9d5/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
new file mode 100644
index 0000000..32fcddb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.security.token;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
+@Category(SmallTests.class)
+public class TestTokenUtil {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestTokenUtil.class);
+
+  @Test
+  public void testObtainToken() throws Exception {
+    URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation();
+    URL urlTU = TokenUtil.class.getProtectionDomain().getCodeSource().getLocation();
+
+    ClassLoader cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+
+    Throwable injected = new com.google.protobuf.ServiceException("injected");
+
+    Class<?> tokenUtil = cl.loadClass(TokenUtil.class.getCanonicalName());
+    Field shouldInjectFault = tokenUtil.getDeclaredField("injectedException");
+    shouldInjectFault.setAccessible(true);
+    shouldInjectFault.set(null, injected);
+
+    try {
+      tokenUtil.getMethod("obtainToken", Connection.class)
+          .invoke(null, new Object[] { null });
+      fail("Should have injected exception.");
+    } catch (InvocationTargetException e) {
+      Throwable t = e;
+      boolean serviceExceptionFound = false;
+      while ((t = t.getCause()) != null) {
+        if (t == injected) { // reference equality
+          serviceExceptionFound = true;
+          break;
+        }
+      }
+      if (!serviceExceptionFound) {
+        throw e; // wrong exception, fail the test
+      }
+    }
+
+    Boolean loaded = (Boolean) cl.loadClass(ProtobufUtil.class.getCanonicalName())
+        .getDeclaredMethod("isClassLoaderLoaded")
+        .invoke(null);
+    assertFalse("Should not have loaded DynamicClassLoader", loaded);
+  }
+}


[43/50] [abbrv] 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/c5a0c7e8
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c5a0c7e8
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c5a0c7e8

Branch: refs/heads/HBASE-19064
Commit: c5a0c7e8d1b97f9be0e58e056de2f38ddac157d1
Parents: 8187688
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 19 18:38:39 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/c5a0c7e8/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/c5a0c7e8/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 91fb5b3..3d8c036 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -247,7 +247,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE);
     waitOnShutdownInSeconds = conf.getInt(ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS,
       DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS);
-    rollWriter();
   }
 
   private static boolean waitingRoll(int epochAndState) {

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5a0c7e8/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/c5a0c7e8/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/c5a0c7e8/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/c5a0c7e8/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 c1a77ee..4d1ed91 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/c5a0c7e8/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/c5a0c7e8/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 7cb3e63..8aa282e 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() throws IOException {
-              if (throwSyncException) {
-                throw new IOException("FAKE! Failed to replace a bad datanode...");
-              }
-              w.sync();
-            }
+    @Override
+    protected Writer createWriterInstance(Path path) throws IOException {
+      final Writer w = super.createWriterInstance(path);
+      return new Writer() {
+        @Override
+        public void close() throws IOException {
+          w.close();
+        }
 
-            @Override
-            public void append(Entry entry) throws IOException {
-              if (throwAppendException) {
-                throw new IOException("FAKE! Failed to replace a bad datanode...");
-              }
-              w.append(entry);
-            }
+        @Override
+        public void sync() throws IOException {
+          if (throwSyncException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...");
+          }
+          w.sync();
+        }
 
-            @Override
-            public long getLength() {
-              return w.getLength();
-              }
-            };
+        @Override
+        public void append(Entry entry) throws IOException {
+          if (throwAppendException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...");
           }
-      }
+          w.append(entry);
+        }
 
+        @Override
+        public long getLength() {
+          return w.getLength();
+        }
+      };
+    }
+  }
+  /**
+   * Reproduce locking up that happens when we get an exceptions appending and syncing.
+   * See HBASE-14317.
+   * First I need to set up some mocks for Server and RegionServerServices. I also need to
+   * set up a dodgy WAL that will throw an exception when we go to append to it.
+   */
+  @Test
+  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/c5a0c7e8/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 31dfa2a..4c58b47 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/c5a0c7e8/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/c5a0c7e8/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/c5a0c7e8/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 5eb8fa8..cf958f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -214,6 +214,7 @@ public class TestWALLockup {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + getName());
     DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
+    dodgyWAL.init();
     Path originalWAL = dodgyWAL.getCurrentFileName();
     // I need a log roller running.
     LogRoller logRoller = new LogRoller(server, services);

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

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

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

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

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


[08/50] [abbrv] hbase git commit: HBASE-19680 BufferedMutatorImpl#mutate should wait the result from AP in order to throw the failed mutations

Posted by zh...@apache.org.
HBASE-19680 BufferedMutatorImpl#mutate should wait the result from AP in order to throw the failed mutations


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

Branch: refs/heads/HBASE-19064
Commit: dad90f6cce5bc51e43e3778068d5e45bcb1c9de0
Parents: abf7de7
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Sat Feb 17 07:16:14 2018 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Sat Feb 17 07:33:58 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncProcess.java       |  54 +----
 .../hadoop/hbase/client/AsyncRequestFuture.java |   6 +-
 .../hbase/client/AsyncRequestFutureImpl.java    |   3 +-
 .../hbase/client/BufferedMutatorImpl.java       | 241 +++++++++----------
 .../hbase/client/ConnectionImplementation.java  |   2 +-
 .../org/apache/hadoop/hbase/client/HTable.java  |   7 +-
 .../hadoop/hbase/client/HTableMultiplexer.java  |   2 +-
 .../apache/hadoop/hbase/client/RowAccess.java   |   4 +-
 .../hadoop/hbase/client/TestAsyncProcess.java   |  90 +++----
 .../TestAsyncProcessWithRegionException.java    |   2 +-
 .../hbase/client/HConnectionTestingUtility.java |   2 +-
 11 files changed, 171 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index 6c4118c..de7449b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -65,17 +65,12 @@ import org.apache.hadoop.hbase.util.Bytes;
  * The class manages internally the retries.
  * </p>
  * <p>
- * The class can be constructed in regular mode, or "global error" mode. In global error mode,
- * AP tracks errors across all calls (each "future" also has global view of all errors). That
- * mode is necessary for backward compat with HTable behavior, where multiple submissions are
- * made and the errors can propagate using any put/flush call, from previous calls.
- * In "regular" mode, the errors are tracked inside the Future object that is returned.
+ * The errors are tracked inside the Future object that is returned.
  * The results are always tracked inside the Future object and can be retrieved when the call
  * has finished. Partial results can also be retrieved if some part of multi-request failed.
  * </p>
  * <p>
- * This class is thread safe in regular mode; in global error code, submitting operations and
- * retrieving errors from different threads may be not thread safe.
+ * This class is thread safe.
  * Internally, the class is thread safe enough to manage simultaneously new submission and results
  * arising from older operations.
  * </p>
@@ -144,7 +139,6 @@ class AsyncProcess {
   final ClusterConnection connection;
   private final RpcRetryingCallerFactory rpcCallerFactory;
   final RpcControllerFactory rpcFactory;
-  final BatchErrors globalErrors;
 
   // Start configuration settings.
   final int startLogErrorsCnt;
@@ -168,14 +162,12 @@ class AsyncProcess {
   private static final int DEFAULT_LOG_DETAILS_PERIOD = 10000;
   private final int periodToLog;
   AsyncProcess(ClusterConnection hc, Configuration conf,
-      RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors,
-      RpcControllerFactory rpcFactory) {
+      RpcRetryingCallerFactory rpcCaller, RpcControllerFactory rpcFactory) {
     if (hc == null) {
       throw new IllegalArgumentException("ClusterConnection cannot be null.");
     }
 
     this.connection = hc;
-    this.globalErrors = useGlobalErrors ? new BatchErrors() : null;
 
     this.id = COUNTER.incrementAndGet();
 
@@ -445,10 +437,10 @@ class AsyncProcess {
 
   private Consumer<Long> getLogger(TableName tableName, long max) {
     return (currentInProgress) -> {
-      LOG.info("#" + id + (max < 0 ? ", waiting for any free slot"
-      : ", waiting for some tasks to finish. Expected max="
-      + max) + ", tasksInProgress=" + currentInProgress +
-      " hasError=" + hasError() + (tableName == null ? "" : ", tableName=" + tableName));
+      LOG.info("#" + id + (max < 0 ?
+          ", waiting for any free slot" :
+          ", waiting for some tasks to finish. Expected max=" + max) + ", tasksInProgress="
+          + currentInProgress + (tableName == null ? "" : ", tableName=" + tableName));
     };
   }
 
@@ -460,38 +452,6 @@ class AsyncProcess {
   void decTaskCounters(Collection<byte[]> regions, ServerName sn) {
     requestController.decTaskCounters(regions, sn);
   }
-  /**
-   * Only used w/useGlobalErrors ctor argument, for HTable backward compat.
-   * @return Whether there were any errors in any request since the last time
-   *          {@link #waitForAllPreviousOpsAndReset(List, TableName)} was called, or AP was created.
-   */
-  public boolean hasError() {
-    return globalErrors != null && globalErrors.hasErrors();
-  }
-
-  /**
-   * Only used w/useGlobalErrors ctor argument, for HTable backward compat.
-   * Waits for all previous operations to finish, and returns errors and (optionally)
-   * failed operations themselves.
-   * @param failedRows an optional list into which the rows that failed since the last time
-   *        {@link #waitForAllPreviousOpsAndReset(List, TableName)} was called, or AP was created, are saved.
-   * @param tableName name of the table
-   * @return all the errors since the last time {@link #waitForAllPreviousOpsAndReset(List, TableName)}
-   *          was called, or AP was created.
-   */
-  public RetriesExhaustedWithDetailsException waitForAllPreviousOpsAndReset(
-      List<Row> failedRows, TableName tableName) throws InterruptedIOException {
-    waitForMaximumCurrentTasks(0, tableName);
-    if (globalErrors == null || !globalErrors.hasErrors()) {
-      return null;
-    }
-    if (failedRows != null) {
-      failedRows.addAll(globalErrors.actions);
-    }
-    RetriesExhaustedWithDetailsException result = globalErrors.makeException(logBatchErrorDetails);
-    globalErrors.clear();
-    return result;
-  }
 
   /**
    * Create a caller. Isolated to be easily overridden in the tests.

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java
index 90bd235..b91e094 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java
@@ -24,10 +24,8 @@ import java.io.InterruptedIOException;
 import java.util.List;
 
 /**
- * The context used to wait for results from one submit call.
- * 1) If AsyncProcess is set to track errors globally, and not per call (for HTable puts),
- *    then errors and failed operations in this object will reflect global errors.
- * 2) If submit call is made with needResults false, results will not be saved.
+ * The context used to wait for results from one submit call. If submit call is made with
+ * needResults false, results will not be saved.
  * @since 2.0.0
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
index ace74f9..a8b8ebf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
@@ -388,8 +388,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
             new ConcurrentHashMap<CancellableRegionServerCallable, Boolean>());
     this.asyncProcess = asyncProcess;
     this.errorsByServer = createServerErrorTracker();
-    this.errors = (asyncProcess.globalErrors != null)
-        ? asyncProcess.globalErrors : new BatchErrors();
+    this.errors = new BatchErrors();
     this.operationTimeout = task.getOperationTimeout();
     this.rpcTimeout = task.getRpcTimeout();
     this.currentCallable = task.getCallable();

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index b171fc4..9d24b4d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -16,8 +16,11 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.client.BufferedMutatorParams.UNSET;
+
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -141,7 +144,13 @@ public class BufferedMutatorImpl implements BufferedMutator {
       RpcControllerFactory rpcFactory, BufferedMutatorParams params) {
     this(conn, params,
       // puts need to track errors globally due to how the APIs currently work.
-      new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, true, rpcFactory));
+      new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, rpcFactory));
+  }
+
+  private void checkClose() {
+    if (closed) {
+      throw new IllegalStateException("Cannot put when the BufferedMutator is closed.");
+    }
   }
 
   @VisibleForTesting
@@ -173,16 +182,13 @@ public class BufferedMutatorImpl implements BufferedMutator {
   @Override
   public void mutate(List<? extends Mutation> ms) throws InterruptedIOException,
       RetriesExhaustedWithDetailsException {
-
-    if (closed) {
-      throw new IllegalStateException("Cannot put when the BufferedMutator is closed.");
-    }
+    checkClose();
 
     long toAddSize = 0;
     int toAddCount = 0;
     for (Mutation m : ms) {
       if (m instanceof Put) {
-        validatePut((Put) m);
+        HTable.validatePut((Put) m, maxKeyValueSize);
       }
       toAddSize += m.heapSize();
       ++toAddCount;
@@ -191,26 +197,10 @@ public class BufferedMutatorImpl implements BufferedMutator {
     if (currentWriteBufferSize.get() == 0) {
       firstRecordInBufferTimestamp.set(System.currentTimeMillis());
     }
-
-    // This behavior is highly non-intuitive... it does not protect us against
-    // 94-incompatible behavior, which is a timing issue because hasError, the below code
-    // and setter of hasError are not synchronized. Perhaps it should be removed.
-    if (ap.hasError()) {
-      currentWriteBufferSize.addAndGet(toAddSize);
-      writeAsyncBuffer.addAll(ms);
-      undealtMutationCount.addAndGet(toAddCount);
-      backgroundFlushCommits(true);
-    } else {
-      currentWriteBufferSize.addAndGet(toAddSize);
-      writeAsyncBuffer.addAll(ms);
-      undealtMutationCount.addAndGet(toAddCount);
-    }
-
-    // Now try and queue what needs to be queued.
-    while (undealtMutationCount.get() != 0
-        && currentWriteBufferSize.get() > writeBufferSize) {
-      backgroundFlushCommits(false);
-    }
+    currentWriteBufferSize.addAndGet(toAddSize);
+    writeAsyncBuffer.addAll(ms);
+    undealtMutationCount.addAndGet(toAddCount);
+    doFlush(false);
   }
 
   @VisibleForTesting
@@ -238,118 +228,40 @@ public class BufferedMutatorImpl implements BufferedMutator {
     }
   }
 
-  // validate for well-formedness
-  public void validatePut(final Put put) throws IllegalArgumentException {
-    HTable.validatePut(put, maxKeyValueSize);
-  }
-
   @Override
   public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    // Stop any running Periodic Flush timer.
+    disableWriteBufferPeriodicFlush();
     try {
-      if (this.closed) {
-        return;
-      }
-
-      // Stop any running Periodic Flush timer.
-      disableWriteBufferPeriodicFlush();
-
       // As we can have an operation in progress even if the buffer is empty, we call
-      // backgroundFlushCommits at least one time.
-      backgroundFlushCommits(true);
+      // doFlush at least one time.
+      doFlush(true);
+    } finally {
       if (cleanupPoolOnClose) {
         this.pool.shutdown();
-        boolean terminated;
-        int loopCnt = 0;
-        do {
-          // wait until the pool has terminated
-          terminated = this.pool.awaitTermination(60, TimeUnit.SECONDS);
-          loopCnt += 1;
-          if (loopCnt >= 10) {
+        try {
+          if (!pool.awaitTermination(600, TimeUnit.SECONDS)) {
             LOG.warn("close() failed to terminate pool after 10 minutes. Abandoning pool.");
-            break;
           }
-        } while (!terminated);
-      }
-    } catch (InterruptedException e) {
-      LOG.warn("waitForTermination interrupted");
-    } finally {
-      this.closed = true;
-    }
-  }
-
-  @Override
-  public synchronized void flush() throws InterruptedIOException,
-      RetriesExhaustedWithDetailsException {
-    // As we can have an operation in progress even if the buffer is empty, we call
-    // backgroundFlushCommits at least one time.
-    backgroundFlushCommits(true);
-  }
-
-  /**
-   * Send the operations in the buffer to the servers. Does not wait for the server's answer. If
-   * the is an error (max retried reach from a previous flush or bad operation), it tries to send
-   * all operations in the buffer and sends an exception.
-   *
-   * @param synchronous - if true, sends all the writes and wait for all of them to finish before
-   *        returning.
-   */
-  private void backgroundFlushCommits(boolean synchronous) throws
-      InterruptedIOException,
-      RetriesExhaustedWithDetailsException {
-    if (!synchronous && writeAsyncBuffer.isEmpty()) {
-      return;
-    }
-
-    if (!synchronous) {
-      QueueRowAccess taker = new QueueRowAccess();
-      AsyncProcessTask task = wrapAsyncProcessTask(taker);
-      try {
-        ap.submit(task);
-        if (ap.hasError()) {
-          LOG.debug(tableName + ": One or more of the operations have failed -"
-              + " waiting for all operation in progress to finish (successfully or not)");
-        }
-      } finally {
-        taker.restoreRemainder();
-      }
-    }
-    if (synchronous || ap.hasError()) {
-      QueueRowAccess taker = new QueueRowAccess();
-      AsyncProcessTask task = wrapAsyncProcessTask(taker);
-      try {
-        while (!taker.isEmpty()) {
-          ap.submit(task);
-          taker.reset();
-        }
-      } finally {
-        taker.restoreRemainder();
-      }
-      RetriesExhaustedWithDetailsException error =
-          ap.waitForAllPreviousOpsAndReset(null, tableName);
-      if (error != null) {
-        if (listener == null) {
-          throw error;
-        } else {
-          this.listener.onException(error, this);
+        } catch (InterruptedException e) {
+          LOG.warn("waitForTermination interrupted");
+          Thread.currentThread().interrupt();
         }
       }
+      closed = true;
     }
   }
 
-  /**
-   * Reuse the AsyncProcessTask when calling
-   * {@link BufferedMutatorImpl#backgroundFlushCommits(boolean)}.
-   * @param taker access the inner buffer.
-   * @return An AsyncProcessTask which always returns the latest rpc and operation timeout.
-   */
-  private AsyncProcessTask wrapAsyncProcessTask(QueueRowAccess taker) {
-    AsyncProcessTask task = AsyncProcessTask.newBuilder()
+  private AsyncProcessTask createTask(QueueRowAccess access) {
+    return new AsyncProcessTask(AsyncProcessTask.newBuilder()
         .setPool(pool)
         .setTableName(tableName)
-        .setRowAccess(taker)
+        .setRowAccess(access)
         .setSubmittedRows(AsyncProcessTask.SubmittedRows.AT_LEAST_ONE)
-        .build();
-    return new AsyncProcessTask(task) {
+        .build()) {
       @Override
       public int getRpcTimeout() {
         return rpcTimeout.get();
@@ -362,6 +274,72 @@ public class BufferedMutatorImpl implements BufferedMutator {
     };
   }
 
+  @Override
+  public void flush() throws InterruptedIOException, RetriesExhaustedWithDetailsException {
+    checkClose();
+    doFlush(true);
+  }
+
+  /**
+   * Send the operations in the buffer to the servers.
+   *
+   * @param flushAll - if true, sends all the writes and wait for all of them to finish before
+   *                 returning. Otherwise, flush until buffer size is smaller than threshold
+   */
+  private void doFlush(boolean flushAll) throws InterruptedIOException,
+      RetriesExhaustedWithDetailsException {
+    List<RetriesExhaustedWithDetailsException> errors = new ArrayList<>();
+    while (true) {
+      if (!flushAll && currentWriteBufferSize.get() <= writeBufferSize) {
+        // There is the room to accept more mutations.
+        break;
+      }
+      AsyncRequestFuture asf;
+      try (QueueRowAccess access = new QueueRowAccess()) {
+        if (access.isEmpty()) {
+          // It means someone has gotten the ticker to run the flush.
+          break;
+        }
+        asf = ap.submit(createTask(access));
+      }
+      // DON'T do the wait in the try-with-resources. Otherwise, the undealt mutations won't
+      // be released.
+      asf.waitUntilDone();
+      if (asf.hasError()) {
+        errors.add(asf.getErrors());
+      }
+    }
+
+    RetriesExhaustedWithDetailsException exception = makeException(errors);
+    if (exception == null) {
+      return;
+    } else if(listener == null) {
+      throw exception;
+    } else {
+      listener.onException(exception, this);
+    }
+  }
+
+  private static RetriesExhaustedWithDetailsException makeException(
+    List<RetriesExhaustedWithDetailsException> errors) {
+    switch (errors.size()) {
+      case 0:
+        return null;
+      case 1:
+        return errors.get(0);
+      default:
+        List<Throwable> exceptions = new ArrayList<>();
+        List<Row> actions = new ArrayList<>();
+        List<String> hostnameAndPort = new ArrayList<>();
+        errors.forEach(e -> {
+          exceptions.addAll(e.exceptions);
+          actions.addAll(e.actions);
+          hostnameAndPort.addAll(e.hostnameAndPort);
+        });
+        return new RetriesExhaustedWithDetailsException(exceptions, actions, hostnameAndPort);
+    }
+  }
+
   /**
    * {@inheritDoc}
    */
@@ -433,12 +411,15 @@ public class BufferedMutatorImpl implements BufferedMutator {
     return undealtMutationCount.get();
   }
 
-  private class QueueRowAccess implements RowAccess<Row> {
+  private class QueueRowAccess implements RowAccess<Row>, Closeable {
     private int remainder = undealtMutationCount.getAndSet(0);
 
-    void reset() {
-      restoreRemainder();
-      remainder = undealtMutationCount.getAndSet(0);
+    @Override
+    public void close() {
+      if (remainder > 0) {
+        undealtMutationCount.addAndGet(remainder);
+        remainder = 0;
+      }
     }
 
     @Override
@@ -474,6 +455,7 @@ public class BufferedMutatorImpl implements BufferedMutator {
           iter.remove();
           currentWriteBufferSize.addAndGet(-last.heapSize());
           --remainder;
+          last = null;
         }
       };
     }
@@ -483,13 +465,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
       return remainder;
     }
 
-    void restoreRemainder() {
-      if (remainder > 0) {
-        undealtMutationCount.addAndGet(remainder);
-        remainder = 0;
-      }
-    }
-
     @Override
     public boolean isEmpty() {
       return remainder <= 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 8807884..a5a0188 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
@@ -261,7 +261,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
     this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats);
     this.backoffPolicy = ClientBackoffPolicyFactory.create(conf);
-    this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, false, rpcControllerFactory);
+    this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory);
     if (conf.getBoolean(CLIENT_SIDE_METRICS_ENABLED_KEY, false)) {
       this.metrics = new MetricsConnection(this);
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 855005e..a4289e9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -1182,10 +1182,9 @@ public class HTable implements Table {
     final List<String> callbackErrorServers = new ArrayList<>();
     Object[] results = new Object[execs.size()];
 
-    AsyncProcess asyncProcess =
-        new AsyncProcess(connection, configuration,
-            RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()),
-            true, RpcControllerFactory.instantiate(configuration));
+    AsyncProcess asyncProcess = new AsyncProcess(connection, configuration,
+        RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()),
+        RpcControllerFactory.instantiate(configuration));
 
     Callback<ClientProtos.CoprocessorServiceResult> resultsCallback
     = (byte[] region, byte[] row, ClientProtos.CoprocessorServiceResult serviceResult) -> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index 4be39a9..e6b061e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -452,7 +452,7 @@ public class HTableMultiplexer {
               HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
       this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
           HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
-      this.ap = new AsyncProcess(conn, conf, rpcCallerFactory, false, rpcControllerFactory);
+      this.ap = new AsyncProcess(conn, conf, rpcCallerFactory, rpcControllerFactory);
       this.executor = executor;
       this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000);
       this.pool = pool;

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
index 16921fe..9f92c66 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -27,7 +25,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * of elements between collections.
  * @param <T>
  */
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 public interface RowAccess<T> extends Iterable<T> {
   /**
    * @return true if there are no elements.

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index fc405df..2979dcd 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -70,7 +70,6 @@ import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
@@ -171,22 +170,17 @@ public class TestAsyncProcess {
     }
 
     public MyAsyncProcess(ClusterConnection hc, Configuration conf) {
-      this(hc, conf, new AtomicInteger());
+      super(hc, conf,
+          new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
+      service = Executors.newFixedThreadPool(5);
     }
 
     public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) {
-      super(hc, conf, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf));
+      super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
       service = new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
           new SynchronousQueue<>(), new CountingThreadFactory(nbThreads));
     }
 
-    public MyAsyncProcess(
-        ClusterConnection hc, Configuration conf, boolean useGlobalErrors) {
-      super(hc, conf,
-          new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf));
-      service = Executors.newFixedThreadPool(5);
-    }
-
     public <CResult> AsyncRequestFuture submit(ExecutorService pool, TableName tableName,
         List<? extends Row> rows, boolean atLeastOne, Batch.Callback<CResult> callback,
         boolean needResults) throws InterruptedIOException {
@@ -324,7 +318,7 @@ public class TestAsyncProcess {
     private final IOException ioe;
 
     public AsyncProcessWithFailure(ClusterConnection hc, Configuration conf, IOException ioe) {
-      super(hc, conf, true);
+      super(hc, conf);
       this.ioe = ioe;
       serverTrackerTimeout = 1L;
     }
@@ -656,7 +650,7 @@ public class TestAsyncProcess {
         + ", minCountSnRequest:" + minCountSnRequest
         + ", minCountSn2Request:" + minCountSn2Request);
 
-    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
     try (BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap)) {
       mutator.mutate(puts);
@@ -807,7 +801,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testFail() throws Exception {
-    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false);
+    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
 
     List<Put> puts = new ArrayList<>(1);
     Put p = createPut(1, false);
@@ -834,7 +828,7 @@ public class TestAsyncProcess {
   @Test
   public void testSubmitTrue() throws IOException {
     ClusterConnection conn = createHConnection();
-    final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, false);
+    final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     final String defaultClazz =
         conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
     conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
@@ -882,7 +876,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testFailAndSuccess() throws Exception {
-    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false);
+    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
 
     List<Put> puts = new ArrayList<>(3);
     puts.add(createPut(1, false));
@@ -909,7 +903,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testFlush() throws Exception {
-    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false);
+    MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF);
 
     List<Put> puts = new ArrayList<>(3);
     puts.add(createPut(1, false));
@@ -927,7 +921,7 @@ public class TestAsyncProcess {
   @Test
   public void testTaskCountWithoutClientBackoffPolicy() throws IOException, InterruptedException {
     ClusterConnection hc = createHConnection();
-    MyAsyncProcess ap = new MyAsyncProcess(hc, CONF, false);
+    MyAsyncProcess ap = new MyAsyncProcess(hc, CONF);
     testTaskCount(ap);
   }
 
@@ -944,7 +938,7 @@ public class TestAsyncProcess {
         conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
     conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
       SimpleRequestController.class.getName());
-    MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf, false);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf);
     testTaskCount(ap);
     if (defaultClazz != null) {
       conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
@@ -981,7 +975,7 @@ public class TestAsyncProcess {
         conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
     conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
       SimpleRequestController.class.getName());
-    final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, false);
+    final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     SimpleRequestController controller = (SimpleRequestController) ap.requestController;
 
 
@@ -1087,7 +1081,7 @@ public class TestAsyncProcess {
   @Test
   public void testHTablePutSuccess() throws Exception {
     ClusterConnection conn = createHConnection();
-    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
     BufferedMutatorImpl ht = new BufferedMutatorImpl(conn, bufferParam, ap);
 
@@ -1104,7 +1098,7 @@ public class TestAsyncProcess {
   @Test
   public void testSettingWriteBufferPeriodicFlushParameters() throws Exception {
     ClusterConnection conn = createHConnection();
-    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
 
     checkPeriodicFlushParameters(conn, ap,
             1234, 1234,
@@ -1150,7 +1144,7 @@ public class TestAsyncProcess {
   @Test
   public void testWriteBufferPeriodicFlushTimeoutMs() throws Exception {
     ClusterConnection conn = createHConnection();
-    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
 
     bufferParam.setWriteBufferPeriodicFlushTimeoutMs(1);     // Flush ASAP
@@ -1217,7 +1211,7 @@ public class TestAsyncProcess {
   @Test
   public void testBufferedMutatorImplWithSharedPool() throws Exception {
     ClusterConnection conn = createHConnection();
-    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
     BufferedMutator ht = new BufferedMutatorImpl(conn, bufferParam, ap);
 
@@ -1226,30 +1220,27 @@ public class TestAsyncProcess {
   }
 
   @Test
-  public void testHTableFailedPutAndNewPut() throws Exception {
+  public void testFailedPutAndNewPut() throws Exception {
     ClusterConnection conn = createHConnection();
-    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE)
             .writeBufferSize(0);
     BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
 
     Put p = createPut(1, false);
-    mutator.mutate(p);
-
-    ap.waitForMaximumCurrentTasks(0, null); // Let's do all the retries.
-
-    // We're testing that we're behaving as we were behaving in 0.94: sending exceptions in the
-    //  doPut if it fails.
-    // This said, it's not a very easy going behavior. For example, when we insert a list of
-    //  puts, we may raise an exception in the middle of the list. It's then up to the caller to
-    //  manage what was inserted, what was tried but failed, and what was not even tried.
-    p = createPut(1, true);
-    Assert.assertEquals(0, mutator.size());
     try {
       mutator.mutate(p);
       Assert.fail();
-    } catch (RetriesExhaustedException expected) {
+    } catch (RetriesExhaustedWithDetailsException expected) {
+      assertEquals(1, expected.getNumExceptions());
+      assertTrue(expected.getRow(0) == p);
     }
+    // Let's do all the retries.
+    ap.waitForMaximumCurrentTasks(0, null);
+    Assert.assertEquals(0, mutator.size());
+
+    // There is no global error so the new put should not fail
+    mutator.mutate(createPut(1, true));
     Assert.assertEquals("the put should not been inserted.", 0, mutator.size());
   }
 
@@ -1277,7 +1268,7 @@ public class TestAsyncProcess {
   public void testBatch() throws IOException, InterruptedException {
     ClusterConnection conn = new MyConnectionImpl(CONF);
     HTable ht = (HTable) conn.getTable(DUMMY_TABLE);
-    ht.multiAp = new MyAsyncProcess(conn, CONF, false);
+    ht.multiAp = new MyAsyncProcess(conn, CONF);
 
     List<Put> puts = new ArrayList<>(7);
     puts.add(createPut(1, true));
@@ -1307,7 +1298,7 @@ public class TestAsyncProcess {
   public void testErrorsServers() throws IOException {
     Configuration configuration = new Configuration(CONF);
     ClusterConnection conn = new MyConnectionImpl(configuration);
-    MyAsyncProcess ap = new MyAsyncProcess(conn, configuration, true);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, configuration);
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
     BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
     configuration.setBoolean(ConnectionImplementation.RETRIES_BY_SERVER_KEY, true);
@@ -1323,21 +1314,22 @@ public class TestAsyncProcess {
       mutator.flush();
       Assert.fail();
     } catch (RetriesExhaustedWithDetailsException expected) {
+      assertEquals(1, expected.getNumExceptions());
+      assertTrue(expected.getRow(0) == p);
     }
     // Checking that the ErrorsServers came into play and didn't make us stop immediately
     Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
   }
 
-  @Ignore @Test // Test is failing with wrong count. FIX!!
+  @Test
   public void testReadAndWriteTimeout() throws IOException {
     final long readTimeout = 10 * 1000;
     final long writeTimeout = 20 * 1000;
     Configuration copyConf = new Configuration(CONF);
     copyConf.setLong(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, readTimeout);
     copyConf.setLong(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, writeTimeout);
-    ClusterConnection conn = createHConnection();
-    Mockito.when(conn.getConfiguration()).thenReturn(copyConf);
-    MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf, true);
+    ClusterConnection conn = new MyConnectionImpl(copyConf);
+    MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf);
     try (HTable ht = (HTable) conn.getTable(DUMMY_TABLE)) {
       ht.multiAp = ap;
       List<Get> gets = new LinkedList<>();
@@ -1368,7 +1360,7 @@ public class TestAsyncProcess {
   }
 
   @Test
-  public void testGlobalErrors() throws IOException {
+  public void testErrors() throws IOException {
     ClusterConnection conn = new MyConnectionImpl(CONF);
     AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, CONF, new IOException("test"));
     BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
@@ -1383,6 +1375,8 @@ public class TestAsyncProcess {
       mutator.flush();
       Assert.fail();
     } catch (RetriesExhaustedWithDetailsException expected) {
+      assertEquals(1, expected.getNumExceptions());
+      assertTrue(expected.getRow(0) == p);
     }
     // Checking that the ErrorsServers came into play and didn't make us stop immediately
     Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
@@ -1404,6 +1398,8 @@ public class TestAsyncProcess {
       mutator.flush();
       Assert.fail();
     } catch (RetriesExhaustedWithDetailsException expected) {
+      assertEquals(1, expected.getNumExceptions());
+      assertTrue(expected.getRow(0) == p);
     }
     // Checking that the ErrorsServers came into play and didn't make us stop immediately
     Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get());
@@ -1703,7 +1699,7 @@ public class TestAsyncProcess {
 
   static class AsyncProcessForThrowableCheck extends AsyncProcess {
     public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf) {
-      super(hc, conf, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(
+      super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(
           conf));
     }
   }
@@ -1759,6 +1755,8 @@ public class TestAsyncProcess {
       mutator.flush();
       Assert.fail();
     } catch (RetriesExhaustedWithDetailsException expected) {
+      assertEquals(1, expected.getNumExceptions());
+      assertTrue(expected.getRow(0) == p);
     }
     long actualSleep = System.currentTimeMillis() - startTime;
     long expectedSleep = 0L;
@@ -1784,6 +1782,8 @@ public class TestAsyncProcess {
       mutator.flush();
       Assert.fail();
     } catch (RetriesExhaustedWithDetailsException expected) {
+      assertEquals(1, expected.getNumExceptions());
+      assertTrue(expected.getRow(0) == p);
     }
     actualSleep = System.currentTimeMillis() - startTime;
     expectedSleep = 0L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
index c46385e..ffc4e51 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java
@@ -202,7 +202,7 @@ public class TestAsyncProcessWithRegionException {
     private final ExecutorService service = Executors.newFixedThreadPool(5);
 
     MyAsyncProcess(ClusterConnection hc, Configuration conf) {
-      super(hc, conf, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf));
+      super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf));
     }
 
     public AsyncRequestFuture submit(TableName tableName, List<? extends Row> rows)

http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index 8ef784c..0f896b3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -127,7 +127,7 @@ public class HConnectionTestingUtility {
     NonceGenerator ng = Mockito.mock(NonceGenerator.class);
     Mockito.when(c.getNonceGenerator()).thenReturn(ng);
     Mockito.when(c.getAsyncProcess()).thenReturn(
-      new AsyncProcess(c, conf, RpcRetryingCallerFactory.instantiate(conf), false,
+      new AsyncProcess(c, conf, RpcRetryingCallerFactory.instantiate(conf),
           RpcControllerFactory.instantiate(conf)));
     Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn(
         RpcRetryingCallerFactory.instantiate(conf,


[13/50] [abbrv] hbase git commit: HBASE-18294 Reduce global heap pressure: flush based on heap occupancy

Posted by zh...@apache.org.
HBASE-18294 Reduce global heap pressure: flush based on heap occupancy


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

Branch: refs/heads/HBASE-19064
Commit: f3bb9b9613beaefbee0a53be1fb66b94fde1ce19
Parents: f3ff55a
Author: eshcar <es...@yahoo-inc.com>
Authored: Sun Feb 18 09:55:44 2018 +0200
Committer: eshcar <es...@yahoo-inc.com>
Committed: Sun Feb 18 09:55:44 2018 +0200

----------------------------------------------------------------------
 .../hadoop/hbase/client/ConnectionUtils.java    |   2 +-
 .../apache/hadoop/hbase/client/Mutation.java    |   2 +-
 .../org/apache/hadoop/hbase/client/Result.java  |   2 +-
 .../apache/hadoop/hbase/ByteBufferKeyValue.java |   2 +-
 .../java/org/apache/hadoop/hbase/CellUtil.java  |   2 +-
 .../apache/hadoop/hbase/PrivateCellUtil.java    |   7 +-
 .../hbase/util/MapReduceExtendedCell.java       |   2 +-
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java  |   2 +-
 .../hbase/regionserver/AbstractMemStore.java    |   4 +-
 .../regionserver/CSLMImmutableSegment.java      |   3 +-
 .../regionserver/CellArrayImmutableSegment.java |   9 +-
 .../regionserver/CellChunkImmutableSegment.java |  52 ++++++-
 .../hbase/regionserver/CompactingMemStore.java  |   8 +-
 .../hbase/regionserver/CompactionPipeline.java  |  47 +++---
 .../regionserver/CompositeImmutableSegment.java |  12 +-
 .../hbase/regionserver/DefaultMemStore.java     |   4 +-
 .../regionserver/FlushAllLargeStoresPolicy.java |   2 +-
 .../regionserver/FlushLargeStoresPolicy.java    |  52 ++++---
 .../FlushNonSloppyStoresFirstPolicy.java        |   2 +-
 .../hadoop/hbase/regionserver/HRegion.java      | 144 ++++++++++---------
 .../hbase/regionserver/HRegionServer.java       |  33 ++++-
 .../hadoop/hbase/regionserver/HStore.java       |   2 +-
 .../regionserver/ImmutableMemStoreLAB.java      |  12 ++
 .../hbase/regionserver/ImmutableSegment.java    |   4 +
 .../hbase/regionserver/MemStoreFlusher.java     |  92 +++++++++---
 .../hadoop/hbase/regionserver/MemStoreLAB.java  |   8 +-
 .../hbase/regionserver/MemStoreLABImpl.java     |  10 ++
 .../hadoop/hbase/regionserver/MemStoreSize.java |  52 ++++++-
 .../hbase/regionserver/MemStoreSizing.java      |  58 +++-----
 .../hbase/regionserver/MemStoreSnapshot.java    |  16 +--
 .../MetricsTableWrapperAggregateImpl.java       |   2 +-
 .../hbase/regionserver/MutableSegment.java      |   7 +-
 .../hadoop/hbase/regionserver/Region.java       |  16 ++-
 .../regionserver/RegionServerAccounting.java    |  45 +++---
 .../regionserver/RegionServicesForStores.java   |   4 +-
 .../hadoop/hbase/regionserver/Segment.java      | 115 ++++++++++-----
 .../hadoop/hbase/regionserver/StoreScanner.java |   2 +-
 .../org/apache/hadoop/hbase/wal/WALEdit.java    |   2 +-
 .../hadoop/hbase/TestGlobalMemStoreSize.java    |   4 +-
 .../hbase/TestPartialResultsFromClientSide.java |   2 +-
 ...TestServerSideScanMetricsFromClientSide.java |   3 +-
 .../hbase/client/TestAsyncRegionAdminApi.java   |  12 +-
 .../hadoop/hbase/client/TestClientPushback.java |   6 +-
 .../hbase/client/TestFlushFromClient.java       |  14 +-
 .../hadoop/hbase/client/TestSizeFailures.java   |   6 +-
 ...NegativeMemStoreSizeWithSlowCoprocessor.java |   3 +-
 .../regionserver/TestCompactingMemStore.java    |   8 +-
 .../TestCompactingToCellFlatMapMemStore.java    |   2 +-
 .../TestEndToEndSplitTransaction.java           |   2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  20 +--
 .../regionserver/TestHRegionReplayEvents.java   |  36 ++---
 .../hadoop/hbase/regionserver/TestHStore.java   |   6 +-
 .../regionserver/TestPerColumnFamilyFlush.java  |  20 +--
 .../TestRegionServerAccounting.java             |  20 +--
 .../hbase/regionserver/TestWALLockup.java       |   4 +-
 .../TestWalAndCompactingMemStoreFlush.java      |  18 +--
 56 files changed, 649 insertions(+), 377 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index 1a093f8..c9e994f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -320,7 +320,7 @@ public final class ConnectionUtils {
     long estimatedHeapSizeOfResult = 0;
     // We don't make Iterator here
     for (Cell cell : rs.rawCells()) {
-      estimatedHeapSizeOfResult += PrivateCellUtil.estimatedHeapSizeOf(cell);
+      estimatedHeapSizeOfResult += PrivateCellUtil.estimatedSizeOfCell(cell);
     }
     return estimatedHeapSizeOfResult;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index 4398fd6..09000ac 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -488,7 +488,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
           size * ClassSize.REFERENCE);
 
       for(Cell cell : entry.getValue()) {
-        heapsize += PrivateCellUtil.estimatedHeapSizeOf(cell);
+        heapsize += PrivateCellUtil.estimatedSizeOfCell(cell);
       }
     }
     heapsize += getAttributeSize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index d30c25f..832689e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -859,7 +859,7 @@ public class Result implements CellScannable, CellScanner {
       return size;
     }
     for (Cell c : result.rawCells()) {
-      size += PrivateCellUtil.estimatedHeapSizeOf(c);
+      size += PrivateCellUtil.estimatedSizeOfCell(c);
     }
     return size;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
index c82ed8d..760d02c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
@@ -255,7 +255,7 @@ public class ByteBufferKeyValue extends ByteBufferExtendedCell {
     if (this.buf.hasArray()) {
       return ClassSize.align(FIXED_OVERHEAD + length);
     }
-    return ClassSize.align(FIXED_OVERHEAD);
+    return ClassSize.align(FIXED_OVERHEAD) + KeyValueUtil.length(this);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 7e13885..de39fcc 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -1016,7 +1016,7 @@ public final class CellUtil {
    */
   @Deprecated
   public static long estimatedHeapSizeOf(final Cell cell) {
-    return PrivateCellUtil.estimatedHeapSizeOf(cell);
+    return PrivateCellUtil.estimatedSizeOfCell(cell);
   }
 
   /********************* tags *************************************/

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
index d48ab60..4ebe26c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
@@ -250,7 +250,7 @@ public final class PrivateCellUtil {
 
     @Override
     public long heapSize() {
-      long sum = HEAP_SIZE_OVERHEAD + estimatedHeapSizeOf(cell);
+      long sum = HEAP_SIZE_OVERHEAD + estimatedSizeOfCell(cell);
       if (this.tags != null) {
         sum += ClassSize.sizeOf(this.tags);
       }
@@ -446,7 +446,7 @@ public final class PrivateCellUtil {
 
     @Override
     public long heapSize() {
-      long sum = HEAP_SIZE_OVERHEAD + estimatedHeapSizeOf(cell);
+      long sum = HEAP_SIZE_OVERHEAD + estimatedSizeOfCell(cell);
       // this.tags is on heap byte[]
       if (this.tags != null) {
         sum += ClassSize.sizeOf(this.tags);
@@ -2783,10 +2783,11 @@ public final class PrivateCellUtil {
    * {@link HeapSize} we call {@link HeapSize#heapSize()} so cell can give a correct value. In other
    * cases we just consider the bytes occupied by the cell components ie. row, CF, qualifier,
    * timestamp, type, value and tags.
+   * Note that this can be the JVM heap space (on-heap) or the OS heap (off-heap)
    * @param cell
    * @return estimate of the heap space
    */
-  public static long estimatedHeapSizeOf(final Cell cell) {
+  public static long estimatedSizeOfCell(final Cell cell) {
     if (cell instanceof HeapSize) {
       return ((HeapSize) cell).heapSize();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
index 73eb7d8..75b57f4 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
@@ -241,7 +241,7 @@ public class MapReduceExtendedCell extends ByteBufferExtendedCell {
 
   @Override
   public long heapSize() {
-    return PrivateCellUtil.estimatedHeapSizeOf(cell);
+    return PrivateCellUtil.estimatedSizeOfCell(cell);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index 7b8815f..e8818be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -266,7 +266,7 @@ public class HFileBlockIndex {
 
         // Adding blockKeys
         for (Cell key : blockKeys) {
-          heapSize += ClassSize.align(PrivateCellUtil.estimatedHeapSizeOf(key));
+          heapSize += ClassSize.align(PrivateCellUtil.estimatedSizeOfCell(key));
         }
       }
       // Add comparator and the midkey atomicreference

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
index 6dbe0a8..e6fd04d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
@@ -171,7 +171,9 @@ public abstract class AbstractMemStore implements MemStore {
   }
 
   MemStoreSizing getSnapshotSizing() {
-    return new MemStoreSizing(this.snapshot.keySize(), this.snapshot.heapSize());
+    return new MemStoreSizing(this.snapshot.keySize(),
+        this.snapshot.heapSize(),
+        this.snapshot.offHeapSize());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CSLMImmutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CSLMImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CSLMImmutableSegment.java
index b5fe033..6af84cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CSLMImmutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CSLMImmutableSegment.java
@@ -39,7 +39,8 @@ public class CSLMImmutableSegment extends ImmutableSegment {
   protected CSLMImmutableSegment(Segment segment) {
     super(segment);
     // update the segment metadata heap size
-    incSize(0, -MutableSegment.DEEP_OVERHEAD + DEEP_OVERHEAD_CSLM);
+    long indexOverhead = -MutableSegment.DEEP_OVERHEAD + DEEP_OVERHEAD_CSLM;
+    incSize(0, indexOverhead, 0); // CSLM is always on-heap
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java
index 7e00899..4631200 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java
@@ -45,7 +45,7 @@ public class CellArrayImmutableSegment extends ImmutableSegment {
   protected CellArrayImmutableSegment(CellComparator comparator, MemStoreSegmentsIterator iterator,
       MemStoreLAB memStoreLAB, int numOfCells, MemStoreCompactionStrategy.Action action) {
     super(null, comparator, memStoreLAB); // initiailize the CellSet with NULL
-    incSize(0, DEEP_OVERHEAD_CAM);
+    incSize(0, DEEP_OVERHEAD_CAM, 0); // CAM is always on-heap
     // build the new CellSet based on CellArrayMap and update the CellSet of the new Segment
     initializeCellSet(numOfCells, iterator, action);
   }
@@ -58,7 +58,8 @@ public class CellArrayImmutableSegment extends ImmutableSegment {
   protected CellArrayImmutableSegment(CSLMImmutableSegment segment, MemStoreSizing memstoreSizing,
       MemStoreCompactionStrategy.Action action) {
     super(segment); // initiailize the upper class
-    incSize(0, DEEP_OVERHEAD_CAM - CSLMImmutableSegment.DEEP_OVERHEAD_CSLM);
+    long indexOverhead = DEEP_OVERHEAD_CAM - CSLMImmutableSegment.DEEP_OVERHEAD_CSLM;
+    incSize(0, indexOverhead, 0); // CAM is always on-heap
     int numOfCells = segment.getCellsCount();
     // build the new CellSet based on CellChunkMap and update the CellSet of this Segment
     reinitializeCellSet(numOfCells, segment.getScanner(Long.MAX_VALUE), segment.getCellSet(),
@@ -66,8 +67,8 @@ public class CellArrayImmutableSegment extends ImmutableSegment {
     // arrange the meta-data size, decrease all meta-data sizes related to SkipList;
     // add sizes of CellArrayMap entry (reinitializeCellSet doesn't take the care for the sizes)
     long newSegmentSizeDelta = numOfCells*(indexEntrySize()-ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
-    incSize(0, newSegmentSizeDelta);
-    memstoreSizing.incMemStoreSize(0, newSegmentSizeDelta);
+    incSize(0, newSegmentSizeDelta, 0);
+    memstoreSizing.incMemStoreSize(0, newSegmentSizeDelta, 0);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
index bf9b191..53458f1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java
@@ -53,7 +53,15 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
   protected CellChunkImmutableSegment(CellComparator comparator, MemStoreSegmentsIterator iterator,
       MemStoreLAB memStoreLAB, int numOfCells, MemStoreCompactionStrategy.Action action) {
     super(null, comparator, memStoreLAB); // initialize the CellSet with NULL
-    incSize(0, DEEP_OVERHEAD_CCM); // initiate the heapSize with the size of the segment metadata
+    long indexOverhead = DEEP_OVERHEAD_CCM;
+    // memStoreLAB cannot be null in this class
+    boolean onHeap = getMemStoreLAB().isOnHeap();
+    // initiate the heapSize with the size of the segment metadata
+    if(onHeap) {
+      incSize(0, indexOverhead, 0);
+    } else {
+      incSize(0, 0, indexOverhead);
+    }
     // build the new CellSet based on CellArrayMap and update the CellSet of the new Segment
     initializeCellSet(numOfCells, iterator, action);
   }
@@ -66,7 +74,15 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
   protected CellChunkImmutableSegment(CSLMImmutableSegment segment,
       MemStoreSizing memstoreSizing, MemStoreCompactionStrategy.Action action) {
     super(segment); // initiailize the upper class
-    incSize(0,-CSLMImmutableSegment.DEEP_OVERHEAD_CSLM + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM);
+    long indexOverhead = -CSLMImmutableSegment.DEEP_OVERHEAD_CSLM + DEEP_OVERHEAD_CCM;
+    // memStoreLAB cannot be null in this class
+    boolean onHeap = getMemStoreLAB().isOnHeap();
+    // initiate the heapSize with the size of the segment metadata
+    if(onHeap) {
+      incSize(0, indexOverhead, 0);
+    } else {
+      incSize(0, -CSLMImmutableSegment.DEEP_OVERHEAD_CSLM, DEEP_OVERHEAD_CCM);
+    }
     int numOfCells = segment.getCellsCount();
     // build the new CellSet based on CellChunkMap
     reinitializeCellSet(numOfCells, segment.getScanner(Long.MAX_VALUE), segment.getCellSet(),
@@ -75,9 +91,32 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
     // add sizes of CellChunkMap entry, decrease also Cell object sizes
     // (reinitializeCellSet doesn't take the care for the sizes)
     long newSegmentSizeDelta = numOfCells*(indexEntrySize()-ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
+    if(onHeap) {
+      incSize(0, newSegmentSizeDelta, 0);
+      memstoreSizing.incMemStoreSize(0, newSegmentSizeDelta, 0);
+    } else {
+      incSize(0, 0, newSegmentSizeDelta);
+      memstoreSizing.incMemStoreSize(0, 0, newSegmentSizeDelta);
+
+    }
+  }
 
-    incSize(0, newSegmentSizeDelta);
-    memstoreSizing.incMemStoreSize(0, newSegmentSizeDelta);
+  @Override
+  protected long indexEntryOnHeapSize(boolean onHeap) {
+    if(onHeap) {
+      return indexEntrySize();
+    }
+    // else the index is allocated off-heap
+    return 0;
+  }
+
+  @Override
+  protected long indexEntryOffHeapSize(boolean offHeap) {
+    if(offHeap) {
+      return indexEntrySize();
+    }
+    // else the index is allocated on-heap
+    return 0;
   }
 
   @Override
@@ -257,13 +296,16 @@ public class CellChunkImmutableSegment extends ImmutableSegment {
     // The actual size of the cell is not added yet, and will be added (only in compaction)
     // in initializeCellSet#updateMetaInfo().
     long oldHeapSize = heapSizeChange(cell, true);
+    long oldOffHeapSize = offHeapSizeChange(cell, true);
     long oldCellSize = getCellLength(cell);
     cell = maybeCloneWithAllocator(cell, true);
     long newHeapSize = heapSizeChange(cell, true);
+    long newOffHeapSize = offHeapSizeChange(cell, true);
     long newCellSize = getCellLength(cell);
     long heapOverhead = newHeapSize - oldHeapSize;
+    long offHeapOverhead = newOffHeapSize - oldOffHeapSize;
     //TODO: maybe need to update the dataSize of the region
-    incSize(newCellSize - oldCellSize, heapOverhead);
+    incSize(newCellSize - oldCellSize, heapOverhead, offHeapOverhead);
     return cell;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
index 3cb4103..bcecdc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
@@ -157,9 +157,9 @@ public class CompactingMemStore extends AbstractMemStore {
   @Override
   public MemStoreSize size() {
     MemStoreSizing memstoreSizing = new MemStoreSizing();
-    memstoreSizing.incMemStoreSize(this.active.keySize(), this.active.heapSize());
+    memstoreSizing.incMemStoreSize(active.getMemStoreSize());
     for (Segment item : pipeline.getSegments()) {
-      memstoreSizing.incMemStoreSize(item.keySize(), item.heapSize());
+      memstoreSizing.incMemStoreSize(item.getMemStoreSize());
     }
     return memstoreSizing;
   }
@@ -231,13 +231,13 @@ public class CompactingMemStore extends AbstractMemStore {
       // if snapshot is empty the tail of the pipeline (or everything in the memstore) is flushed
       if (compositeSnapshot) {
         snapshotSizing = pipeline.getPipelineSizing();
-        snapshotSizing.incMemStoreSize(this.active.keySize(), this.active.heapSize());
+        snapshotSizing.incMemStoreSize(active.getMemStoreSize());
       } else {
         snapshotSizing = pipeline.getTailSizing();
       }
     }
     return snapshotSizing.getDataSize() > 0 ? snapshotSizing
-        : new MemStoreSize(this.active.keySize(), this.active.heapSize());
+        : new MemStoreSize(active.getMemStoreSize());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
index 461f900..adf57d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
@@ -138,16 +138,25 @@ public class CompactionPipeline {
       if(segment != null) newDataSize = segment.keySize();
       long dataSizeDelta = suffixDataSize - newDataSize;
       long suffixHeapSize = getSegmentsHeapSize(suffix);
+      long suffixOffHeapSize = getSegmentsOffHeapSize(suffix);
       long newHeapSize = 0;
-      if(segment != null) newHeapSize = segment.heapSize();
+      long newOffHeapSize = 0;
+      if(segment != null) {
+        newHeapSize = segment.heapSize();
+        newOffHeapSize = segment.offHeapSize();
+      }
+      long offHeapSizeDelta = suffixOffHeapSize - newOffHeapSize;
       long heapSizeDelta = suffixHeapSize - newHeapSize;
-      region.addMemStoreSize(new MemStoreSizing(-dataSizeDelta, -heapSizeDelta));
-      LOG.debug("Suffix data size={}, new segment data size={}, suffix heap size={}," +
-              "new segment heap size={}",
-          suffixDataSize,
-          newDataSize,
-          suffixHeapSize,
-          newHeapSize);
+      region.addMemStoreSize(new MemStoreSize(-dataSizeDelta, -heapSizeDelta, -offHeapSizeDelta));
+      LOG.debug("Suffix data size={}, new segment data size={}, "
+              + "suffix heap size={}," + "new segment heap size={}"
+              + "suffix off heap size={}," + "new segment off heap size={}"
+          , suffixDataSize
+          , newDataSize
+          , suffixHeapSize
+          , newHeapSize
+          , suffixOffHeapSize
+          , newOffHeapSize);
     }
     return true;
   }
@@ -160,6 +169,14 @@ public class CompactionPipeline {
     return res;
   }
 
+  private static long getSegmentsOffHeapSize(List<? extends Segment> list) {
+    long res = 0;
+    for (Segment segment : list) {
+      res += segment.offHeapSize();
+    }
+    return res;
+  }
+
   private static long getSegmentsKeySize(List<? extends Segment> list) {
     long res = 0;
     for (Segment segment : list) {
@@ -201,7 +218,8 @@ public class CompactionPipeline {
           if(region != null) {
             // update the global memstore size counter
             // upon flattening there is no change in the data size
-            region.addMemStoreSize(new MemStoreSize(0, newMemstoreAccounting.getHeapSize()));
+            region.addMemStoreSize(new MemStoreSize(0, newMemstoreAccounting.getHeapSize(),
+                newMemstoreAccounting.getOffHeapSize()));
           }
           LOG.debug("Compaction pipeline segment {} flattened", s);
           return true;
@@ -239,19 +257,16 @@ public class CompactionPipeline {
   public MemStoreSizing getTailSizing() {
     LinkedList<? extends Segment> localCopy = readOnlyCopy;
     if (localCopy.isEmpty()) return new MemStoreSizing();
-    return new MemStoreSizing(localCopy.peekLast().keySize(), localCopy.peekLast().heapSize());
+    return new MemStoreSizing(localCopy.peekLast().getMemStoreSize());
   }
 
   public MemStoreSizing getPipelineSizing() {
-    long keySize = 0;
-    long heapSize = 0;
+    MemStoreSizing memStoreSizing = new MemStoreSizing();
     LinkedList<? extends Segment> localCopy = readOnlyCopy;
-    if (localCopy.isEmpty()) return new MemStoreSizing();
     for (Segment segment : localCopy) {
-      keySize += segment.keySize();
-      heapSize += segment.heapSize();
+      memStoreSizing.incMemStoreSize(segment.getMemStoreSize());
     }
-    return new MemStoreSizing(keySize, heapSize);
+    return memStoreSizing;
   }
 
   private void swapSuffix(List<? extends Segment> suffix, ImmutableSegment segment,

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java
index 8bd990a..b6bbb59 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java
@@ -43,7 +43,7 @@ public class CompositeImmutableSegment extends ImmutableSegment {
   private long keySize = 0;
 
   public CompositeImmutableSegment(CellComparator comparator, List<ImmutableSegment> segments) {
-    super(comparator);
+    super(comparator, segments);
     this.segments = segments;
     for (ImmutableSegment s : segments) {
       this.timeRangeTracker.includeTimestamp(s.getTimeRangeTracker().getMax());
@@ -87,14 +87,6 @@ public class CompositeImmutableSegment extends ImmutableSegment {
   }
 
   /**
-   * @return the first cell in the segment that has equal or greater key than the given cell
-   */
-  @Override
-  public Cell getFirstAfter(Cell cell) {
-    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
-  }
-
-  /**
    * Closing a segment before it is being discarded
    */
   @Override
@@ -206,7 +198,7 @@ public class CompositeImmutableSegment extends ImmutableSegment {
    * Updates the heap size counter of the segment by the given delta
    */
   @Override
-  protected void incSize(long delta, long heapOverhead) {
+  protected void incSize(long delta, long heapOverhead, long offHeapOverhead) {
     throw new IllegalStateException("Not supported by CompositeImmutableScanner");
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
index 061e4d0..9ef6a6c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
@@ -106,7 +106,7 @@ public class DefaultMemStore extends AbstractMemStore {
   public MemStoreSize getFlushableSize() {
     MemStoreSize snapshotSize = getSnapshotSize();
     return snapshotSize.getDataSize() > 0 ? snapshotSize
-        : new MemStoreSize(keySize(), heapSize());
+        : new MemStoreSize(active.getMemStoreSize());
   }
 
   @Override
@@ -155,7 +155,7 @@ public class DefaultMemStore extends AbstractMemStore {
 
   @Override
   public MemStoreSize size() {
-    return new MemStoreSize(this.active.keySize(), this.active.heapSize());
+    return new MemStoreSize(active.getMemStoreSize());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java
index 0f01178..1ca20a2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java
@@ -44,7 +44,7 @@ public class FlushAllLargeStoresPolicy extends FlushLargeStoresPolicy {
       // Family number might also be zero in some of our unit test case
       return;
     }
-    this.flushSizeLowerBound = getFlushSizeLowerBound(region);
+    setFlushSizeLowerBounds(region);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
index 2d2de24..4da1857 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
@@ -43,10 +43,11 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
 
   protected long flushSizeLowerBound = -1;
 
-  protected long getFlushSizeLowerBound(HRegion region) { int familyNumber = region.getTableDescriptor().getColumnFamilyCount();
+  protected void setFlushSizeLowerBounds(HRegion region) {
+    int familyNumber = region.getTableDescriptor().getColumnFamilyCount();
     // For multiple families, lower bound is the "average flush size" by default
     // unless setting in configuration is larger.
-    long flushSizeLowerBound = region.getMemStoreFlushSize() / familyNumber;
+    flushSizeLowerBound = region.getMemStoreFlushSize() / familyNumber;
     long minimumLowerBound =
         getConf().getLong(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
           DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN);
@@ -57,36 +58,45 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
     String flushedSizeLowerBoundString =
         region.getTableDescriptor().getValue(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND);
     if (flushedSizeLowerBoundString == null) {
-      LOG.debug("No {} set in table {} descriptor;" +
-          "using region.getMemStoreFlushSize/# of families ({}) instead.",
-          HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
-          region.getTableDescriptor().getTableName(),
-          StringUtils.humanSize(flushSizeLowerBound) + ")");
+      LOG.debug("No {} set in table {} descriptor;"
+          + "using region.getMemStoreFlushHeapSize/# of families ({}) "
+          + "instead."
+          , HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND
+          , region.getTableDescriptor().getTableName()
+          , StringUtils.humanSize(flushSizeLowerBound)
+          + ")");
     } else {
       try {
         flushSizeLowerBound = Long.parseLong(flushedSizeLowerBoundString);
       } catch (NumberFormatException nfe) {
         // fall back for fault setting
-        LOG.warn("Number format exception parsing {} for table {}: {}, {}; " +
-            "using region.getMemStoreFlushSize/# of families ({}) instead.",
-            HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
-            region.getTableDescriptor().getTableName(),
-            flushedSizeLowerBoundString,
-            nfe,
-            flushSizeLowerBound);
+        LOG.warn("Number format exception parsing {} for table {}: {}, {}; "
+                + "using region.getMemStoreFlushHeapSize/# of families ({}) "
+                + "and region.getMemStoreFlushOffHeapSize/# of families ({}) "
+                + "instead."
+            , HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND
+            , region.getTableDescriptor().getTableName()
+            , flushedSizeLowerBoundString
+            , nfe
+            , flushSizeLowerBound
+        );
 
       }
     }
-    return flushSizeLowerBound;
   }
 
   protected boolean shouldFlush(HStore store) {
-    if (store.getMemStoreSize().getDataSize() > this.flushSizeLowerBound) {
-      LOG.debug("Flush {} of {}; memstoreSize={} > lowerBound={}",
-          store.getColumnFamilyName(),
-          region.getRegionInfo().getEncodedName(),
-          store.getMemStoreSize().getDataSize(),
-          this.flushSizeLowerBound);
+    if (store.getMemStoreSize().getHeapSize()
+        + store.getMemStoreSize().getOffHeapSize() > this.flushSizeLowerBound) {
+      LOG.debug("Flush {} of {}; "
+              + "heap memstoreSize={} +"
+              + "off heap memstoreSize={} > memstore lowerBound={}"
+          , store.getColumnFamilyName()
+          , region.getRegionInfo().getEncodedName()
+          , store.getMemStoreSize().getHeapSize()
+          , store.getMemStoreSize().getOffHeapSize()
+          , this.flushSizeLowerBound
+      );
       return true;
     }
     return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java
index ed23e3d..e95de9d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java
@@ -63,7 +63,7 @@ public class FlushNonSloppyStoresFirstPolicy extends FlushLargeStoresPolicy {
   @Override
   protected void configureForRegion(HRegion region) {
     super.configureForRegion(region);
-    this.flushSizeLowerBound = getFlushSizeLowerBound(region);
+    setFlushSizeLowerBounds(region);
     for (HStore store : region.stores.values()) {
       if (store.isSloppyMemStore()) {
         sloppyStores.add(store);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/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 9464fdb..e26cc43 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
@@ -65,7 +65,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -287,7 +286,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   // TODO: account for each registered handler in HeapSize computation
   private Map<String, com.google.protobuf.Service> coprocessorServiceHandlers = Maps.newHashMap();
 
-  private final AtomicLong memstoreDataSize = new AtomicLong(0);// Track data size in all memstores
+  // Track data size in all memstores
+  private final MemStoreSizing memStoreSize = new MemStoreSizing();
   private final RegionServicesForStores regionServicesForStores = new RegionServicesForStores(this);
 
   // Debug possible data loss due to WAL off
@@ -829,12 +829,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     if (flushSize <= 0) {
       flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
-        TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE);
+          TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE);
     }
     this.memstoreFlushSize = flushSize;
-    this.blockingMemStoreSize = this.memstoreFlushSize *
-        conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER,
-                HConstants.DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER);
+    long mult = conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER,
+        HConstants.DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER);
+    this.blockingMemStoreSize = this.memstoreFlushSize * mult;
   }
 
   /**
@@ -1192,32 +1192,38 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   /**
    * Increase the size of mem store in this region and the size of global mem
    * store
-   * @return the size of memstore in this region
    */
-  public long addAndGetMemStoreSize(MemStoreSize memstoreSize) {
+  public void incMemStoreSize(MemStoreSize memStoreSize) {
     if (this.rsAccounting != null) {
-      rsAccounting.incGlobalMemStoreSize(memstoreSize);
+      rsAccounting.incGlobalMemStoreSize(memStoreSize);
     }
-    long size = this.memstoreDataSize.addAndGet(memstoreSize.getDataSize());
-    checkNegativeMemStoreDataSize(size, memstoreSize.getDataSize());
-    return size;
+    long dataSize;
+    synchronized (this.memStoreSize) {
+      this.memStoreSize.incMemStoreSize(memStoreSize);
+      dataSize = this.memStoreSize.getDataSize();
+    }
+    checkNegativeMemStoreDataSize(dataSize, memStoreSize.getDataSize());
   }
 
-  public void decrMemStoreSize(MemStoreSize memstoreSize) {
+  public void decrMemStoreSize(MemStoreSize memStoreSize) {
     if (this.rsAccounting != null) {
-      rsAccounting.decGlobalMemStoreSize(memstoreSize);
+      rsAccounting.decGlobalMemStoreSize(memStoreSize);
+    }
+    long size;
+    synchronized (this.memStoreSize) {
+      this.memStoreSize.decMemStoreSize(memStoreSize);
+      size = this.memStoreSize.getDataSize();
     }
-    long size = this.memstoreDataSize.addAndGet(-memstoreSize.getDataSize());
-    checkNegativeMemStoreDataSize(size, -memstoreSize.getDataSize());
+    checkNegativeMemStoreDataSize(size, -memStoreSize.getDataSize());
   }
 
-  private void checkNegativeMemStoreDataSize(long memstoreDataSize, long delta) {
-    // This is extremely bad if we make memstoreSize negative. Log as much info on the offending
+  private void checkNegativeMemStoreDataSize(long memStoreDataSize, long delta) {
+    // This is extremely bad if we make memStoreSize negative. Log as much info on the offending
     // caller as possible. (memStoreSize might be a negative value already -- freeing memory)
-    if (memstoreDataSize < 0) {
+    if (memStoreDataSize < 0) {
       LOG.error("Asked to modify this region's (" + this.toString()
-          + ") memstoreSize to a negative value which is incorrect. Current memstoreSize="
-          + (memstoreDataSize - delta) + ", delta=" + delta, new Exception());
+          + ") memStoreSize to a negative value which is incorrect. Current memStoreSize="
+          + (memStoreDataSize - delta) + ", delta=" + delta, new Exception());
     }
   }
 
@@ -1250,8 +1256,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   @Override
-  public long getMemStoreSize() {
-    return memstoreDataSize.get();
+  public long getMemStoreDataSize() {
+    return memStoreSize.getDataSize();
+  }
+
+  @Override
+  public long getMemStoreHeapSize() {
+    return memStoreSize.getHeapSize();
+  }
+
+  @Override
+  public long getMemStoreOffHeapSize() {
+    return memStoreSize.getOffHeapSize();
   }
 
   /** @return store services for this region, to access services required by store level needs */
@@ -1521,7 +1537,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         int failedfFlushCount = 0;
         int flushCount = 0;
         long tmp = 0;
-        long remainingSize = this.memstoreDataSize.get();
+        long remainingSize = this.memStoreSize.getDataSize();
         while (remainingSize > 0) {
           try {
             internalFlushcache(status);
@@ -1530,7 +1546,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
                   " (carrying snapshot?) " + this);
             }
             flushCount++;
-            tmp = this.memstoreDataSize.get();
+            tmp = this.memStoreSize.getDataSize();
             if (tmp >= remainingSize) {
               failedfFlushCount++;
             }
@@ -1570,7 +1586,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               getRegionServerServices().abort("Assertion failed while closing store "
                 + getRegionInfo().getRegionNameAsString() + " " + store
                 + ". flushableSize expected=0, actual= " + flushableSize
-                + ". Current memstoreSize=" + getMemStoreSize() + ". Maybe a coprocessor "
+                + ". Current memStoreSize=" + getMemStoreDataSize() + ". Maybe a coprocessor "
                 + "operation failed and left the memstore in a partially updated state.", null);
             }
           }
@@ -1613,9 +1629,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
       this.closed.set(true);
       if (!canFlush) {
-        this.decrMemStoreSize(new MemStoreSizing(memstoreDataSize.get(), getMemStoreHeapSize()));
-      } else if (memstoreDataSize.get() != 0) {
-        LOG.error("Memstore size is " + memstoreDataSize.get());
+        this.decrMemStoreSize(new MemStoreSize(memStoreSize));
+      } else if (memStoreSize.getDataSize() != 0) {
+        LOG.error("Memstore data size is " + memStoreSize.getDataSize());
       }
       if (coprocessorHost != null) {
         status.setStatus("Running coprocessor post-close hooks");
@@ -1635,10 +1651,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  private long getMemStoreHeapSize() {
-    return stores.values().stream().mapToLong(s -> s.getMemStoreSize().getHeapSize()).sum();
-  }
-
   /** Wait for all current flushes and compactions of the region to complete */
   // TODO HBASE-18906. Check the usage (if any) in Phoenix and expose this or give alternate way for
   // Phoenix needs.
@@ -1752,7 +1764,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     * @return True if its worth doing a flush before we put up the close flag.
     */
   private boolean worthPreFlushing() {
-    return this.memstoreDataSize.get() >
+    return this.memStoreSize.getDataSize() >
       this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
   }
 
@@ -2370,12 +2382,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs
     // to no other that it can use to associate with the bulk load. Hence this little dance below
     // to go get one.
-    if (this.memstoreDataSize.get() <= 0) {
+    if (this.memStoreSize.getDataSize() <= 0) {
       // Take an update lock so no edits can come into memory just yet.
       this.updatesLock.writeLock().lock();
       WriteEntry writeEntry = null;
       try {
-        if (this.memstoreDataSize.get() <= 0) {
+        if (this.memStoreSize.getDataSize() <= 0) {
           // Presume that if there are still no edits in the memstore, then there are no edits for
           // this region out in the WAL subsystem so no need to do any trickery clearing out
           // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for
@@ -2511,8 +2523,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
             .append(StringUtils.byteDesc(store.getFlushableSize().getDataSize()));
       }
     }
-    LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() +
-        " column families, memstore=" + StringUtils.byteDesc(this.memstoreDataSize.get()) +
+    LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() + " column families," +
+        " memstore data size=" + StringUtils.byteDesc(this.memStoreSize.getDataSize()) +
+        " memstore heap size=" + StringUtils.byteDesc(this.memStoreSize.getHeapSize()) +
         ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
         ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId));
   }
@@ -2699,11 +2712,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
 
     long time = EnvironmentEdgeManager.currentTime() - startTime;
-    long memstoresize = this.memstoreDataSize.get();
-    String msg = "Finished memstore flush of ~"
-        + StringUtils.byteDesc(prepareResult.totalFlushableSize.getDataSize()) + "/"
-        + prepareResult.totalFlushableSize.getDataSize() + ", currentsize="
-        + StringUtils.byteDesc(memstoresize) + "/" + memstoresize
+    long flushableDataSize = prepareResult.totalFlushableSize.getDataSize();
+    long flushableHeapSize = prepareResult.totalFlushableSize.getHeapSize();
+    long memstoresize = this.memStoreSize.getDataSize();
+    String msg = "Finished memstore flush."
+        + " Flushed data size ~" + StringUtils.byteDesc(flushableDataSize) + "/" + flushableDataSize
+        + " Flushed Heap size ~" + StringUtils.byteDesc(flushableHeapSize) + "/" + flushableHeapSize
+        + ", currentsize=" + StringUtils.byteDesc(memstoresize) + "/" + memstoresize
         + " for region " + this + " in " + time + "ms, sequenceid="
         + flushOpSeqId +  ", compaction requested=" + compactionRequested
         + ((wal == null) ? "; wal=null" : "");
@@ -3037,7 +3052,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         return true;
       });
       // update memStore size
-      region.addAndGetMemStoreSize(memStoreAccounting);
+      region.incMemStoreSize(memStoreAccounting);
     }
 
     public boolean isDone() {
@@ -3806,8 +3821,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           initialized = true;
         }
         doMiniBatchMutate(batchOp);
-        long newSize = this.getMemStoreSize();
-        requestFlushIfNeeded(newSize);
+        requestFlushIfNeeded();
       }
     } finally {
       batchOp.closeRegionOperation();
@@ -4165,7 +4179,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // If catalog region, do not impose resource constraints or block updates.
     if (this.getRegionInfo().isMetaRegion()) return;
 
-    if (this.memstoreDataSize.get() > this.blockingMemStoreSize) {
+    if (this.memStoreSize.getHeapSize()
+        + this.memStoreSize.getOffHeapSize() > this.blockingMemStoreSize) {
       blockedRequestsCount.increment();
       requestFlush();
       // Don't print current limit because it will vary too much. The message is used as a key
@@ -4293,8 +4308,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param size
    * @return True if size is over the flush threshold
    */
-  private boolean isFlushSize(final long size) {
-    return size > this.memstoreFlushSize;
+  private boolean isFlushSize(MemStoreSize size) {
+    return size.getHeapSize() + size.getOffHeapSize() > getMemStoreFlushSize();
   }
 
   /**
@@ -4585,7 +4600,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
             rsAccounting.addRegionReplayEditsSize(getRegionInfo().getRegionName(),
                 memstoreSize);
           }
-          flush = isFlushSize(this.addAndGetMemStoreSize(memstoreSize));
+          incMemStoreSize(memstoreSize);
+          flush = isFlushSize(this.memStoreSize);
           if (flush) {
             internalFlushcache(null, currentEditSeqId, stores.values(), status, false,
               FlushLifeCycleTracker.DUMMY);
@@ -6522,7 +6538,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
             scannerContext.incrementBatchProgress(results.size());
             for (Cell cell : results) {
               scannerContext.incrementSizeProgress(PrivateCellUtil.estimatedSerializedSizeOf(cell),
-                PrivateCellUtil.estimatedHeapSizeOf(cell));
+                PrivateCellUtil.estimatedSizeOfCell(cell));
             }
           }
 
@@ -7264,7 +7280,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       return null;
     }
     ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
-    stats.setMemStoreLoad((int) (Math.min(100, (this.memstoreDataSize.get() * 100) / this
+    stats.setMemStoreLoad((int) (Math.min(100, (this.memStoreSize.getHeapSize() * 100) / this
         .memstoreFlushSize)));
     if (rsServices.getHeapMemoryManager() != null) {
       // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM,
@@ -7412,8 +7428,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     } finally {
       closeRegionOperation();
       if (!mutations.isEmpty()) {
-        long newSize = this.addAndGetMemStoreSize(memstoreAccounting);
-        requestFlushIfNeeded(newSize);
+        this.incMemStoreSize(memstoreAccounting);
+        requestFlushIfNeeded();
       }
     }
   }
@@ -7566,9 +7582,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         rowLock.release();
       }
       // Request a cache flush if over the limit.  Do it outside update lock.
-      if (isFlushSize(addAndGetMemStoreSize(memstoreAccounting))) {
-        requestFlush();
-      }
+      incMemStoreSize(memstoreAccounting);
+      requestFlushIfNeeded();
       closeRegionOperation(op);
       if (this.metricsRegion != null) {
         switch (op) {
@@ -7894,7 +7909,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
       ClassSize.OBJECT + // closeLock
       (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
-      (4 * ClassSize.ATOMIC_LONG) + // memStoreSize, numPutsWithoutWAL, dataInMemoryWithoutWAL,
+      (3 * ClassSize.ATOMIC_LONG) + // numPutsWithoutWAL, dataInMemoryWithoutWAL,
                                     // compactionsFailed
       (2 * ClassSize.CONCURRENT_HASHMAP) +  // lockedRows, scannerReadPoints
       WriteState.HEAP_SIZE + // writestate
@@ -7935,8 +7950,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
     if (coprocessorServiceHandlers.containsKey(serviceName)) {
       LOG.error("Coprocessor service " + serviceName +
-              " already registered, rejecting request from " + instance
-      );
+          " already registered, rejecting request from " + instance);
       return false;
     }
 
@@ -8211,8 +8225,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     dataInMemoryWithoutWAL.add(mutationSize);
   }
 
-  private void lock(final Lock lock)
-      throws RegionTooBusyException, InterruptedIOException {
+  private void lock(final Lock lock) throws RegionTooBusyException, InterruptedIOException {
     lock(lock, 1);
   }
 
@@ -8401,6 +8414,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return this.memstoreFlushSize;
   }
 
+
   //// method for debugging tests
   void throwException(String title, String regionName) {
     StringBuilder buf = new StringBuilder();
@@ -8416,7 +8430,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
     buf.append("end-of-stores");
     buf.append(", memstore size ");
-    buf.append(getMemStoreSize());
+    buf.append(getMemStoreDataSize());
     if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) {
       throw new RuntimeException(buf.toString());
     }
@@ -8447,8 +8461,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       RpcServer.getRequestUser().orElse(null));
   }
 
-  private void requestFlushIfNeeded(long memstoreTotalSize) throws RegionTooBusyException {
-    if (memstoreTotalSize > this.getMemStoreFlushSize()) {
+  private void requestFlushIfNeeded() throws RegionTooBusyException {
+    if(isFlushSize(memStoreSize)) {
       requestFlush();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/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 0d59b12..e2aac03 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
@@ -1606,7 +1606,7 @@ public class HRegionServer extends HasThread implements
     int storefiles = 0;
     int storeUncompressedSizeMB = 0;
     int storefileSizeMB = 0;
-    int memstoreSizeMB = (int) (r.getMemStoreSize() / 1024 / 1024);
+    int memstoreSizeMB = (int) (r.getMemStoreDataSize() / 1024 / 1024);
     long storefileIndexSizeKB = 0;
     int rootLevelIndexSizeKB = 0;
     int totalStaticIndexSizeKB = 0;
@@ -2743,11 +2743,11 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * @return A new Map of online regions sorted by region size with the first entry being the
-   * biggest.  If two regions are the same size, then the last one found wins; i.e. this method
-   * may NOT return all regions.
+   * @return A new Map of online regions sorted by region off-heap size with the first entry being
+   *   the biggest.  If two regions are the same size, then the last one found wins; i.e. this
+   *   method may NOT return all regions.
    */
-  SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
+  SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedByOffHeapSize() {
     // we'll sort the regions in reverse
     SortedMap<Long, HRegion> sortedRegions = new TreeMap<>(
         new Comparator<Long>() {
@@ -2758,7 +2758,28 @@ public class HRegionServer extends HasThread implements
         });
     // Copy over all regions. Regions are sorted by size with biggest first.
     for (HRegion region : this.onlineRegions.values()) {
-      sortedRegions.put(region.getMemStoreSize(), region);
+      sortedRegions.put(region.getMemStoreOffHeapSize(), region);
+    }
+    return sortedRegions;
+  }
+
+  /**
+   * @return A new Map of online regions sorted by region heap size with the first entry being the
+   *   biggest.  If two regions are the same size, then the last one found wins; i.e. this method
+   *   may NOT return all regions.
+   */
+  SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedByOnHeapSize() {
+    // we'll sort the regions in reverse
+    SortedMap<Long, HRegion> sortedRegions = new TreeMap<>(
+        new Comparator<Long>() {
+          @Override
+          public int compare(Long a, Long b) {
+            return -1 * a.compareTo(b);
+          }
+        });
+    // Copy over all regions. Regions are sorted by size with biggest first.
+    for (HRegion region : this.onlineRegions.values()) {
+      sortedRegions.put(region.getMemStoreHeapSize(), region);
     }
     return sortedRegions;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index f283a65..bef50b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -2188,7 +2188,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
       this.cacheFlushCount = snapshot.getCellsCount();
       this.cacheFlushSize = snapshot.getDataSize();
       committedFiles = new ArrayList<>(1);
-      return new MemStoreSize(snapshot.getDataSize(), snapshot.getHeapSize());
+      return new MemStoreSize(snapshot.getMemStoreSize());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java
index 871f526..71648a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableMemStoreLAB.java
@@ -133,4 +133,16 @@ public class ImmutableMemStoreLAB implements MemStoreLAB {
       checkAndCloseMSLABs(count);
     }
   }
+
+  @Override
+  public boolean isOnHeap() {
+    return !isOffHeap();
+  }
+
+  @Override
+  public boolean isOffHeap() {
+    return ChunkCreator.getInstance().isOffheap();
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java
index c899eab..b781aab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableSegment.java
@@ -48,6 +48,10 @@ public abstract class ImmutableSegment extends Segment {
     super(comparator, TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC));
   }
 
+  protected ImmutableSegment(CellComparator comparator, List<ImmutableSegment> segments) {
+    super(comparator, segments, TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC));
+  }
+
   /**------------------------------------------------------------------------
    * C-tor to be used to build the derived classes
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
index d7c7c5a..6e4191e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
@@ -87,6 +87,8 @@ class MemStoreFlusher implements FlushRequester {
   private final FlushHandler[] flushHandlers;
   private List<FlushRequestListener> flushRequestListeners = new ArrayList<>(1);
 
+  private FlushType flushType;
+
   /**
    * @param conf
    * @param server
@@ -116,6 +118,10 @@ class MemStoreFlusher implements FlushRequester {
     return this.updatesBlockedMsHighWater;
   }
 
+  public void setFlushType(FlushType flushType) {
+    this.flushType = flushType;
+  }
+
   /**
    * The memstore across all regions has exceeded the low water mark. Pick
    * one region to flush and flush it synchronously (this is called from the
@@ -123,7 +129,17 @@ class MemStoreFlusher implements FlushRequester {
    * @return true if successful
    */
   private boolean flushOneForGlobalPressure() {
-    SortedMap<Long, HRegion> regionsBySize = server.getCopyOfOnlineRegionsSortedBySize();
+    SortedMap<Long, HRegion> regionsBySize = null;
+    switch(flushType) {
+      case ABOVE_OFFHEAP_HIGHER_MARK:
+      case ABOVE_OFFHEAP_LOWER_MARK:
+        regionsBySize = server.getCopyOfOnlineRegionsSortedByOffHeapSize();
+        break;
+      case ABOVE_ONHEAP_HIGHER_MARK:
+      case ABOVE_ONHEAP_LOWER_MARK:
+      default:
+        regionsBySize = server.getCopyOfOnlineRegionsSortedByOnHeapSize();
+    }
     Set<HRegion> excludedRegions = new HashSet<>();
 
     double secondaryMultiplier
@@ -147,8 +163,25 @@ class MemStoreFlusher implements FlushRequester {
       }
 
       HRegion regionToFlush;
+      long bestAnyRegionSize;
+      long bestFlushableRegionSize;
+      switch(flushType) {
+        case ABOVE_OFFHEAP_HIGHER_MARK:
+        case ABOVE_OFFHEAP_LOWER_MARK:
+          bestAnyRegionSize = bestAnyRegion.getMemStoreOffHeapSize();
+          bestFlushableRegionSize = bestFlushableRegion.getMemStoreOffHeapSize();
+          break;
+        case ABOVE_ONHEAP_HIGHER_MARK:
+        case ABOVE_ONHEAP_LOWER_MARK:
+          bestAnyRegionSize = bestAnyRegion.getMemStoreHeapSize();
+          bestFlushableRegionSize = bestFlushableRegion.getMemStoreHeapSize();
+          break;
+        default:
+          bestAnyRegionSize = bestAnyRegion.getMemStoreDataSize();
+          bestFlushableRegionSize = bestFlushableRegion.getMemStoreDataSize();
+      }
       if (bestFlushableRegion != null &&
-          bestAnyRegion.getMemStoreSize() > 2 * bestFlushableRegion.getMemStoreSize()) {
+          bestAnyRegionSize > 2 * bestFlushableRegionSize) {
         // Even if it's not supposed to be flushed, pick a region if it's more than twice
         // as big as the best flushable one - otherwise when we're under pressure we make
         // lots of little flushes and cause lots of compactions, etc, which just makes
@@ -157,9 +190,10 @@ class MemStoreFlusher implements FlushRequester {
           LOG.debug("Under global heap pressure: " + "Region "
               + bestAnyRegion.getRegionInfo().getRegionNameAsString()
               + " has too many " + "store files, but is "
-              + TraditionalBinaryPrefix.long2String(bestAnyRegion.getMemStoreSize(), "", 1)
+              + TraditionalBinaryPrefix.long2String(bestAnyRegionSize, "", 1)
               + " vs best flushable region's "
-              + TraditionalBinaryPrefix.long2String(bestFlushableRegion.getMemStoreSize(), "", 1)
+              + TraditionalBinaryPrefix.long2String(
+              bestFlushableRegionSize, "", 1)
               + ". Choosing the bigger.");
         }
         regionToFlush = bestAnyRegion;
@@ -171,19 +205,36 @@ class MemStoreFlusher implements FlushRequester {
         }
       }
 
+      long regionToFlushSize;
+      long bestRegionReplicaSize;
+      switch(flushType) {
+        case ABOVE_OFFHEAP_HIGHER_MARK:
+        case ABOVE_OFFHEAP_LOWER_MARK:
+          regionToFlushSize = regionToFlush.getMemStoreOffHeapSize();
+          bestRegionReplicaSize = bestRegionReplica.getMemStoreOffHeapSize();
+          break;
+        case ABOVE_ONHEAP_HIGHER_MARK:
+        case ABOVE_ONHEAP_LOWER_MARK:
+          regionToFlushSize = regionToFlush.getMemStoreHeapSize();
+          bestRegionReplicaSize = bestRegionReplica.getMemStoreHeapSize();
+          break;
+        default:
+          regionToFlushSize = regionToFlush.getMemStoreDataSize();
+          bestRegionReplicaSize = bestRegionReplica.getMemStoreDataSize();
+      }
+
       Preconditions.checkState(
-        (regionToFlush != null && regionToFlush.getMemStoreSize() > 0) ||
-        (bestRegionReplica != null && bestRegionReplica.getMemStoreSize() > 0));
+        (regionToFlush != null && regionToFlushSize > 0) ||
+        (bestRegionReplica != null && bestRegionReplicaSize > 0));
 
       if (regionToFlush == null ||
           (bestRegionReplica != null &&
            ServerRegionReplicaUtil.isRegionReplicaStoreFileRefreshEnabled(conf) &&
-           (bestRegionReplica.getMemStoreSize()
-               > secondaryMultiplier * regionToFlush.getMemStoreSize()))) {
+           (bestRegionReplicaSize > secondaryMultiplier * regionToFlushSize))) {
         LOG.info("Refreshing storefiles of region " + bestRegionReplica +
-            " due to global heap pressure. Total memstore datasize=" +
+            " due to global heap pressure. Total memstore off heap size=" +
             TraditionalBinaryPrefix.long2String(
-              server.getRegionServerAccounting().getGlobalMemStoreDataSize(), "", 1) +
+              server.getRegionServerAccounting().getGlobalMemStoreOffHeapSize(), "", 1) +
             " memstore heap size=" + TraditionalBinaryPrefix.long2String(
               server.getRegionServerAccounting().getGlobalMemStoreHeapSize(), "", 1));
         flushedOne = refreshStoreFilesAndReclaimMemory(bestRegionReplica);
@@ -194,11 +245,15 @@ class MemStoreFlusher implements FlushRequester {
         }
       } else {
         LOG.info("Flush of region " + regionToFlush + " due to global heap pressure. " +
-            "Total Memstore size=" +
+            "Flush type=" + flushType.toString() +
+            "Total Memstore Heap size=" +
             TraditionalBinaryPrefix.long2String(
-              server.getRegionServerAccounting().getGlobalMemStoreDataSize(), "", 1) +
+                server.getRegionServerAccounting().getGlobalMemStoreHeapSize(), "", 1) +
+            "Total Memstore Off-Heap size=" +
+            TraditionalBinaryPrefix.long2String(
+                server.getRegionServerAccounting().getGlobalMemStoreOffHeapSize(), "", 1) +
             ", Region memstore size=" +
-            TraditionalBinaryPrefix.long2String(regionToFlush.getMemStoreSize(), "", 1));
+            TraditionalBinaryPrefix.long2String(regionToFlushSize, "", 1));
         flushedOne = flushRegion(regionToFlush, true, false, FlushLifeCycleTracker.DUMMY);
 
         if (!flushedOne) {
@@ -582,6 +637,7 @@ class MemStoreFlusher implements FlushRequester {
         try {
           flushType = isAboveHighWaterMark();
           while (flushType != FlushType.NORMAL && !server.isStopped()) {
+            server.cacheFlusher.setFlushType(flushType);
             if (!blocked) {
               startTime = EnvironmentEdgeManager.currentTime();
               if (!server.getRegionServerAccounting().isOffheap()) {
@@ -592,7 +648,7 @@ class MemStoreFlusher implements FlushRequester {
                 switch (flushType) {
                 case ABOVE_OFFHEAP_HIGHER_MARK:
                   logMsg("the global offheap memstore datasize",
-                      server.getRegionServerAccounting().getGlobalMemStoreDataSize(),
+                      server.getRegionServerAccounting().getGlobalMemStoreOffHeapSize(),
                       server.getRegionServerAccounting().getGlobalMemStoreLimit());
                   break;
                 case ABOVE_ONHEAP_HIGHER_MARK:
@@ -633,8 +689,12 @@ class MemStoreFlusher implements FlushRequester {
           LOG.info("Unblocking updates for server " + server.toString());
         }
       }
-    } else if (isAboveLowWaterMark() != FlushType.NORMAL) {
-      wakeupFlushThread();
+    } else {
+      flushType = isAboveLowWaterMark();
+      if (flushType != FlushType.NORMAL) {
+        server.cacheFlusher.setFlushType(flushType);
+        wakeupFlushThread();
+      }
     }
     if(scope!= null) {
       scope.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
index 6bc8886..8b77981 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
@@ -114,7 +114,7 @@ public interface MemStoreLAB {
   */
   Chunk getNewExternalJumboChunk(int size);
 
-  public static MemStoreLAB newInstance(Configuration conf) {
+  static MemStoreLAB newInstance(Configuration conf) {
     MemStoreLAB memStoreLAB = null;
     if (isEnabled(conf)) {
       String className = conf.get(MSLAB_CLASS_NAME, MemStoreLABImpl.class.getName());
@@ -124,7 +124,11 @@ public interface MemStoreLAB {
     return memStoreLAB;
   }
 
-  public static boolean isEnabled(Configuration conf) {
+  static boolean isEnabled(Configuration conf) {
     return conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT);
   }
+
+  boolean isOnHeap();
+
+  boolean isOffHeap();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index f7728ac..4ff0480 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -317,6 +317,16 @@ public class MemStoreLABImpl implements MemStoreLAB {
     return c;
   }
 
+  @Override
+  public boolean isOnHeap() {
+    return !isOffHeap();
+  }
+
+  @Override
+  public boolean isOffHeap() {
+    return this.chunkCreator.isOffheap();
+  }
+
   @VisibleForTesting
   Chunk getCurrentChunk() {
     return this.curChunk.get();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java
index 557a61a..382e6e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java
@@ -27,29 +27,58 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 public class MemStoreSize {
+  // MemStore size tracks 3 sizes:
+  // (1) data size: the aggregated size of all key-value not including meta data such as
+  // index, time range etc.
+  // (2) heap size: the aggregated size of all data that is allocated on-heap including all
+  // key-values that reside on-heap and the metadata that resides on-heap
+  // (3) off-heap size: the aggregated size of all data that is allocated off-heap including all
+  // key-values that reside off-heap and the metadata that resides off-heap
+  //
+  // 3 examples to illustrate their usage:
+  // Consider a store with 100MB of key-values allocated on-heap and 20MB of metadata allocated
+  // on-heap. The counters are <100MB, 120MB, 0>, respectively.
+  // Consider a store with 100MB of key-values allocated off-heap and 20MB of metadata
+  // allocated on-heap (e.g, CAM index). The counters are <100MB, 20MB, 100MB>, respectively.
+  // Consider a store with 100MB of key-values from which 95MB are allocated off-heap and 5MB
+  // are allocated on-heap (e.g., due to upserts) and 20MB of metadata from which 15MB allocated
+  // off-heap (e.g, CCM index) and 5MB allocated on-heap (e.g, CSLM index in active).
+  // The counters are <100MB, 10MB, 110MB>, respectively.
+
   /**
    *'dataSize' tracks the Cell's data bytes size alone (Key bytes, value bytes). A cell's data can
    * be in on heap or off heap area depending on the MSLAB and its configuration to be using on heap
    * or off heap LABs
    */
-  protected long dataSize;
+  protected volatile long dataSize;
 
   /** 'heapSize' tracks all Cell's heap size occupancy. This will include Cell POJO heap overhead.
    * When Cells in on heap area, this will include the cells data size as well.
    */
-  protected long heapSize;
+  protected volatile long heapSize;
+
+  /** off-heap size: the aggregated size of all data that is allocated off-heap including all
+   * key-values that reside off-heap and the metadata that resides off-heap
+   */
+  protected volatile long offHeapSize;
 
   public MemStoreSize() {
-    this(0L, 0L);
+    this(0L, 0L, 0L);
   }
 
-  public MemStoreSize(long dataSize, long heapSize) {
+  public MemStoreSize(long dataSize, long heapSize, long offHeapSize) {
     this.dataSize = dataSize;
     this.heapSize = heapSize;
+    this.offHeapSize = offHeapSize;
   }
 
+  protected MemStoreSize(MemStoreSize memStoreSize) {
+    this.dataSize = memStoreSize.dataSize;
+    this.heapSize = memStoreSize.heapSize;
+    this.offHeapSize = memStoreSize.offHeapSize;
+  }
   public boolean isEmpty() {
-    return this.dataSize == 0 && this.heapSize == 0;
+    return this.dataSize == 0 && this.heapSize == 0 && this.offHeapSize == 0;
   }
 
   public long getDataSize() {
@@ -60,24 +89,33 @@ public class MemStoreSize {
     return this.heapSize;
   }
 
+  public long getOffHeapSize() {
+    return this.offHeapSize;
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (obj == null || getClass() != obj.getClass()) {
       return false;
     }
     MemStoreSize other = (MemStoreSize) obj;
-    return this.dataSize == other.dataSize && this.heapSize == other.heapSize;
+    return this.dataSize == other.dataSize
+        && this.heapSize == other.heapSize
+        && this.offHeapSize == other.offHeapSize;
   }
 
   @Override
   public int hashCode() {
     long h = 13 * this.dataSize;
     h = h + 14 * this.heapSize;
+    h = h + 15 * this.offHeapSize;
     return (int) h;
   }
 
   @Override
   public String toString() {
-    return "dataSize=" + this.dataSize + " , heapSize=" + this.heapSize;
+    return "dataSize=" + this.dataSize
+        + " , heapSize=" + this.heapSize
+        + " , offHeapSize=" + this.offHeapSize;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSizing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSizing.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSizing.java
index b13201d..0b3e925 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSizing.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSizing.java
@@ -28,23 +28,14 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class MemStoreSizing extends MemStoreSize {
   public static final MemStoreSizing DUD = new MemStoreSizing() {
-    @Override
-    public void incMemStoreSize(MemStoreSize delta) {
-      incMemStoreSize(delta.getDataSize(), delta.getHeapSize());
-    }
 
-    @Override
-    public void incMemStoreSize(long dataSizeDelta, long heapSizeDelta) {
+    @Override public void incMemStoreSize(long dataSizeDelta, long heapSizeDelta,
+        long offHeapSizeDelta) {
       throw new RuntimeException("I'm a dud, you can't use me!");
     }
 
-    @Override
-    public void decMemStoreSize(MemStoreSize delta) {
-      decMemStoreSize(delta.getDataSize(), delta.getHeapSize());
-    }
-
-    @Override
-    public void decMemStoreSize(long dataSizeDelta, long heapSizeDelta) {
+    @Override public void decMemStoreSize(long dataSizeDelta, long heapSizeDelta,
+        long offHeapSizeDelta) {
       throw new RuntimeException("I'm a dud, you can't use me!");
     }
   };
@@ -53,51 +44,38 @@ public class MemStoreSizing extends MemStoreSize {
     super();
   }
 
-  public MemStoreSizing(long dataSize, long heapSize) {
-    super(dataSize, heapSize);
+  public MemStoreSizing(long dataSize, long heapSize, long offHeapSize) {
+    super(dataSize, heapSize, offHeapSize);
   }
 
-  public void incMemStoreSize(long dataSizeDelta, long heapSizeDelta) {
+  public MemStoreSizing(MemStoreSize memStoreSize) {
+    super(memStoreSize);
+  }
+
+  public void incMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta) {
     this.dataSize += dataSizeDelta;
     this.heapSize += heapSizeDelta;
+    this.offHeapSize += offHeapSizeDelta;
   }
 
   public void incMemStoreSize(MemStoreSize delta) {
-    incMemStoreSize(delta.getDataSize(), delta.getHeapSize());
+    incMemStoreSize(delta.getDataSize(), delta.getHeapSize(), delta.getOffHeapSize());
   }
 
-  public void decMemStoreSize(long dataSizeDelta, long heapSizeDelta) {
+  public void decMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta) {
     this.dataSize -= dataSizeDelta;
     this.heapSize -= heapSizeDelta;
+    this.offHeapSize -= offHeapSizeDelta;
   }
 
   public void decMemStoreSize(MemStoreSize delta) {
-    decMemStoreSize(delta.getDataSize(), delta.getHeapSize());
+    decMemStoreSize(delta.getDataSize(), delta.getHeapSize(), delta.getOffHeapSize());
   }
 
   public void empty() {
     this.dataSize = 0L;
     this.heapSize = 0L;
+    this.offHeapSize = 0L;
   }
 
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null || (getClass() != obj.getClass())) {
-      return false;
-    }
-    MemStoreSizing other = (MemStoreSizing) obj;
-    return this.dataSize == other.dataSize && this.heapSize == other.heapSize;
-  }
-
-  @Override
-  public int hashCode() {
-    long h = 13 * this.dataSize;
-    h = h + 14 * this.heapSize;
-    return (int) h;
-  }
-
-  @Override
-  public String toString() {
-    return "dataSize=" + this.dataSize + " , heapSize=" + this.heapSize;
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java
index 1a0317d..cbd60e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSnapshot.java
@@ -30,8 +30,7 @@ import java.util.List;
 public class MemStoreSnapshot implements Closeable {
   private final long id;
   private final int cellsCount;
-  private final long dataSize;
-  private final long heapSize;
+  private final MemStoreSize memStoreSize;
   private final TimeRangeTracker timeRangeTracker;
   private final List<KeyValueScanner> scanners;
   private final boolean tagsPresent;
@@ -39,8 +38,7 @@ public class MemStoreSnapshot implements Closeable {
   public MemStoreSnapshot(long id, ImmutableSegment snapshot) {
     this.id = id;
     this.cellsCount = snapshot.getCellsCount();
-    this.dataSize = snapshot.keySize();
-    this.heapSize = snapshot.heapSize();
+    this.memStoreSize = snapshot.getMemStoreSize();
     this.timeRangeTracker = snapshot.getTimeRangeTracker();
     this.scanners = snapshot.getScanners(Long.MAX_VALUE, Long.MAX_VALUE);
     this.tagsPresent = snapshot.isTagsPresent();
@@ -60,15 +58,12 @@ public class MemStoreSnapshot implements Closeable {
     return cellsCount;
   }
 
-  /**
-   * @return Total memory size occupied by this snapshot.
-   */
   public long getDataSize() {
-    return dataSize;
+    return memStoreSize.getDataSize();
   }
 
-  public long getHeapSize() {
-    return heapSize;
+  public MemStoreSize getMemStoreSize() {
+    return memStoreSize;
   }
 
   /**
@@ -100,4 +95,5 @@ public class MemStoreSnapshot implements Closeable {
       }
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
index 168b42e..f06f747 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
@@ -72,7 +72,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
         for (Store store : r.getStores()) {
           tempStorefilesSize += store.getStorefilesSize();
         }
-        metricsTable.setMemStoresSize(metricsTable.getMemStoresSize() + r.getMemStoreSize());
+        metricsTable.setMemStoresSize(metricsTable.getMemStoresSize() + r.getMemStoreDataSize());
         metricsTable.setStoreFilesSize(metricsTable.getStoreFilesSize() + tempStorefilesSize);
         metricsTable.setTableSize(metricsTable.getMemStoresSize() + metricsTable.getStoreFilesSize());
         metricsTable.setReadRequestsCount(metricsTable.getReadRequestsCount() + r.getReadRequestsCount());

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
index fe7bdf9..1349921 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
@@ -44,7 +44,7 @@ public class MutableSegment extends Segment {
 
   protected MutableSegment(CellSet cellSet, CellComparator comparator, MemStoreLAB memStoreLAB) {
     super(cellSet, comparator, memStoreLAB, TimeRangeTracker.create(TimeRangeTracker.Type.SYNC));
-    incSize(0,DEEP_OVERHEAD); // update the mutable segment metadata
+    incSize(0, DEEP_OVERHEAD, 0); // update the mutable segment metadata
   }
 
   /**
@@ -88,9 +88,10 @@ public class MutableSegment extends Segment {
             // removed cell is from MSLAB or not. Will do once HBASE-16438 is in
             int cellLen = getCellLength(cur);
             long heapSize = heapSizeChange(cur, true);
-            this.incSize(-cellLen, -heapSize);
+            long offHeapSize = offHeapSizeChange(cur, true);
+            this.incSize(-cellLen, -heapSize, -offHeapSize);
             if (memStoreSizing != null) {
-              memStoreSizing.decMemStoreSize(cellLen, heapSize);
+              memStoreSizing.decMemStoreSize(cellLen, heapSize, offHeapSize);
             }
             it.remove();
           } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 52d01fe..27771ce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -150,7 +150,21 @@ public interface Region extends ConfigurationObserver {
    *         the memstores of this Region. Means size in bytes for key, value and tags within Cells.
    *         It wont consider any java heap overhead for the cell objects or any other.
    */
-  long getMemStoreSize();
+  long getMemStoreDataSize();
+
+  /**
+   * @return memstore heap size for this region, in bytes. It accounts data size of cells
+   *         added to the memstores of this Region, as well as java heap overhead for the cell
+   *         objects or any other.
+   */
+  long getMemStoreHeapSize();
+
+  /**
+   * @return memstore off-heap size for this region, in bytes. It accounts data size of cells
+   *         added to the memstores of this Region, as well as overhead for the cell
+   *         objects or any other that is allocated off-heap.
+   */
+  long getMemStoreOffHeapSize();
 
   /** @return the number of mutations processed bypassing the WAL */
   long getNumMutationsWithoutWAL();


[20/50] [abbrv] hbase git commit: HBASE-20029 @Ignore TestQuotaThrottle and TestReplicasClient#testCancelOfMultiGet

Posted by zh...@apache.org.
HBASE-20029 @Ignore TestQuotaThrottle and TestReplicasClient#testCancelOfMultiGet


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

Branch: refs/heads/HBASE-19064
Commit: 148356a7bfeacce17aa85029348e5548579e78d6
Parents: 51cea3e
Author: Michael Stack <st...@apache.org>
Authored: Tue Feb 20 12:08:35 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Feb 20 12:09:47 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/client/TestReplicasClient.java    | 2 ++
 .../java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java     | 2 ++
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/148356a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index c55c6ca..311f651 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -61,6 +61,7 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
@@ -616,6 +617,7 @@ public class TestReplicasClient {
     }
   }
 
+  @Ignore // Disabled because it is flakey. Fails 17% on constrained GCE. %3 on Apache.
   @Test
   public void testCancelOfMultiGet() throws Exception {
     openRegion(hriSecondary);

http://git-wip-us.apache.org/repos/asf/hbase/blob/148356a7/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
index a7b8d9d..3b06d92 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
@@ -41,11 +41,13 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@Ignore // Disabled because flakey. Fails ~30% on a resource constrained GCE though not on Apache.
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestQuotaThrottle {
 


[11/50] [abbrv] hbase git commit: HBASE-20015 TestMergeTableRegionsProcedure and TestRegionMergeTransactionOnCluster flakey

Posted by zh...@apache.org.
HBASE-20015 TestMergeTableRegionsProcedure and TestRegionMergeTransactionOnCluster flakey


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

Branch: refs/heads/HBASE-19064
Commit: f3ff55a2b4bb7a8b4980fdbb5b1f7a8d033631f3
Parents: 68d509b
Author: Michael Stack <st...@apache.org>
Authored: Fri Feb 16 22:59:32 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 17 11:38:26 2018 -0800

----------------------------------------------------------------------
 .../master/assignment/MergeTableRegionsProcedure.java | 14 ++++++++++----
 .../master/assignment/SplitTableRegionProcedure.java  |  1 +
 2 files changed, 11 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f3ff55a2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 4bccab7..bd41208 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -255,10 +255,16 @@ public class MergeTableRegionsProcedure
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
       }
     } catch (IOException e) {
-      LOG.warn("Error trying to merge regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
-        " in the table " + getTableName() + " (in state=" + state + ")", e);
-
-      setFailure("master-merge-regions", e);
+      String msg = "Error trying to merge regions " +
+        RegionInfo.getShortNameToLog(regionsToMerge) + " in the table " + getTableName() +
+           " (in state=" + state + ")";
+      if (!isRollbackSupported(state)) {
+        // We reach a state that cannot be rolled back. We just need to keep retry.
+        LOG.warn(msg, e);
+      } else {
+        LOG.error(msg, e);
+        setFailure("master-merge-regions", e);
+      }
     }
     return Flow.HAS_MORE_STATE;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3ff55a2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index c139566..e898d6a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -270,6 +270,7 @@ public class SplitTableRegionProcedure
       } else {
         LOG.error(msg, e);
         setFailure(e);
+        setFailure("master-split-regions", e);
       }
     }
     // if split fails,  need to call ((HRegion)parent).clearSplit() when it is a force split


[22/50] [abbrv] hbase git commit: HBASE-19954 Separate TestBlockReorder into individual tests to avoid ShutdownHook suppression error against hadoop3

Posted by zh...@apache.org.
HBASE-19954 Separate TestBlockReorder into individual tests to avoid ShutdownHook suppression error against hadoop3


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

Branch: refs/heads/HBASE-19064
Commit: b75a8f4047e2a33031b121dfe091a992809ebae1
Parents: 6b84393
Author: Ted Yu <yu...@gmail.com>
Authored: Tue Feb 20 14:56:20 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Feb 20 14:56:20 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/fs/TestBlockReorder.java       | 258 +----------------
 .../hbase/fs/TestBlockReorderBlockLocation.java | 159 +++++++++++
 .../hbase/fs/TestBlockReorderMultiBlocks.java   | 279 +++++++++++++++++++
 3 files changed, 443 insertions(+), 253 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b75a8f40/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
index 59d2229..a75c0d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
@@ -17,15 +17,10 @@
  */
 package org.apache.hadoop.hbase.fs;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.BindException;
 import java.net.ServerSocket;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -35,32 +30,14 @@ 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.HConstants;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.ipc.RemoteException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -74,6 +51,11 @@ import org.slf4j.LoggerFactory;
 
 /**
  * Tests for the hdfs fix from HBASE-6435.
+ *
+ * Please don't add new subtest which involves starting / stopping MiniDFSCluster in this class.
+ * When stopping MiniDFSCluster, shutdown hooks would be cleared in hadoop's ShutdownHookManager
+ *   in hadoop 3.
+ * This leads to 'Failed suppression of fs shutdown hook' error in region server.
  */
 @Category({MiscTests.class, LargeTests.class})
 public class TestBlockReorder {
@@ -249,234 +231,4 @@ public class TestBlockReorder {
     }
   }
 
-  /**
-   * Test that the hook works within HBase, including when there are multiple blocks.
-   */
-  @Test()
-  public void testHBaseCluster() throws Exception {
-    byte[] sb = Bytes.toBytes("sb");
-    htu.startMiniZKCluster();
-
-    MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 1);
-    hbm.waitForActiveAndReadyMaster();
-    HRegionServer targetRs = LoadBalancer.isTablesOnMaster(hbm.getConf())? hbm.getMaster():
-      hbm.getRegionServer(0);
-
-    // We want to have a datanode with the same name as the region server, so
-    //  we're going to get the regionservername, and start a new datanode with this name.
-    String host4 = targetRs.getServerName().getHostname();
-    LOG.info("Starting a new datanode with the name=" + host4);
-    cluster.startDataNodes(conf, 1, true, null, new String[]{"/r4"}, new String[]{host4}, null);
-    cluster.waitClusterUp();
-
-    final int repCount = 3;
-
-    // We use the regionserver file system & conf as we expect it to have the hook.
-    conf = targetRs.getConfiguration();
-    HFileSystem rfs = (HFileSystem) targetRs.getFileSystem();
-    Table h = htu.createTable(TableName.valueOf(name.getMethodName()), sb);
-
-    // Now, we have 4 datanodes and a replication count of 3. So we don't know if the datanode
-    // with the same node will be used. We can't really stop an existing datanode, this would
-    // make us fall in nasty hdfs bugs/issues. So we're going to try multiple times.
-
-    // Now we need to find the log file, its locations, and look at it
-
-    String rootDir = new Path(FSUtils.getRootDir(conf) + "/" + HConstants.HREGION_LOGDIR_NAME +
-            "/" + targetRs.getServerName().toString()).toUri().getPath();
-
-    DistributedFileSystem mdfs = (DistributedFileSystem)
-        hbm.getMaster().getMasterFileSystem().getFileSystem();
-
-
-    int nbTest = 0;
-    while (nbTest < 10) {
-      final List<HRegion> regions = targetRs.getRegions(h.getName());
-      final CountDownLatch latch = new CountDownLatch(regions.size());
-      // listen for successful log rolls
-      final WALActionsListener listener = new WALActionsListener() {
-            @Override
-            public void postLogRoll(final Path oldPath, final Path newPath) throws IOException {
-              latch.countDown();
-            }
-          };
-      for (HRegion region : regions) {
-        region.getWAL().registerWALActionsListener(listener);
-      }
-
-      htu.getAdmin().rollWALWriter(targetRs.getServerName());
-
-      // wait
-      try {
-        latch.await();
-      } catch (InterruptedException exception) {
-        LOG.warn("Interrupted while waiting for the wal of '" + targetRs + "' to roll. If later " +
-            "tests fail, it's probably because we should still be waiting.");
-        Thread.currentThread().interrupt();
-      }
-      for (Region region : regions) {
-        ((HRegion)region).getWAL().unregisterWALActionsListener(listener);
-      }
-
-      // We need a sleep as the namenode is informed asynchronously
-      Thread.sleep(100);
-
-      // insert one put to ensure a minimal size
-      Put p = new Put(sb);
-      p.addColumn(sb, sb, sb);
-      h.put(p);
-
-      DirectoryListing dl = dfs.getClient().listPaths(rootDir, HdfsFileStatus.EMPTY_NAME);
-      HdfsFileStatus[] hfs = dl.getPartialListing();
-
-      // As we wrote a put, we should have at least one log file.
-      Assert.assertTrue(hfs.length >= 1);
-      for (HdfsFileStatus hf : hfs) {
-        // Because this is a live cluster, log files might get archived while we're processing
-        try {
-          LOG.info("Log file found: " + hf.getLocalName() + " in " + rootDir);
-          String logFile = rootDir + "/" + hf.getLocalName();
-          FileStatus fsLog = rfs.getFileStatus(new Path(logFile));
-
-          LOG.info("Checking log file: " + logFile);
-          // Now checking that the hook is up and running
-          // We can't call directly getBlockLocations, it's not available in HFileSystem
-          // We're trying multiple times to be sure, as the order is random
-
-          BlockLocation[] bls = rfs.getFileBlockLocations(fsLog, 0, 1);
-          if (bls.length > 0) {
-            BlockLocation bl = bls[0];
-
-            LOG.info(bl.getHosts().length + " replicas for block 0 in " + logFile + " ");
-            for (int i = 0; i < bl.getHosts().length - 1; i++) {
-              LOG.info(bl.getHosts()[i] + "    " + logFile);
-              Assert.assertNotSame(bl.getHosts()[i], host4);
-            }
-            String last = bl.getHosts()[bl.getHosts().length - 1];
-            LOG.info(last + "    " + logFile);
-            if (host4.equals(last)) {
-              nbTest++;
-              LOG.info(logFile + " is on the new datanode and is ok");
-              if (bl.getHosts().length == 3) {
-                // We can test this case from the file system as well
-                // Checking the underlying file system. Multiple times as the order is random
-                testFromDFS(dfs, logFile, repCount, host4);
-
-                // now from the master
-                testFromDFS(mdfs, logFile, repCount, host4);
-              }
-            }
-          }
-        } catch (FileNotFoundException exception) {
-          LOG.debug("Failed to find log file '" + hf.getLocalName() + "'; it probably was " +
-              "archived out from under us so we'll ignore and retry. If this test hangs " +
-              "indefinitely you should treat this failure as a symptom.", exception);
-        } catch (RemoteException exception) {
-          if (exception.unwrapRemoteException() instanceof FileNotFoundException) {
-            LOG.debug("Failed to find log file '" + hf.getLocalName() + "'; it probably was " +
-                "archived out from under us so we'll ignore and retry. If this test hangs " +
-                "indefinitely you should treat this failure as a symptom.", exception);
-          } else {
-            throw exception;
-          }
-        }
-      }
-    }
-  }
-
-  private void testFromDFS(DistributedFileSystem dfs, String src, int repCount, String localhost)
-      throws Exception {
-    // Multiple times as the order is random
-    for (int i = 0; i < 10; i++) {
-      LocatedBlocks l;
-      // The NN gets the block list asynchronously, so we may need multiple tries to get the list
-      final long max = System.currentTimeMillis() + 10000;
-      boolean done;
-      do {
-        Assert.assertTrue("Can't get enouth replica.", System.currentTimeMillis() < max);
-        l = getNamenode(dfs.getClient()).getBlockLocations(src, 0, 1);
-        Assert.assertNotNull("Can't get block locations for " + src, l);
-        Assert.assertNotNull(l.getLocatedBlocks());
-        Assert.assertTrue(l.getLocatedBlocks().size() > 0);
-
-        done = true;
-        for (int y = 0; y < l.getLocatedBlocks().size() && done; y++) {
-          done = (l.get(y).getLocations().length == repCount);
-        }
-      } while (!done);
-
-      for (int y = 0; y < l.getLocatedBlocks().size() && done; y++) {
-        Assert.assertEquals(localhost, l.get(y).getLocations()[repCount - 1].getHostName());
-      }
-    }
-  }
-
-  private static ClientProtocol getNamenode(DFSClient dfsc) throws Exception {
-    Field nf = DFSClient.class.getDeclaredField("namenode");
-    nf.setAccessible(true);
-    return (ClientProtocol) nf.get(dfsc);
-  }
-
-  /**
-   * Test that the reorder algo works as we expect.
-   */
-  @Test
-  public void testBlockLocation() throws Exception {
-    // We need to start HBase to get  HConstants.HBASE_DIR set in conf
-    htu.startMiniZKCluster();
-    MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 1);
-    conf = hbm.getConfiguration();
-
-
-    // The "/" is mandatory, without it we've got a null pointer exception on the namenode
-    final String fileName = "/helloWorld";
-    Path p = new Path(fileName);
-
-    final int repCount = 3;
-    Assert.assertTrue((short) cluster.getDataNodes().size() >= repCount);
-
-    // Let's write the file
-    FSDataOutputStream fop = dfs.create(p, (short) repCount);
-    final double toWrite = 875.5613;
-    fop.writeDouble(toWrite);
-    fop.close();
-
-    for (int i=0; i<10; i++){
-      // The interceptor is not set in this test, so we get the raw list at this point
-      LocatedBlocks l;
-      final long max = System.currentTimeMillis() + 10000;
-      do {
-        l = getNamenode(dfs.getClient()).getBlockLocations(fileName, 0, 1);
-        Assert.assertNotNull(l.getLocatedBlocks());
-        Assert.assertEquals(1, l.getLocatedBlocks().size());
-        Assert.assertTrue("Expecting " + repCount + " , got " + l.get(0).getLocations().length,
-            System.currentTimeMillis() < max);
-      } while (l.get(0).getLocations().length != repCount);
-
-      // Should be filtered, the name is different => The order won't change
-      Object originalList[] = l.getLocatedBlocks().toArray();
-      HFileSystem.ReorderWALBlocks lrb = new HFileSystem.ReorderWALBlocks();
-      lrb.reorderBlocks(conf, l, fileName);
-      Assert.assertArrayEquals(originalList, l.getLocatedBlocks().toArray());
-
-      // Should be reordered, as we pretend to be a file name with a compliant stuff
-      Assert.assertNotNull(conf.get(HConstants.HBASE_DIR));
-      Assert.assertFalse(conf.get(HConstants.HBASE_DIR).isEmpty());
-      String pseudoLogFile = conf.get(HConstants.HBASE_DIR) + "/" +
-          HConstants.HREGION_LOGDIR_NAME + "/" + host1 + ",6977,6576" + "/mylogfile";
-
-      // Check that it will be possible to extract a ServerName from our construction
-      Assert.assertNotNull("log= " + pseudoLogFile,
-        AbstractFSWALProvider.getServerNameFromWALDirectoryName(dfs.getConf(), pseudoLogFile));
-
-      // And check we're doing the right reorder.
-      lrb.reorderBlocks(conf, l, pseudoLogFile);
-      Assert.assertEquals(host1, l.get(0).getLocations()[2].getHostName());
-
-      // Check again, it should remain the same.
-      lrb.reorderBlocks(conf, l, pseudoLogFile);
-      Assert.assertEquals(host1, l.get(0).getLocations()[2].getHostName());
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b75a8f40/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderBlockLocation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderBlockLocation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderBlockLocation.java
new file mode 100644
index 0000000..5ef5ba4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderBlockLocation.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.fs;
+
+import java.lang.reflect.Field;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+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.HConstants;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Tests for the hdfs fix from HBASE-6435.
+ *
+ * Please don't add new subtest which involves starting / stopping MiniDFSCluster in this class.
+ * When stopping MiniDFSCluster, shutdown hooks would be cleared in hadoop's ShutdownHookManager
+ *   in hadoop 3.
+ * This leads to 'Failed suppression of fs shutdown hook' error in region server.
+ */
+@Category({MiscTests.class, LargeTests.class})
+public class TestBlockReorderBlockLocation {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBlockReorderBlockLocation.class);
+
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private HBaseTestingUtility htu;
+  private DistributedFileSystem dfs;
+  private static final String host1 = "host1";
+  private static final String host2 = "host2";
+  private static final String host3 = "host3";
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    htu = new HBaseTestingUtility();
+    htu.getConfiguration().setInt("dfs.blocksize", 1024);// For the test with multiple blocks
+    htu.getConfiguration().setInt("dfs.replication", 3);
+    htu.startMiniDFSCluster(3,
+        new String[]{"/r1", "/r2", "/r3"}, new String[]{host1, host2, host3});
+
+    conf = htu.getConfiguration();
+    cluster = htu.getDFSCluster();
+    dfs = (DistributedFileSystem) FileSystem.get(conf);
+  }
+
+  @After
+  public void tearDownAfterClass() throws Exception {
+    htu.shutdownMiniCluster();
+  }
+
+
+  private static ClientProtocol getNamenode(DFSClient dfsc) throws Exception {
+    Field nf = DFSClient.class.getDeclaredField("namenode");
+    nf.setAccessible(true);
+    return (ClientProtocol) nf.get(dfsc);
+  }
+
+  /**
+   * Test that the reorder algo works as we expect.
+   */
+  @Test
+  public void testBlockLocation() throws Exception {
+    // We need to start HBase to get  HConstants.HBASE_DIR set in conf
+    htu.startMiniZKCluster();
+    MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 1);
+    conf = hbm.getConfiguration();
+
+
+    // The "/" is mandatory, without it we've got a null pointer exception on the namenode
+    final String fileName = "/helloWorld";
+    Path p = new Path(fileName);
+
+    final int repCount = 3;
+    Assert.assertTrue((short) cluster.getDataNodes().size() >= repCount);
+
+    // Let's write the file
+    FSDataOutputStream fop = dfs.create(p, (short) repCount);
+    final double toWrite = 875.5613;
+    fop.writeDouble(toWrite);
+    fop.close();
+
+    for (int i=0; i<10; i++){
+      // The interceptor is not set in this test, so we get the raw list at this point
+      LocatedBlocks l;
+      final long max = System.currentTimeMillis() + 10000;
+      do {
+        l = getNamenode(dfs.getClient()).getBlockLocations(fileName, 0, 1);
+        Assert.assertNotNull(l.getLocatedBlocks());
+        Assert.assertEquals(1, l.getLocatedBlocks().size());
+        Assert.assertTrue("Expecting " + repCount + " , got " + l.get(0).getLocations().length,
+            System.currentTimeMillis() < max);
+      } while (l.get(0).getLocations().length != repCount);
+
+      // Should be filtered, the name is different => The order won't change
+      Object originalList [] = l.getLocatedBlocks().toArray();
+      HFileSystem.ReorderWALBlocks lrb = new HFileSystem.ReorderWALBlocks();
+      lrb.reorderBlocks(conf, l, fileName);
+      Assert.assertArrayEquals(originalList, l.getLocatedBlocks().toArray());
+
+      // Should be reordered, as we pretend to be a file name with a compliant stuff
+      Assert.assertNotNull(conf.get(HConstants.HBASE_DIR));
+      Assert.assertFalse(conf.get(HConstants.HBASE_DIR).isEmpty());
+      String pseudoLogFile = conf.get(HConstants.HBASE_DIR) + "/" +
+          HConstants.HREGION_LOGDIR_NAME + "/" + host1 + ",6977,6576" + "/mylogfile";
+
+      // Check that it will be possible to extract a ServerName from our construction
+      Assert.assertNotNull("log= " + pseudoLogFile,
+        AbstractFSWALProvider.getServerNameFromWALDirectoryName(dfs.getConf(), pseudoLogFile));
+
+      // And check we're doing the right reorder.
+      lrb.reorderBlocks(conf, l, pseudoLogFile);
+      Assert.assertEquals(host1, l.get(0).getLocations()[2].getHostName());
+
+      // Check again, it should remain the same.
+      lrb.reorderBlocks(conf, l, pseudoLogFile);
+      Assert.assertEquals(host1, l.get(0).getLocations()[2].getHostName());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b75a8f40/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
new file mode 100644
index 0000000..937b5dd
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
@@ -0,0 +1,279 @@
+/**
+ * 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.fs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.ipc.RemoteException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests for the hdfs fix from HBASE-6435.
+ *
+ * Please don't add new subtest which involves starting / stopping MiniDFSCluster in this class.
+ * When stopping MiniDFSCluster, shutdown hooks would be cleared in hadoop's ShutdownHookManager
+ *   in hadoop 3.
+ * This leads to 'Failed suppression of fs shutdown hook' error in region server.
+ */
+@Category({MiscTests.class, LargeTests.class})
+public class TestBlockReorderMultiBlocks {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBlockReorderMultiBlocks.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestBlockReorderMultiBlocks.class);
+
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private HBaseTestingUtility htu;
+  private DistributedFileSystem dfs;
+  private static final String host1 = "host1";
+  private static final String host2 = "host2";
+  private static final String host3 = "host3";
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    htu = new HBaseTestingUtility();
+    htu.getConfiguration().setInt("dfs.blocksize", 1024);// For the test with multiple blocks
+    htu.getConfiguration().setInt("dfs.replication", 3);
+    htu.startMiniDFSCluster(3,
+        new String[]{"/r1", "/r2", "/r3"}, new String[]{host1, host2, host3});
+
+    conf = htu.getConfiguration();
+    cluster = htu.getDFSCluster();
+    dfs = (DistributedFileSystem) FileSystem.get(conf);
+  }
+
+  @After
+  public void tearDownAfterClass() throws Exception {
+    htu.shutdownMiniCluster();
+  }
+
+  /**
+   * Test that the hook works within HBase, including when there are multiple blocks.
+   */
+  @Test()
+  public void testHBaseCluster() throws Exception {
+    byte[] sb = Bytes.toBytes("sb");
+    htu.startMiniZKCluster();
+
+    MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 1);
+    hbm.waitForActiveAndReadyMaster();
+    HRegionServer targetRs = LoadBalancer.isTablesOnMaster(hbm.getConf())? hbm.getMaster():
+      hbm.getRegionServer(0);
+
+    // We want to have a datanode with the same name as the region server, so
+    //  we're going to get the regionservername, and start a new datanode with this name.
+    String host4 = targetRs.getServerName().getHostname();
+    LOG.info("Starting a new datanode with the name=" + host4);
+    cluster.startDataNodes(conf, 1, true, null, new String[]{"/r4"}, new String[]{host4}, null);
+    cluster.waitClusterUp();
+
+    final int repCount = 3;
+
+    // We use the regionserver file system & conf as we expect it to have the hook.
+    conf = targetRs.getConfiguration();
+    HFileSystem rfs = (HFileSystem) targetRs.getFileSystem();
+    Table h = htu.createTable(TableName.valueOf(name.getMethodName()), sb);
+
+    // Now, we have 4 datanodes and a replication count of 3. So we don't know if the datanode
+    // with the same node will be used. We can't really stop an existing datanode, this would
+    // make us fall in nasty hdfs bugs/issues. So we're going to try multiple times.
+
+    // Now we need to find the log file, its locations, and look at it
+
+    String rootDir = new Path(FSUtils.getRootDir(conf) + "/" + HConstants.HREGION_LOGDIR_NAME +
+            "/" + targetRs.getServerName().toString()).toUri().getPath();
+
+    DistributedFileSystem mdfs = (DistributedFileSystem)
+        hbm.getMaster().getMasterFileSystem().getFileSystem();
+
+
+    int nbTest = 0;
+    while (nbTest < 10) {
+      final List<HRegion> regions = targetRs.getRegions(h.getName());
+      final CountDownLatch latch = new CountDownLatch(regions.size());
+      // listen for successful log rolls
+      final WALActionsListener listener = new WALActionsListener() {
+            @Override
+            public void postLogRoll(final Path oldPath, final Path newPath) throws IOException {
+              latch.countDown();
+            }
+          };
+      for (HRegion region : regions) {
+        region.getWAL().registerWALActionsListener(listener);
+      }
+
+      htu.getAdmin().rollWALWriter(targetRs.getServerName());
+
+      // wait
+      try {
+        latch.await();
+      } catch (InterruptedException exception) {
+        LOG.warn("Interrupted while waiting for the wal of '" + targetRs + "' to roll. If later " +
+            "tests fail, it's probably because we should still be waiting.");
+        Thread.currentThread().interrupt();
+      }
+      for (Region region : regions) {
+        ((HRegion)region).getWAL().unregisterWALActionsListener(listener);
+      }
+
+      // We need a sleep as the namenode is informed asynchronously
+      Thread.sleep(100);
+
+      // insert one put to ensure a minimal size
+      Put p = new Put(sb);
+      p.addColumn(sb, sb, sb);
+      h.put(p);
+
+      DirectoryListing dl = dfs.getClient().listPaths(rootDir, HdfsFileStatus.EMPTY_NAME);
+      HdfsFileStatus[] hfs = dl.getPartialListing();
+
+      // As we wrote a put, we should have at least one log file.
+      Assert.assertTrue(hfs.length >= 1);
+      for (HdfsFileStatus hf : hfs) {
+        // Because this is a live cluster, log files might get archived while we're processing
+        try {
+          LOG.info("Log file found: " + hf.getLocalName() + " in " + rootDir);
+          String logFile = rootDir + "/" + hf.getLocalName();
+          FileStatus fsLog = rfs.getFileStatus(new Path(logFile));
+
+          LOG.info("Checking log file: " + logFile);
+          // Now checking that the hook is up and running
+          // We can't call directly getBlockLocations, it's not available in HFileSystem
+          // We're trying multiple times to be sure, as the order is random
+
+          BlockLocation[] bls = rfs.getFileBlockLocations(fsLog, 0, 1);
+          if (bls.length > 0) {
+            BlockLocation bl = bls[0];
+
+            LOG.info(bl.getHosts().length + " replicas for block 0 in " + logFile + " ");
+            for (int i = 0; i < bl.getHosts().length - 1; i++) {
+              LOG.info(bl.getHosts()[i] + "    " + logFile);
+              Assert.assertNotSame(bl.getHosts()[i], host4);
+            }
+            String last = bl.getHosts()[bl.getHosts().length - 1];
+            LOG.info(last + "    " + logFile);
+            if (host4.equals(last)) {
+              nbTest++;
+              LOG.info(logFile + " is on the new datanode and is ok");
+              if (bl.getHosts().length == 3) {
+                // We can test this case from the file system as well
+                // Checking the underlying file system. Multiple times as the order is random
+                testFromDFS(dfs, logFile, repCount, host4);
+
+                // now from the master
+                testFromDFS(mdfs, logFile, repCount, host4);
+              }
+            }
+          }
+        } catch (FileNotFoundException exception) {
+          LOG.debug("Failed to find log file '" + hf.getLocalName() + "'; it probably was " +
+              "archived out from under us so we'll ignore and retry. If this test hangs " +
+              "indefinitely you should treat this failure as a symptom.", exception);
+        } catch (RemoteException exception) {
+          if (exception.unwrapRemoteException() instanceof FileNotFoundException) {
+            LOG.debug("Failed to find log file '" + hf.getLocalName() + "'; it probably was " +
+                "archived out from under us so we'll ignore and retry. If this test hangs " +
+                "indefinitely you should treat this failure as a symptom.", exception);
+          } else {
+            throw exception;
+          }
+        }
+      }
+    }
+  }
+
+  private void testFromDFS(DistributedFileSystem dfs, String src, int repCount, String localhost)
+      throws Exception {
+    // Multiple times as the order is random
+    for (int i = 0; i < 10; i++) {
+      LocatedBlocks l;
+      // The NN gets the block list asynchronously, so we may need multiple tries to get the list
+      final long max = System.currentTimeMillis() + 10000;
+      boolean done;
+      do {
+        Assert.assertTrue("Can't get enouth replica.", System.currentTimeMillis() < max);
+        l = getNamenode(dfs.getClient()).getBlockLocations(src, 0, 1);
+        Assert.assertNotNull("Can't get block locations for " + src, l);
+        Assert.assertNotNull(l.getLocatedBlocks());
+        Assert.assertTrue(l.getLocatedBlocks().size() > 0);
+
+        done = true;
+        for (int y = 0; y < l.getLocatedBlocks().size() && done; y++) {
+          done = (l.get(y).getLocations().length == repCount);
+        }
+      } while (!done);
+
+      for (int y = 0; y < l.getLocatedBlocks().size() && done; y++) {
+        Assert.assertEquals(localhost, l.get(y).getLocations()[repCount - 1].getHostName());
+      }
+    }
+  }
+
+  private static ClientProtocol getNamenode(DFSClient dfsc) throws Exception {
+    Field nf = DFSClient.class.getDeclaredField("namenode");
+    nf.setAccessible(true);
+    return (ClientProtocol) nf.get(dfsc);
+  }
+
+}


[41/50] [abbrv] 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/b54b94ba/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 85b2e85..7b8c43b 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
@@ -54,6 +54,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;
@@ -136,6 +137,8 @@ public class ReplicationSourceManager implements ReplicationListener {
   // For recovered source, the queue id's format is peer_id-servername-*
   private final ConcurrentMap<String, Map<String, SortedSet<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
@@ -172,9 +175,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;
@@ -187,10 +189,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.
@@ -254,8 +257,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 {
@@ -274,13 +280,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
@@ -311,7 +320,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));
   }
@@ -363,6 +375,10 @@ public class ReplicationSourceManager implements ReplicationListener {
         }
       }
     }
+    ReplicationPeerConfig peerConfig = peer.getPeerConfig();
+    if (peerConfig.isSyncReplication()) {
+      syncReplicationPeerMappingManager.add(peer.getId(), peerConfig);
+    }
     src.startup();
     return src;
   }
@@ -442,6 +458,7 @@ public class ReplicationSourceManager implements ReplicationListener {
     // Delete queue from storage and memory
     deleteQueue(src.getQueueId());
     this.walsById.remove(src.getQueueId());
+
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/b54b94ba/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/b54b94ba/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/b54b94ba/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/b54b94ba/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/b54b94ba/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/b54b94ba/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/b54b94ba/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/b54b94ba/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 3d7cdaf..9b03386 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -174,7 +174,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/b54b94ba/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());
   }
 }


[33/50] [abbrv] hbase git commit: HBASE-2004 TestClientClusterStatus is flakey

Posted by zh...@apache.org.
HBASE-2004 TestClientClusterStatus is flakey


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

Branch: refs/heads/HBASE-19064
Commit: 3a3994223c5d634bdd7ef01ef7f31ff860849575
Parents: 3f82098
Author: Michael Stack <st...@apache.org>
Authored: Wed Feb 21 14:52:10 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Feb 21 14:53:31 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/TestClientClusterStatus.java       | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3a399422/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
index bfdae8a..392ff6e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -100,13 +101,14 @@ public class TestClientClusterStatus {
 
   @Test
   public void testNone() throws Exception {
-    ClusterStatus status0
-      = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)));
-    ClusterStatus status1
-      = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class)));
-    Assert.assertEquals(status0, status1);
-    checkPbObjectNotNull(status0);
-    checkPbObjectNotNull(status1);
+    ClusterMetrics status0 = ADMIN.getClusterMetrics(EnumSet.allOf(Option.class));
+    ClusterMetrics status1 = ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class));
+    // Do a rough compare. More specific compares can fail because all regions not deployed yet
+    // or more requests than expected.
+    Assert.assertEquals(status0.getLiveServerMetrics().size(),
+        status1.getLiveServerMetrics().size());
+    checkPbObjectNotNull(new ClusterStatus(status0));
+    checkPbObjectNotNull(new ClusterStatus(status1));
   }
 
   @Test


[30/50] [abbrv] hbase git commit: HBASE-19166 AsyncProtobufLogWriter persists ProtobufLogWriter as class name for backward compatibility

Posted by zh...@apache.org.
HBASE-19166 AsyncProtobufLogWriter persists ProtobufLogWriter as class name for backward compatibility


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

Branch: refs/heads/HBASE-19064
Commit: bf5f034463d357f31e2c7d02c6477c2fcd93d7f4
Parents: 5d994a2
Author: tedyu <yu...@gmail.com>
Authored: Wed Feb 21 13:04:59 2018 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Feb 21 13:04:59 2018 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AsyncProtobufLogWriter.java      | 9 +++++++++
 .../regionserver/wal/SecureAsyncProtobufLogWriter.java      | 8 ++++++++
 2 files changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bf5f0344/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
index 67258ec..abdc24e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
@@ -108,6 +108,15 @@ public class AsyncProtobufLogWriter extends AbstractProtobufLogWriter
     this.channelClass = channelClass;
   }
 
+  /*
+   * @return class name which is recognized by hbase-1.x to avoid ProtobufLogReader throwing error:
+   *   IOException: Got unknown writer class: AsyncProtobufLogWriter
+   */
+  @Override
+  protected String getWriterClassName() {
+    return "ProtobufLogWriter";
+  }
+
   @Override
   public void append(Entry entry) {
     int buffered = output.buffered();

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf5f0344/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java
index 7741b5e..e2d294a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java
@@ -38,6 +38,14 @@ public class SecureAsyncProtobufLogWriter extends AsyncProtobufLogWriter {
     super(eventLoopGroup, channelClass);
   }
 
+  /*
+   * @return class name which is recognized by hbase-1.x to avoid ProtobufLogReader throwing error:
+   *   IOException: Got unknown writer class: SecureAsyncProtobufLogWriter
+   */
+  @Override
+  protected String getWriterClassName() {
+    return "SecureProtobufLogWriter";
+  }
   @Override
   protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
       throws IOException {


[26/50] [abbrv] hbase git commit: HBASE-20037 Race when calling SequenceIdAccounting.resetHighest

Posted by zh...@apache.org.
HBASE-20037 Race when calling SequenceIdAccounting.resetHighest


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

Branch: refs/heads/HBASE-19064
Commit: a27ef55a4074a5b69590c134ce7c0cb09124cdb4
Parents: 92d04d5
Author: zhangduo <zh...@apache.org>
Authored: Wed Feb 21 18:19:01 2018 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Feb 21 10:05:02 2018 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AbstractFSWAL.java   | 37 +++++++++--------
 .../hbase/regionserver/wal/AsyncFSWAL.java      | 42 ++++++++++----------
 .../wal/AsyncProtobufLogWriter.java             |  4 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  6 +--
 .../regionserver/wal/ProtobufLogWriter.java     | 11 +++--
 .../regionserver/wal/SequenceIdAccounting.java  | 40 ++++++++++---------
 6 files changed, 74 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a27ef55a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 14fbe10..ce8dafa 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
@@ -661,9 +661,26 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     }
   }
 
+  protected final void logRollAndSetupWalProps(Path oldPath, Path newPath, long oldFileLen) {
+    int oldNumEntries = this.numEntries.getAndSet(0);
+    String newPathString = newPath != null ? CommonFSUtils.getPath(newPath) : null;
+    if (oldPath != null) {
+      this.walFile2Props.put(oldPath,
+        new WalProps(this.sequenceIdAccounting.resetHighest(), oldFileLen));
+      this.totalLogSize.addAndGet(oldFileLen);
+      LOG.info("Rolled WAL {} with entries={}, filesize={}; new WAL {}",
+        CommonFSUtils.getPath(oldPath), oldNumEntries, StringUtils.byteDesc(oldFileLen),
+        newPathString);
+    } else {
+      LOG.info("New WAL {}", newPathString);
+    }
+  }
+
   /**
+   * <p>
    * Cleans up current writer closing it and then puts in place the passed in
    * <code>nextWriter</code>.
+   * </p>
    * <p>
    * <ul>
    * <li>In the case of creating a new WAL, oldPath will be null.</li>
@@ -672,26 +689,17 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
    * <li>In the case of closing out this FSHLog with no further use newPath and nextWriter will be
    * null.</li>
    * </ul>
+   * </p>
    * @param oldPath may be null
    * @param newPath may be null
    * @param nextWriter may be null
    * @return the passed in <code>newPath</code>
    * @throws IOException if there is a problem flushing or closing the underlying FS
    */
+  @VisibleForTesting
   Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
     try (TraceScope scope = TraceUtil.createTrace("FSHFile.replaceWriter")) {
-      long oldFileLen = doReplaceWriter(oldPath, newPath, nextWriter);
-      int oldNumEntries = this.numEntries.getAndSet(0);
-      final String newPathString = (null == newPath ? null : CommonFSUtils.getPath(newPath));
-      if (oldPath != null) {
-        this.walFile2Props.put(oldPath,
-          new WalProps(this.sequenceIdAccounting.resetHighest(), oldFileLen));
-        this.totalLogSize.addAndGet(oldFileLen);
-        LOG.info("Rolled WAL " + CommonFSUtils.getPath(oldPath) + " with entries=" + oldNumEntries +
-          ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " + newPathString);
-      } else {
-        LOG.info("New WAL " + newPathString);
-      }
+      doReplaceWriter(oldPath, newPath, nextWriter);
       return newPath;
     }
   }
@@ -1021,10 +1029,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   protected abstract W createWriterInstance(Path path)
       throws IOException, CommonFSUtils.StreamLacksCapabilityException;
 
-  /**
-   * @return old wal file size
-   */
-  protected abstract long doReplaceWriter(Path oldPath, Path newPath, W nextWriter)
+  protected abstract void doReplaceWriter(Path oldPath, Path newPath, W nextWriter)
       throws IOException;
 
   protected abstract void doShutdown() throws IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a27ef55a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index 19d89df..d22d1ec 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
@@ -631,11 +631,29 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
+  private long closeWriter() {
+    AsyncWriter oldWriter = this.writer;
+    if (oldWriter != null) {
+      long fileLength = oldWriter.getLength();
+      closeExecutor.execute(() -> {
+        try {
+          oldWriter.close();
+        } catch (IOException e) {
+          LOG.warn("close old writer failed", e);
+        }
+      });
+      return fileLength;
+    } else {
+      return 0L;
+    }
+  }
+
   @Override
-  protected long doReplaceWriter(Path oldPath, Path newPath, AsyncWriter nextWriter)
+  protected void doReplaceWriter(Path oldPath, Path newPath, AsyncWriter nextWriter)
       throws IOException {
     waitForSafePoint();
-    final AsyncWriter oldWriter = this.writer;
+    long oldFileLen = closeWriter();
+    logRollAndSetupWalProps(oldPath, newPath, oldFileLen);
     this.writer = nextWriter;
     if (nextWriter != null && nextWriter instanceof AsyncProtobufLogWriter) {
       this.fsOut = ((AsyncProtobufLogWriter) nextWriter).getOutput();
@@ -654,13 +672,12 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     } finally {
       consumeLock.unlock();
     }
-    return executeClose(closeExecutor, oldWriter);
   }
 
   @Override
   protected void doShutdown() throws IOException {
     waitForSafePoint();
-    executeClose(closeExecutor, writer);
+    closeWriter();
     closeExecutor.shutdown();
     try {
       if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) {
@@ -698,23 +715,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  private static long executeClose(ExecutorService closeExecutor, AsyncWriter writer) {
-    long fileLength;
-    if (writer != null) {
-      fileLength = writer.getLength();
-      closeExecutor.execute(() -> {
-        try {
-          writer.close();
-        } catch (IOException e) {
-          LOG.warn("close old writer failed", e);
-        }
-      });
-    } else {
-      fileLength = 0L;
-    }
-    return fileLength;
-  }
-
   @Override
   protected void doAppend(AsyncWriter writer, FSWALEntry entry) {
     writer.append(entry);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a27ef55a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
index 9688bbd..67258ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
@@ -24,7 +24,6 @@ import java.nio.ByteBuffer;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.function.Consumer;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -34,13 +33,14 @@ import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutputHelper;
 import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a27ef55a/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 2023932..4cc3393 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
@@ -298,7 +298,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   }
 
   @Override
-  protected long doReplaceWriter(Path oldPath, Path newPath, Writer nextWriter) throws IOException {
+  protected void doReplaceWriter(Path oldPath, Path newPath, Writer nextWriter) throws IOException {
     // Ask the ring buffer writer to pause at a safe point. Once we do this, the writer
     // thread will eventually pause. An error hereafter needs to release the writer thread
     // regardless -- hence the finally block below. Note, this method is called from the FSHLog
@@ -320,7 +320,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
       zigzagLatch = this.ringBufferEventHandler.attainSafePoint();
     }
     afterCreatingZigZagLatch();
-    long oldFileLen = 0L;
     try {
       // Wait on the safe point to be achieved. Send in a sync in case nothing has hit the
       // ring buffer between the above notification of writer that we want it to go to
@@ -343,6 +342,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
         LOG.warn(
           "Failed sync-before-close but no outstanding appends; closing WAL" + e.getMessage());
       }
+      long oldFileLen = 0L;
       // It is at the safe point. Swap out writer from under the blocked writer thread.
       // TODO: This is close is inline with critical section. Should happen in background?
       if (this.writer != null) {
@@ -363,6 +363,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           }
         }
       }
+      logRollAndSetupWalProps(oldPath, newPath, oldFileLen);
       this.writer = nextWriter;
       if (nextWriter != null && nextWriter instanceof ProtobufLogWriter) {
         this.hdfs_out = ((ProtobufLogWriter) nextWriter).getStream();
@@ -397,7 +398,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
         }
       }
     }
-    return oldFileLen;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a27ef55a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
index aeb2c19..9d36429 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -16,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
@@ -25,15 +23,16 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
+import org.apache.hadoop.hbase.wal.FSHLogProvider;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
-import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
-import org.apache.hadoop.hbase.wal.FSHLogProvider;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
 
 /**
  * Writer for protobuf-based WAL.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a27ef55a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
index f4cacb2..e14ce0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -29,25 +27,28 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ImmutableByteArray;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ImmutableByteArray;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
+ * <p>
  * Accounting of sequence ids per region and then by column family. So we can our accounting
  * current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance can
  * keep abreast of the state of sequence id persistence. Also call update per append.
+ * </p>
  * <p>
  * For the implementation, we assume that all the {@code encodedRegionName} passed in is gotten by
- * {@link HRegionInfo#getEncodedNameAsBytes()}. So it is safe to use it as a hash key. And for
- * family name, we use {@link ImmutableByteArray} as key. This is because hash based map is much
- * faster than RBTree or CSLM and here we are on the critical write path. See HBASE-16278 for more
- * details.
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#getEncodedNameAsBytes()}. So it is safe to use
+ * it as a hash key. And for family name, we use {@link ImmutableByteArray} as key. This is because
+ * hash based map is much faster than RBTree or CSLM and here we are on the critical write path. See
+ * HBASE-16278 for more details.
+ * </p>
  */
 @InterfaceAudience.Private
 class SequenceIdAccounting {
@@ -93,14 +94,17 @@ class SequenceIdAccounting {
    */
   private final Map<byte[], Map<ImmutableByteArray, Long>> flushingSequenceIds = new HashMap<>();
 
- /**
-  * Map of region encoded names to the latest/highest region sequence id.  Updated on each
-  * call to append.
-  * <p>
-  * This map uses byte[] as the key, and uses reference equality. It works in our use case as we
-  * use {@link HRegionInfo#getEncodedNameAsBytes()} as keys. For a given region, it always returns
-  * the same array.
-  */
+  /**
+   * <p>
+   * Map of region encoded names to the latest/highest region sequence id. Updated on each call to
+   * append.
+   * </p>
+   * <p>
+   * This map uses byte[] as the key, and uses reference equality. It works in our use case as we
+   * use {@link org.apache.hadoop.hbase.client.RegionInfo#getEncodedNameAsBytes()} as keys. For a
+   * given region, it always returns the same array.
+   * </p>
+   */
   private Map<byte[], Long> highestSequenceIds = new HashMap<>();
 
   /**


[36/50] [abbrv] hbase git commit: HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory

Posted by zh...@apache.org.
HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory


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

Branch: refs/heads/HBASE-19064
Commit: a0900857c7d58e03a39794e96224bf6213307ce7
Parents: 13223c2
Author: Esteban Gutierrez <es...@apache.org>
Authored: Wed Feb 21 15:28:50 2018 -0800
Committer: huaxiangsun <hu...@gmail.com>
Committed: Wed Feb 21 15:56:36 2018 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/HRegionFileSystem.java   | 29 ++++++++++----------
 1 file changed, 15 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a0900857/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 6ad3f1a..3c2d392 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -969,28 +969,29 @@ public class HRegionFileSystem {
   public static HRegionFileSystem createRegionOnFileSystem(final Configuration conf,
       final FileSystem fs, final Path tableDir, final RegionInfo regionInfo) throws IOException {
     HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo);
-    Path regionDir = regionFs.getRegionDir();
 
-    if (fs.exists(regionDir)) {
-      LOG.warn("Trying to create a region that already exists on disk: " + regionDir);
-      throw new IOException("The specified region already exists on disk: " + regionDir);
-    }
+    // We only create a .regioninfo and the region directory if this is the default region replica
+    if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
+      Path regionDir = regionFs.getRegionDir();
+      if (fs.exists(regionDir)) {
+        LOG.warn("Trying to create a region that already exists on disk: " + regionDir);
+        throw new IOException("The specified region already exists on disk: " + regionDir);
+      }
 
-    // Create the region directory
-    if (!createDirOnFileSystem(fs, conf, regionDir)) {
-      LOG.warn("Unable to create the region directory: " + regionDir);
-      throw new IOException("Unable to create region directory: " + regionDir);
-    }
+      // Create the region directory
+      if (!createDirOnFileSystem(fs, conf, regionDir)) {
+        LOG.warn("Unable to create the region directory: " + regionDir);
+        throw new IOException("Unable to create region directory: " + regionDir);
+      }
 
-    // Write HRI to a file in case we need to recover hbase:meta
-    // Only primary replicas should write region info
-    if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
+      // Write HRI to a file in case we need to recover hbase:meta
       regionFs.writeRegionInfoOnFilesystem(false);
+      return regionFs;
     } else {
       if (LOG.isDebugEnabled())
         LOG.debug("Skipping creation of .regioninfo file for " + regionInfo);
     }
-    return regionFs;
+    return null;
   }
 
   /**


[32/50] [abbrv] hbase git commit: HBASE-20042 TestRegionServerAbort flakey

Posted by zh...@apache.org.
HBASE-20042 TestRegionServerAbort flakey


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

Branch: refs/heads/HBASE-19064
Commit: 3f82098d4b7ae595aa6702d3fb7cc2fac682691b
Parents: 173a5bf
Author: Michael Stack <st...@apache.org>
Authored: Wed Feb 21 13:24:32 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Feb 21 13:26:10 2018 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/TestRegionServerAbort.java       | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3f82098d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
index fdb6887..0c778fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Optional;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,6 +53,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -146,6 +148,16 @@ public class TestRegionServerAbort {
     // should have triggered an abort due to FileNotFoundException
 
     // verify that the regionserver is stopped
+    List<HRegion> regions = null;
+    do {
+      regions = cluster.findRegionsForTable(tableName);
+      if (regions != null && regions.size() > 0) {
+        break;
+      }
+      LOG.warn("Waiting on regions for {} to online");
+      Threads.sleep(100);
+    } while(true);
+
     HRegion firstRegion = cluster.findRegionsForTable(tableName).get(0);
     assertNotNull(firstRegion);
     assertNotNull(firstRegion.getRegionServerServices());


[47/50] [abbrv] 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/3d720648
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3d720648
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3d720648

Branch: refs/heads/HBASE-19064
Commit: 3d7206481679519986433adf1d41691d8a0ae08d
Parents: fb26883
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Feb 6 16:00:59 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 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/3d720648/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index 4c10c46..69565a7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -25,6 +25,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -217,6 +219,13 @@ public class ReplicationPeerConfig {
     return this.remoteWALDir;
   }
 
+  /**
+   * Use remote wal dir to decide whether a peer is sync replication peer
+   */
+  public boolean isSyncReplication() {
+    return !StringUtils.isBlank(this.remoteWALDir);
+  }
+
   public static ReplicationPeerConfigBuilder newBuilder() {
     return new ReplicationPeerConfigBuilderImpl();
   }

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

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


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

Branch: refs/heads/HBASE-19064
Commit: 8187688316da60a506189b33ee754256184b46ff
Parents: f044dc4
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Jan 13 18:55:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/81876883/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index a234a9b..642149b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -315,6 +315,9 @@ public final class ReplicationPeerConfigUtil {
         excludeNamespacesList.stream().map(ByteString::toStringUtf8).collect(Collectors.toSet()));
     }
 
+    if (peer.hasRemoteWALDir()) {
+      builder.setRemoteWALDir(peer.getRemoteWALDir());
+    }
     return builder.build();
   }
 
@@ -371,6 +374,9 @@ public final class ReplicationPeerConfigUtil {
       }
     }
 
+    if (peerConfig.getRemoteWALDir() != null) {
+      builder.setRemoteWALDir(peerConfig.getRemoteWALDir());
+    }
     return builder.build();
   }
 

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/81876883/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index 19fc7f4..d715e2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -146,6 +146,21 @@ public class ReplicationPeerManager {
           oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
           " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
     }
+
+    if (!isStringEquals(peerConfig.getRemoteWALDir(), oldPeerConfig.getRemoteWALDir())) {
+      throw new DoNotRetryIOException(
+          "Changing the remote wal dir on an existing peer is not allowed. Existing remote wal " +
+              "dir '" + oldPeerConfig.getRemoteWALDir() + "' for peer " + peerId +
+              " does not match new remote wal dir '" + peerConfig.getRemoteWALDir() + "'");
+    }
+
+    if (oldPeerConfig.getRemoteWALDir() != null) {
+      if (!ReplicationUtils.isKeyConfigEqual(oldPeerConfig, peerConfig)) {
+        throw new DoNotRetryIOException(
+            "Changing the replicated namespace/table config on a synchronous replication "
+                + "peer(peerId: " + peerId + ") is not allowed.");
+      }
+    }
   }
 
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)

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

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

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


[23/50] [abbrv] hbase git commit: HBASE-20032 Receving multiple warnings for missing reporting.plugins.plugin.version

Posted by zh...@apache.org.
HBASE-20032 Receving multiple warnings for missing reporting.plugins.plugin.version


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

Branch: refs/heads/HBASE-19064
Commit: 0068b95c854d13e359b619aaa9879b5cbb993729
Parents: b75a8f4
Author: Artem Ervits <ge...@gmail.com>
Authored: Tue Feb 20 15:15:34 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Feb 20 15:16:22 2018 -0800

----------------------------------------------------------------------
 pom.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0068b95c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d33478a..951747e 100755
--- a/pom.xml
+++ b/pom.xml
@@ -3491,6 +3491,7 @@
     <plugins>
       <plugin>
         <artifactId>maven-project-info-reports-plugin</artifactId>
+        <version>2.9</version>
         <reportSets>
           <reportSet>
             <reports>
@@ -3520,6 +3521,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-javadoc-plugin</artifactId>
+        <version>3.0.0</version>
         <reportSets>
           <!-- Dev API -->
           <reportSet>
@@ -3721,6 +3723,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven.checkstyle.version}</version>
         <configuration>
           <excludes>target/**</excludes>
         </configuration>
@@ -3729,6 +3732,7 @@
       <plugin>
         <groupId>org.scala-tools</groupId>
         <artifactId>maven-scala-plugin</artifactId>
+        <version>2.15.2</version>
       </plugin>
     </plugins>
   </reporting>


[04/50] [abbrv] hbase git commit: HBASE-19988 Fixed chatty log from HRegion#lockRowsAndBuildMiniBatch() when interrupted while waiting on a row lock

Posted by zh...@apache.org.
HBASE-19988 Fixed chatty log from HRegion#lockRowsAndBuildMiniBatch() when interrupted while waiting on a row lock


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

Branch: refs/heads/HBASE-19064
Commit: 9497eac4e01a12acfa6660b5b16e73a45bcf5707
Parents: 40f8d20
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Mon Feb 12 17:49:47 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Feb 16 07:01:56 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/regionserver/HRegion.java     | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9497eac4/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 0fffe88..6170888 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
@@ -3166,8 +3166,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         try {
           // if atomic then get exclusive lock, else shared lock
           rowLock = region.getRowLockInternal(mutation.getRow(), !isAtomic(), prevRowLock);
-        } catch (TimeoutIOException e) {
-          // We will retry when other exceptions, but we should stop if we timeout .
+        } catch (TimeoutIOException|InterruptedIOException e) {
+          // NOTE: We will retry when other exceptions, but we should stop if we receive
+          // TimeoutIOException or InterruptedIOException as operation has timed out or
+          // interrupted respectively.
           throw e;
         } catch (IOException ioe) {
           LOG.warn("Failed getting lock, row=" + Bytes.toStringBinary(mutation.getRow()), ioe);


[18/50] [abbrv] hbase git commit: HBASE-20023 CompactionTool command line examples are incorrect

Posted by zh...@apache.org.
HBASE-20023 CompactionTool command line examples are incorrect

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

Branch: refs/heads/HBASE-19064
Commit: 2b1969898f76e227ba3c89c7e5e8127f11236496
Parents: 69d2bec
Author: Artem Ervits <ge...@gmail.com>
Authored: Tue Feb 20 10:15:27 2018 -0500
Committer: tedyu <yu...@gmail.com>
Committed: Tue Feb 20 08:13:12 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/2b196989/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
index 00ad54b..78db6fc 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
@@ -460,10 +460,10 @@ public class CompactionTool extends Configured implements Tool {
     System.err.println();
     System.err.println("Examples:");
     System.err.println(" To compact the full 'TestTable' using MapReduce:");
-    System.err.println(" $ hbase " + this.getClass().getName() + " -mapred hdfs:///hbase/data/default/TestTable");
+    System.err.println(" $ hbase " + this.getClass().getName() + " -mapred hdfs://hbase/data/default/TestTable");
     System.err.println();
     System.err.println(" To compact column family 'x' of the table 'TestTable' region 'abc':");
-    System.err.println(" $ hbase " + this.getClass().getName() + " hdfs:///hbase/data/default/TestTable/abc/x");
+    System.err.println(" $ hbase " + this.getClass().getName() + " hdfs://hbase/data/default/TestTable/abc/x");
   }
 
   public static void main(String[] args) throws Exception {


[21/50] [abbrv] hbase git commit: HBASE-19400 Add missing security checks in MasterRpcServices

Posted by zh...@apache.org.
HBASE-19400 Add missing security checks in MasterRpcServices

- Added ADMIN permission check for following rpc calls:
  normalize, setNormalizerRunning, runCatalogScan, enableCatalogJanitor, runCleanerChore,
  setCleanerChoreRunning, execMasterService, execProcedure, execProcedureWithRet
- Moved authorizationEnabled check to start of AccessChecker's functions. Currently, and IDK why,
  we call authManager.authorize() first and then discard its result if authorizationEnabled is false. Weird.


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

Branch: refs/heads/HBASE-19064
Commit: 6b8439366134b6749895b6019bc068d169865b48
Parents: 148356a
Author: Apekshit Sharma <ap...@apache.org>
Authored: Tue Feb 13 12:33:43 2018 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Tue Feb 20 13:37:20 2018 -0700

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     |   3 +-
 .../hadoop/hbase/master/MasterRpcServices.java  | 120 ++++-----
 .../hbase/regionserver/HRegionServer.java       |   2 +-
 .../hbase/regionserver/RSRpcServices.java       |  25 +-
 .../hbase/security/access/AccessChecker.java    | 102 ++++----
 .../hbase/security/access/AccessController.java |   5 +-
 .../security/access/TestAccessController.java   |  13 +-
 .../access/TestAdminOnlyOperations.java         | 244 +++++++++++++++++++
 8 files changed, 389 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 435d138..7fec32d 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -79,7 +79,6 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.security.access.TableAuthManager;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -127,7 +126,7 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
 
   @Override
   public void stop(CoprocessorEnvironment env) {
-    TableAuthManager.release(accessChecker.getAuthManager());
+    accessChecker.stop();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 377a9c6..b4f0faf 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
@@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
+import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
@@ -349,6 +350,24 @@ public class MasterRpcServices extends RSRpcServices
     return new MasterAnnotationReadingPriorityFunction(this);
   }
 
+  /**
+   * Checks for the following pre-checks in order:
+   * <ol>
+   *   <li>Master is initialized</li>
+   *   <li>Rpc caller has admin permissions</li>
+   * </ol>
+   * @param requestName name of rpc request. Used in reporting failures to provide context.
+   * @throws ServiceException If any of the above listed pre-check fails.
+   */
+  private void rpcPreCheck(String requestName) throws ServiceException {
+    try {
+      master.checkInitialized();
+      requirePermission(requestName, Permission.Action.ADMIN);
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
   enum BalanceSwitchMode {
     SYNC,
     ASYNC
@@ -393,25 +412,6 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   /**
-   * Sets normalizer on/off flag in ZK.
-   */
-  public boolean normalizerSwitch(boolean on) {
-    boolean oldValue = master.getRegionNormalizerTracker().isNormalizerOn();
-    boolean newValue = on;
-    try {
-      try {
-        master.getRegionNormalizerTracker().setNormalizerOn(newValue);
-      } catch (KeeperException ke) {
-        throw new IOException(ke);
-      }
-      LOG.info(master.getClientIdAuditPrefix() + " set normalizerSwitch=" + newValue);
-    } catch (IOException ioe) {
-      LOG.warn("Error flipping normalizer switch", ioe);
-    }
-    return oldValue;
-  }
-
-  /**
    * @return list of blocking services and their security info classes that this server supports
    */
   @Override
@@ -696,24 +696,16 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
       EnableCatalogJanitorRequest req) throws ServiceException {
-    try {
-      master.checkInitialized();
-    } catch (IOException ioe) {
-      throw new ServiceException(ioe);
-    }
+    rpcPreCheck("enableCatalogJanitor");
     return EnableCatalogJanitorResponse.newBuilder().setPrevValue(
       master.catalogJanitorChore.setEnabled(req.getEnable())).build();
   }
 
   @Override
-  public SetCleanerChoreRunningResponse setCleanerChoreRunning(RpcController c,
-                                                               SetCleanerChoreRunningRequest req)
-    throws ServiceException {
-    try {
-      master.checkInitialized();
-    } catch (IOException ioe) {
-      throw new ServiceException(ioe);
-    }
+  public SetCleanerChoreRunningResponse setCleanerChoreRunning(
+    RpcController c, SetCleanerChoreRunningRequest req) throws ServiceException {
+    rpcPreCheck("setCleanerChoreRunning");
+
     boolean prevValue =
       master.getLogCleaner().getEnabled() && master.getHFileCleaner().getEnabled();
     master.getLogCleaner().setEnabled(req.getOn());
@@ -793,10 +785,9 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
       final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {
+    rpcPreCheck("execMasterService");
     try {
-      master.checkInitialized();
       ServerRpcController execController = new ServerRpcController();
-
       ClientProtos.CoprocessorServiceCall call = request.getCall();
       String serviceName = call.getServiceName();
       String methodName = call.getMethodName();
@@ -816,14 +807,11 @@ public class MasterRpcServices extends RSRpcServices
       final com.google.protobuf.Message.Builder responseBuilder =
           service.getResponsePrototype(methodDesc).newBuilderForType();
       service.callMethod(methodDesc, execController, execRequest,
-          new com.google.protobuf.RpcCallback<com.google.protobuf.Message>() {
-        @Override
-        public void run(com.google.protobuf.Message message) {
+        (message) -> {
           if (message != null) {
             responseBuilder.mergeFrom(message);
           }
-        }
-      });
+        });
       com.google.protobuf.Message execResult = responseBuilder.build();
       if (execController.getFailedOn() != null) {
         throw execController.getFailedOn();
@@ -841,8 +829,8 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public ExecProcedureResponse execProcedure(RpcController controller,
       ExecProcedureRequest request) throws ServiceException {
+    rpcPreCheck("execProcedure");
     try {
-      master.checkInitialized();
       ProcedureDescription desc = request.getProcedure();
       MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
         desc.getSignature());
@@ -850,12 +838,8 @@ public class MasterRpcServices extends RSRpcServices
         throw new ServiceException(new DoNotRetryIOException("The procedure is not registered: "
           + desc.getSignature()));
       }
-
-      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "
-        + desc.getSignature());
-
+      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
       mpm.execProcedure(desc);
-
       // send back the max amount of time the client should wait for the procedure
       // to complete
       long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME;
@@ -876,21 +860,16 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public ExecProcedureResponse execProcedureWithRet(RpcController controller,
       ExecProcedureRequest request) throws ServiceException {
+    rpcPreCheck("execProcedureWithRet");
     try {
-      master.checkInitialized();
       ProcedureDescription desc = request.getProcedure();
-      MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
-        desc.getSignature());
+      MasterProcedureManager mpm =
+        master.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature());
       if (mpm == null) {
-        throw new ServiceException("The procedure is not registered: "
-          + desc.getSignature());
+        throw new ServiceException("The procedure is not registered: " + desc.getSignature());
       }
-
-      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "
-        + desc.getSignature());
-
+      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
       byte[] data = mpm.execProcedureWithRet(desc);
-
       ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder();
       // set return data if available
       if (data != null) {
@@ -1189,8 +1168,7 @@ public class MasterRpcServices extends RSRpcServices
 
   @Override
   public AbortProcedureResponse abortProcedure(
-      RpcController rpcController,
-      AbortProcedureRequest request) throws ServiceException {
+      RpcController rpcController, AbortProcedureRequest request) throws ServiceException {
     try {
       AbortProcedureResponse.Builder response = AbortProcedureResponse.newBuilder();
       boolean abortResult =
@@ -1419,8 +1397,8 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public RunCatalogScanResponse runCatalogScan(RpcController c,
       RunCatalogScanRequest req) throws ServiceException {
+    rpcPreCheck("runCatalogScan");
     try {
-      master.checkInitialized();
       return ResponseConverter.buildRunCatalogScanResponse(master.catalogJanitorChore.scan());
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
@@ -1430,13 +1408,9 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public RunCleanerChoreResponse runCleanerChore(RpcController c, RunCleanerChoreRequest req)
     throws ServiceException {
-    try {
-      master.checkInitialized();
-      boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
-      return ResponseConverter.buildRunCleanerChoreResponse(result);
-    } catch (IOException ioe) {
-      throw new ServiceException(ioe);
-    }
+    rpcPreCheck("runCleanerChore");
+    boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
+    return ResponseConverter.buildRunCleanerChoreResponse(result);
   }
 
   @Override
@@ -1769,6 +1743,7 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public NormalizeResponse normalize(RpcController controller,
       NormalizeRequest request) throws ServiceException {
+    rpcPreCheck("normalize");
     try {
       return NormalizeResponse.newBuilder().setNormalizerRan(master.normalizeRegions()).build();
     } catch (IOException ex) {
@@ -1779,13 +1754,18 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller,
       SetNormalizerRunningRequest request) throws ServiceException {
+    rpcPreCheck("setNormalizerRunning");
+
+    // Sets normalizer on/off flag in ZK.
+    boolean prevValue = master.getRegionNormalizerTracker().isNormalizerOn();
+    boolean newValue = request.getOn();
     try {
-      master.checkInitialized();
-      boolean prevValue = normalizerSwitch(request.getOn());
-      return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
-    } catch (IOException ioe) {
-      throw new ServiceException(ioe);
+      master.getRegionNormalizerTracker().setNormalizerOn(newValue);
+    } catch (KeeperException ke) {
+      LOG.warn("Error flipping normalizer switch", ke);
     }
+    LOG.info("{} set normalizerSwitch={}", master.getClientIdAuditPrefix(), newValue);
+    return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/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 e2aac03..49b7b80 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
@@ -630,7 +630,7 @@ public class HRegionServer extends HasThread implements
         masterAddressTracker = null;
         clusterStatusTracker = null;
       }
-      this.rpcServices.start();
+      this.rpcServices.start(zooKeeper);
       // This violates 'no starting stuff in Constructor' but Master depends on the below chore
       // and executor being created and takes a different startup route. Lots of overlap between HRS
       // and M (An M IS A HRS now). Need to refactor so less duplication between M and its super

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/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 5b4e3b8..05bbb47 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
@@ -118,6 +118,8 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessChecker;
+import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DNS;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -128,6 +130,7 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -321,6 +324,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
   final AtomicBoolean clearCompactionQueues = new AtomicBoolean(false);
 
+  // We want to vet all accesses at the point of entry itself; limiting scope of access checker
+  // instance to only this class to prevent its use from spreading deeper into implementation.
+  // Initialized in start() since AccessChecker needs ZKWatcher which is created by HRegionServer
+  // after RSRpcServices constructor and before start() is called.
+  // Initialized only if authorization is enabled, else remains null.
+  private AccessChecker accessChecker;
+
   /**
    * Services launched in RSRpcServices. By default they are on but you can use the below
    * booleans to selectively enable/disable either Admin or Client Service (Rare is the case
@@ -1240,6 +1250,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return new AnnotationReadingPriorityFunction(this);
   }
 
+  protected void requirePermission(String request, Permission.Action perm) throws IOException {
+    if (accessChecker != null) {
+      accessChecker.requirePermission(RpcServer.getRequestUser().orElse(null), request, perm);
+    }
+  }
+
+
   public static String getHostname(Configuration conf, boolean isMaster)
       throws UnknownHostException {
     String hostname = conf.get(isMaster? HRegionServer.MASTER_HOSTNAME_KEY :
@@ -1404,12 +1421,18 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return regionServer.getRegionServerSpaceQuotaManager();
   }
 
-  void start() {
+  void start(ZKWatcher zkWatcher) {
+    if (AccessChecker.isAuthorizationSupported(getConfiguration())) {
+      accessChecker = new AccessChecker(getConfiguration(), zkWatcher);
+    }
     this.scannerIdGenerator = new ScannerIdGenerator(this.regionServer.serverName);
     rpcServer.start();
   }
 
   void stop() {
+    if (accessChecker != null) {
+      accessChecker.stop();
+    }
     closeAllScanners();
     rpcServer.stop();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
index d88e522..7feeaa0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
@@ -40,6 +40,9 @@ import org.slf4j.LoggerFactory;
 public final class AccessChecker {
   private static final Logger AUDITLOG =
       LoggerFactory.getLogger("SecurityLogger." + AccessChecker.class.getName());
+  // TODO: we should move to a design where we don't even instantiate an AccessChecker if
+  // authorization is not enabled (like in RSRpcServices), instead of always instantiating one and
+  // calling requireXXX() only to do nothing (since authorizationEnabled will be false).
   private TableAuthManager authManager;
   /**
    * if we are active, usually false, only true if "hbase.security.authorization"
@@ -59,8 +62,6 @@ public final class AccessChecker {
    */
   public AccessChecker(final Configuration conf, final ZKWatcher zkw)
       throws RuntimeException {
-    // If zk is null or IOException while obtaining auth manager,
-    // throw RuntimeException so that the coprocessor is unloaded.
     if (zkw != null) {
       try {
         this.authManager = TableAuthManager.getOrCreate(zkw, conf);
@@ -73,6 +74,13 @@ public final class AccessChecker {
     authorizationEnabled = isAuthorizationSupported(conf);
   }
 
+  /**
+   * Releases {@link TableAuthManager}'s reference.
+   */
+  public void stop() {
+    TableAuthManager.release(authManager);
+  }
+
   public TableAuthManager getAuthManager() {
     return authManager;
   }
@@ -87,6 +95,9 @@ public final class AccessChecker {
    */
   public void requireAccess(User user, String request, TableName tableName,
       Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
@@ -101,7 +112,7 @@ public final class AccessChecker {
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -130,22 +141,21 @@ public final class AccessChecker {
   public void requireGlobalPermission(User user, String request,
       Action perm, TableName tableName,
       Map<byte[], ? extends Collection<byte[]>> familyMap)throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result;
     if (authManager.authorize(user, perm)) {
-      result = AuthResult.allow(request, "Global check allowed",
-          user, perm, tableName, familyMap);
+      result = AuthResult.allow(request, "Global check allowed", user, perm, tableName, familyMap);
       result.getParams().setTableName(tableName).setFamilies(familyMap);
       logResult(result);
     } else {
-      result = AuthResult.deny(request, "Global check failed",
-          user, perm, tableName, familyMap);
+      result = AuthResult.deny(request, "Global check failed", user, perm, tableName, familyMap);
       result.getParams().setTableName(tableName).setFamilies(familyMap);
       logResult(result);
-      if (authorizationEnabled) {
-        throw new AccessDeniedException(
-            "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
-                + "' (global, action=" + perm.toString() + ")");
-      }
+      throw new AccessDeniedException(
+          "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
+              + "' (global, action=" + perm.toString() + ")");
     }
   }
 
@@ -159,22 +169,21 @@ public final class AccessChecker {
    */
   public void requireGlobalPermission(User user, String request, Action perm,
       String namespace) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult authResult;
     if (authManager.authorize(user, perm)) {
-      authResult = AuthResult.allow(request, "Global check allowed",
-          user, perm, null);
+      authResult = AuthResult.allow(request, "Global check allowed", user, perm, null);
       authResult.getParams().setNamespace(namespace);
       logResult(authResult);
     } else {
-      authResult = AuthResult.deny(request, "Global check failed",
-          user, perm, null);
+      authResult = AuthResult.deny(request, "Global check failed", user, perm, null);
       authResult.getParams().setNamespace(namespace);
       logResult(authResult);
-      if (authorizationEnabled) {
-        throw new AccessDeniedException(
-            "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
-                + "' (global, action=" + perm.toString() + ")");
-      }
+      throw new AccessDeniedException(
+          "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
+              + "' (global, action=" + perm.toString() + ")");
     }
   }
 
@@ -186,22 +195,23 @@ public final class AccessChecker {
    */
   public void requireNamespacePermission(User user, String request, String namespace,
       Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
       if (authManager.authorize(user, namespace, permission)) {
         result =
-            AuthResult.allow(request, "Namespace permission granted",
-                user, permission, namespace);
+            AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
         break;
       } else {
         // rest of the world
-        result = AuthResult.deny(request, "Insufficient permissions",
-            user, permission, namespace);
+        result = AuthResult.deny(request, "Insufficient permissions", user, permission, namespace);
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -215,24 +225,25 @@ public final class AccessChecker {
   public void requireNamespacePermission(User user, String request, String namespace,
       TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap,
       Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
       if (authManager.authorize(user, namespace, permission)) {
         result =
-            AuthResult.allow(request, "Namespace permission granted",
-                user, permission, namespace);
+            AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
         result.getParams().setTableName(tableName).setFamilies(familyMap);
         break;
       } else {
         // rest of the world
-        result = AuthResult.deny(request, "Insufficient permissions",
-            user, permission, namespace);
+        result = AuthResult.deny(request, "Insufficient permissions", user, permission, namespace);
         result.getParams().setTableName(tableName).setFamilies(familyMap);
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -249,23 +260,24 @@ public final class AccessChecker {
    */
   public void requirePermission(User user, String request, TableName tableName, byte[] family,
       byte[] qualifier, Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
       if (authManager.authorize(user, tableName, family, qualifier, permission)) {
         result = AuthResult.allow(request, "Table permission granted",
-            user, permission, tableName, family,
-                qualifier);
+            user, permission, tableName, family, qualifier);
         break;
       } else {
         // rest of the world
         result = AuthResult.deny(request, "Insufficient permissions",
-                user, permission, tableName, family,
-                qualifier);
+                user, permission, tableName, family, qualifier);
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -283,6 +295,9 @@ public final class AccessChecker {
   public void requireTablePermission(User user, String request,
       TableName tableName,byte[] family, byte[] qualifier,
       Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
@@ -299,7 +314,7 @@ public final class AccessChecker {
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -321,12 +336,13 @@ public final class AccessChecker {
 
   public static void logResult(AuthResult result) {
     if (AUDITLOG.isTraceEnabled()) {
-      AUDITLOG.trace("Access " + (result.isAllowed() ? "allowed" : "denied") + " for user " + (
-          result.getUser() != null ?
-              result.getUser().getShortName() :
-              "UNKNOWN") + "; reason: " + result.getReason() + "; remote address: "
-          + RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("")
-          + "; request: " + result.getRequest() + "; context: " + result.toContextString());
+      AUDITLOG.trace(
+        "Access {} for user {}; reason: {}; remote address: {}; request: {}; context: {}",
+        (result.isAllowed() ? "allowed" : "denied"),
+        (result.getUser() != null ? result.getUser().getShortName() : "UNKNOWN"),
+        result.getReason(),
+        RpcServer.getRemoteAddress().map(InetAddress::toString).orElse(""),
+        result.getRequest(), result.toContextString());
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 1fbf01d..5fef8ab 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
@@ -795,13 +795,14 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
 
     // set the user-provider.
     this.userProvider = UserProvider.instantiate(env.getConfiguration());
+    // Throws RuntimeException if fails to load TableAuthManager so that coprocessor is unloaded.
     accessChecker = new AccessChecker(env.getConfiguration(), zk);
     tableAcls = new MapMaker().weakValues().makeMap();
   }
 
   @Override
   public void stop(CoprocessorEnvironment env) {
-    TableAuthManager.release(getAuthManager());
+    accessChecker.stop();
   }
 
   /*********************************** Observer/Service Getters ***********************************/
@@ -2456,7 +2457,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       throws IOException {
     requirePermission(ctx, "replicateLogEntries", Action.WRITE);
   }
-  
+
   @Override
   public void  preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
           throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/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 d5a0063..2e9be30 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
@@ -223,14 +223,15 @@ public class TestAccessController extends SecureTestUtil {
     conf.setBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY, true);
 
     TEST_UTIL.startMiniCluster();
-    MasterCoprocessorHost cpHost =
+    MasterCoprocessorHost masterCpHost =
       TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
-    cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
-    ACCESS_CONTROLLER = cpHost.findCoprocessor(AccessController.class);
-    CP_ENV = cpHost.createEnvironment(ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
-    RegionServerCoprocessorHost rsHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
+    masterCpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
+    ACCESS_CONTROLLER = masterCpHost.findCoprocessor(AccessController.class);
+    CP_ENV = masterCpHost.createEnvironment(
+        ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
+    RegionServerCoprocessorHost rsCpHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
       .getRegionServerCoprocessorHost();
-    RSCP_ENV = rsHost.createEnvironment(ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
+    RSCP_ENV = rsCpHost.createEnvironment(ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
 
     // Wait for the ACL table to become available
     TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java
new file mode 100644
index 0000000..d4b0650
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java
@@ -0,0 +1,244 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.security.access;
+
+import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
+import java.util.HashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * This class tests operations in MasterRpcServices which require ADMIN access.
+ * It doesn't test all operations which require ADMIN access, only those which get vetted within
+ * MasterRpcServices at the point of entry itself (unlike old approach of using
+ * hooks in AccessController).
+ *
+ * Sidenote:
+ * There is one big difference between how security tests for AccessController hooks work, and how
+ * the tests in this class for security in MasterRpcServices work.
+ * The difference arises because of the way AC & MasterRpcServices get the user.
+ *
+ * In AccessController, it first checks if there is an active rpc user in ObserverContext. If not,
+ * it uses UserProvider for current user. This *might* make sense in the context of coprocessors,
+ * because they can be called outside the context of RPCs.
+ * But in the context of MasterRpcServices, only one way makes sense - RPCServer.getRequestUser().
+ *
+ * In AC tests, when we do FooUser.runAs on AccessController instance directly, it bypasses
+ * the rpc framework completely, but works because UserProvider provides the correct user, i.e.
+ * FooUser in this case.
+ *
+ * But this doesn't work for the tests here, so we go around by doing complete RPCs.
+ */
+@Category({SecurityTests.class, MediumTests.class})
+public class TestAdminOnlyOperations {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestAdminOnlyOperations.class);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
+
+  // user granted with all global permission
+  private static User USER_ADMIN;
+  // user without admin permissions
+  private static User USER_NON_ADMIN;
+
+  private static final String GROUP_ADMIN = "admin_group";
+  private static User USER_GROUP_ADMIN;
+
+  // Dummy service to test execService calls. Needs to be public so can be loaded as Coprocessor.
+  public static class DummyCpService implements MasterCoprocessor {
+    public DummyCpService() {}
+
+    @Override
+    public Iterable<Service> getServices() {
+      return Collections.singleton(mock(TestRpcServiceProtos.TestProtobufRpcProto.class));
+    }
+  }
+
+  private static void enableSecurity(Configuration conf) throws IOException {
+    conf.set("hadoop.security.authorization", "false");
+    conf.set("hadoop.security.authentication", "simple");
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName() +
+      "," + DummyCpService.class.getName());
+    conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+    conf.set(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, "true");
+    SecureTestUtil.configureSuperuser(conf);
+  }
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+
+    // Enable security
+    enableSecurity(conf);
+    TEST_UTIL.startMiniCluster();
+
+    // Wait for the ACL table to become available
+    TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
+
+    // Create users
+    USER_ADMIN = User.createUserForTesting(conf, "admin", new String[0]);
+    USER_NON_ADMIN = User.createUserForTesting(conf, "non_admin", new String[0]);
+    USER_GROUP_ADMIN =
+        User.createUserForTesting(conf, "user_group_admin", new String[] { GROUP_ADMIN });
+
+    // Assign permissions to users and groups
+    SecureTestUtil.grantGlobal(TEST_UTIL, USER_ADMIN.getShortName(), Permission.Action.ADMIN);
+    SecureTestUtil.grantGlobal(TEST_UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN);
+    // No permissions to USER_NON_ADMIN
+  }
+
+  interface Action {
+    void run(Admin admin) throws Exception;
+  }
+
+  private void verifyAllowed(User user, Action action) throws Exception {
+    user.runAs((PrivilegedExceptionAction<?>) () -> {
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          Admin admin = conn.getAdmin()) {
+        action.run(admin);
+      } catch (IOException e) {
+        fail(e.toString());
+      }
+      return null;
+    });
+  }
+
+  private void verifyDenied(User user, Action action) throws Exception {
+    user.runAs((PrivilegedExceptionAction<?>) () -> {
+      boolean accessDenied = false;
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          Admin admin = conn.getAdmin()) {
+        action.run(admin);
+      } catch (AccessDeniedException e) {
+        accessDenied = true;
+      }
+      assertTrue("Expected access to be denied", accessDenied);
+      return null;
+    });
+  }
+
+  private void verifyAdminCheckForAction(Action action) throws Exception {
+    verifyAllowed(USER_ADMIN, action);
+    verifyAllowed(USER_GROUP_ADMIN, action);
+    verifyDenied(USER_NON_ADMIN, action);
+  }
+
+  @Test
+  public void testEnableCatalogJanitor() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.enableCatalogJanitor(true));
+  }
+
+  @Test
+  public void testRunCatalogJanitor() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.runCatalogJanitor());
+  }
+
+  @Test
+  public void testCleanerChoreRunning() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.cleanerChoreSwitch(true));
+  }
+
+  @Test
+  public void testRunCleanerChore() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.runCleanerChore());
+  }
+
+  @Test
+  public void testExecProcedure() throws Exception {
+    verifyAdminCheckForAction((admin) -> {
+      // Using existing table instead of creating a new one.
+      admin.execProcedure("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(),
+          new HashMap<>());
+    });
+  }
+
+  @Test
+  public void testExecService() throws Exception {
+    Action action = (admin) -> {
+      TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface service =
+          TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(admin.coprocessorService());
+      service.ping(null, TestProtos.EmptyRequestProto.getDefaultInstance());
+    };
+
+    verifyAllowed(USER_ADMIN, action);
+    verifyAllowed(USER_GROUP_ADMIN, action);
+    // This is same as above verifyAccessDenied
+    USER_NON_ADMIN.runAs((PrivilegedExceptionAction<?>) () -> {
+      boolean accessDenied = false;
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          Admin admin = conn.getAdmin()) {
+        action.run(admin);
+      } catch (ServiceException e) {
+        // For MasterRpcServices.execService.
+        if (e.getCause() instanceof AccessDeniedException) {
+          accessDenied = true;
+        }
+      }
+      assertTrue("Expected access to be denied", accessDenied);
+      return null;
+    });
+  }
+
+  @Test
+  public void testExecProcedureWithRet() throws Exception {
+    verifyAdminCheckForAction((admin) -> {
+      // Using existing table instead of creating a new one.
+      admin.execProcedureWithReturn("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(),
+          new HashMap<>());
+    });
+  }
+
+  @Test
+  public void testNormalize() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.normalize());
+  }
+
+  @Test
+  public void testSetNormalizerRunning() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.normalizerSwitch(true));
+  }
+}


[31/50] [abbrv] hbase git commit: HBASE-20027 Add test TestClusterPortAssignment

Posted by zh...@apache.org.
HBASE-20027 Add test TestClusterPortAssignment

LocalHBaseCluster forces random port assignment for sake of concurrent unit test
execution friendliness, but we still need a positive test for RPC and info port
assignment.


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

Branch: refs/heads/HBASE-19064
Commit: 173a5bf1f1ff2f60ea9ef92bdef7a3026597ec0d
Parents: bf5f034
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Feb 20 17:43:58 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 21 13:12:30 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/LocalHBaseCluster.java  | 15 +++-
 .../apache/hadoop/hbase/MiniHBaseCluster.java   |  8 +-
 .../hadoop/hbase/TestClusterPortAssignment.java | 80 ++++++++++++++++++++
 3 files changed, 96 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/173a5bf1/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
index 06199f7..e19e53b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
@@ -66,6 +66,8 @@ public class LocalHBaseCluster {
   public static final String LOCAL = "local";
   /** 'local:' */
   public static final String LOCAL_COLON = LOCAL + ":";
+  public static final String ASSIGN_RANDOM_PORTS = "hbase.localcluster.assign.random.ports";
+
   private final Configuration conf;
   private final Class<? extends HMaster> masterClass;
   private final Class<? extends HRegionServer> regionServerClass;
@@ -139,10 +141,15 @@ public class LocalHBaseCluster {
 
     // Always have masters and regionservers come up on port '0' so we don't
     // clash over default ports.
-    conf.set(HConstants.MASTER_PORT, "0");
-    conf.set(HConstants.REGIONSERVER_PORT, "0");
-    if (conf.getInt(HConstants.REGIONSERVER_INFO_PORT, 0) != -1) {
-      conf.set(HConstants.REGIONSERVER_INFO_PORT, "0");
+    if (conf.getBoolean(ASSIGN_RANDOM_PORTS, true)) {
+      conf.set(HConstants.MASTER_PORT, "0");
+      conf.set(HConstants.REGIONSERVER_PORT, "0");
+      if (conf.getInt(HConstants.REGIONSERVER_INFO_PORT, 0) != -1) {
+        conf.set(HConstants.REGIONSERVER_INFO_PORT, "0");
+      }
+      if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) {
+        conf.set(HConstants.MASTER_INFO_PORT, "0");
+      }
     }
 
     this.masterClass = (Class<? extends HMaster>)

http://git-wip-us.apache.org/repos/asf/hbase/blob/173a5bf1/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 9959e31..a652284 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -108,9 +108,11 @@ public class MiniHBaseCluster extends HBaseCluster {
          Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
       throws IOException, InterruptedException {
     super(conf);
-    conf.set(HConstants.MASTER_PORT, "0");
-    if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) {
-      conf.set(HConstants.MASTER_INFO_PORT, "0");
+    if (conf.getBoolean(LocalHBaseCluster.ASSIGN_RANDOM_PORTS, true)) {
+      conf.set(HConstants.MASTER_PORT, "0");
+      if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) {
+        conf.set(HConstants.MASTER_INFO_PORT, "0");
+      }
     }
 
     // Hadoop 2

http://git-wip-us.apache.org/repos/asf/hbase/blob/173a5bf1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java
new file mode 100644
index 0000000..0c8247f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.net.BindException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(MediumTests.class)
+public class TestClusterPortAssignment {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestClusterPortAssignment.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final Log LOG = LogFactory.getLog(TestClusterPortAssignment.class);
+
+  /**
+   * Check that we can start an HBase cluster specifying a custom set of
+   * RPC and infoserver ports.
+   */
+  @Test
+  public void testClusterPortAssignment() throws Exception {
+    boolean retry = false;
+    do {
+      int masterPort =  HBaseTestingUtility.randomFreePort();
+      int masterInfoPort =  HBaseTestingUtility.randomFreePort();
+      int rsPort =  HBaseTestingUtility.randomFreePort();
+      int rsInfoPort =  HBaseTestingUtility.randomFreePort();
+      TEST_UTIL.getConfiguration().setBoolean(LocalHBaseCluster.ASSIGN_RANDOM_PORTS, false);
+      TEST_UTIL.getConfiguration().setInt(HConstants.MASTER_PORT, masterPort);
+      TEST_UTIL.getConfiguration().setInt(HConstants.MASTER_INFO_PORT, masterInfoPort);
+      TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, rsPort);
+      TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_INFO_PORT, rsInfoPort);
+      try {
+        MiniHBaseCluster cluster = TEST_UTIL.startMiniCluster();
+        assertTrue("Cluster failed to come up", cluster.waitForActiveAndReadyMaster(30000));
+        retry = false;
+        assertEquals("Master RPC port is incorrect", masterPort,
+          cluster.getMaster().getRpcServer().getListenerAddress().getPort());
+        assertEquals("Master info port is incorrect", masterInfoPort,
+          cluster.getMaster().getInfoServer().getPort());
+        assertEquals("RS RPC port is incorrect", rsPort,
+          cluster.getRegionServer(0).getRpcServer().getListenerAddress().getPort());
+        assertEquals("RS info port is incorrect", rsInfoPort,
+          cluster.getRegionServer(0).getInfoServer().getPort());
+      } catch (BindException e) {
+        LOG.info("Failed to bind, need to retry", e);
+        retry = true;
+      } finally {
+        TEST_UTIL.shutdownMiniCluster();
+      }
+    } while (retry);
+  }
+}


[05/50] [abbrv] hbase git commit: HBASE-19991 hbase-rest test fail against h3

Posted by zh...@apache.org.
HBASE-19991 hbase-rest test fail against h3


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

Branch: refs/heads/HBASE-19064
Commit: 0d6acfa0cf878172ef63d4c5900293f95c0a0fea
Parents: 8cf7a9d
Author: Mike Drob <md...@apache.org>
Authored: Tue Feb 13 14:35:26 2018 -0600
Committer: Mike Drob <md...@apache.org>
Committed: Fri Feb 16 09:12:15 2018 -0600

----------------------------------------------------------------------
 hbase-rest/pom.xml |  6 ----
 pom.xml            | 86 ++++++++++++++++++++++++++++++++++++++++++++++++-
 2 files changed, 85 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0d6acfa0/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index d4ab0b4..e70e596 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -411,12 +411,6 @@
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-mapreduce-client-core</artifactId>
           <scope>test</scope>
-          <exclusions>
-            <exclusion>
-              <groupId>com.google.guava</groupId>
-              <artifactId>guava</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d6acfa0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1cb5082..d33478a 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1417,6 +1417,7 @@
     <jetty-jsp.version>9.2.19.v20160908</jetty-jsp.version>
     <servlet.api.version>3.1.0</servlet.api.version>
     <wx.rs.api.version>2.0.1</wx.rs.api.version>
+    <!-- Jersey 2.26 requires jetty 9.4 which breaks hadoop. Tread lightly. -->
     <jersey.version>2.25.1</jersey.version>
     <glassfish.jsp.version>2.3.2</glassfish.jsp.version>
     <glassfish.el.version>3.0.1-b08</glassfish.el.version>
@@ -2502,6 +2503,7 @@
           <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-hdfs</artifactId>
+            <version>${hadoop-two.version}</version>
             <exclusions>
               <exclusion>
                 <groupId>org.apache.htrace</groupId>
@@ -2540,7 +2542,6 @@
                 <artifactId>guava</artifactId>
               </exclusion>
             </exclusions>
-            <version>${hadoop-two.version}</version>
           </dependency>
           <dependency>
             <groupId>org.apache.hadoop</groupId>
@@ -2735,6 +2736,10 @@
                 <groupId>org.codehaus.jackson</groupId>
                 <artifactId>jackson-mapper-asl</artifactId>
               </exclusion>
+             <exclusion>
+               <groupId>com.google.guava</groupId>
+               <artifactId>guava</artifactId>
+             </exclusion>
            </exclusions>
          </dependency>
          <dependency>
@@ -2775,6 +2780,10 @@
                <artifactId>jersey-core</artifactId>
              </exclusion>
              <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-server</artifactId>
+             </exclusion>
+             <exclusion>
                <groupId>javax.servlet.jsp</groupId>
                <artifactId>jsp-api</artifactId>
              </exclusion>
@@ -2843,6 +2852,24 @@
                <groupId>com.google.guava</groupId>
                <artifactId>guava</artifactId>
              </exclusion>
+             <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+           </exclusions>
+         </dependency>
+         <dependency>
+           <groupId>org.apache.hadoop</groupId>
+           <artifactId>hadoop-hdfs</artifactId>
+           <version>${hadoop-three.version}</version>
+           <type>test-jar</type>
+           <classifier>tests</classifier>
+           <scope>test</scope>
+           <exclusions>
+             <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
            </exclusions>
          </dependency>
          <dependency>
@@ -2917,6 +2944,19 @@
          </dependency>
          <dependency>
            <groupId>org.apache.hadoop</groupId>
+           <artifactId>hadoop-common</artifactId>
+           <type>test-jar</type>
+           <classifier>tests</classifier>
+           <version>${hadoop-three.version}</version>
+           <exclusions>
+             <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+           </exclusions>
+         </dependency>
+         <dependency>
+           <groupId>org.apache.hadoop</groupId>
            <artifactId>hadoop-client</artifactId>
            <version>${hadoop-three.version}</version>
          </dependency>
@@ -2963,6 +3003,50 @@
            </exclusions>
          </dependency>
          <dependency>
+           <groupId>org.apache.hadoop</groupId>
+           <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+           <version>${hadoop-three.version}</version>
+           <exclusions>
+             <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+           </exclusions>
+         </dependency>
+         <dependency>
+           <groupId>org.apache.hadoop</groupId>
+           <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+           <version>${hadoop-three.version}</version>
+           <exclusions>
+             <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+           </exclusions>
+         </dependency>
+         <dependency>
+           <groupId>org.apache.hadoop</groupId>
+           <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+           <version>${hadoop-three.version}</version>
+           <exclusions>
+             <exclusion>
+               <groupId>javax.ws.rs</groupId>
+               <artifactId>jsr311-api</artifactId>
+             </exclusion>
+           </exclusions>
+         </dependency>
+         <dependency>
+           <groupId>org.apache.hadoop</groupId>
+           <artifactId>hadoop-yarn-common</artifactId>
+           <version>${hadoop-three.version}</version>
+           <exclusions>
+             <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+           </exclusions>
+         </dependency>
+         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-minikdc</artifactId>
             <version>${hadoop-three.version}</version>


[09/50] [abbrv] hbase git commit: HBASE-20014 TestAdmin1 Times out

Posted by zh...@apache.org.
HBASE-20014 TestAdmin1 Times out


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

Branch: refs/heads/HBASE-19064
Commit: 969895105c77aaa34f4678d1924e83ebda7edb0f
Parents: dad90f6
Author: Michael Stack <st...@apache.org>
Authored: Fri Feb 16 20:55:29 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Feb 16 20:57:10 2018 -0800

----------------------------------------------------------------------
 .../src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java  | 2 +-
 src/main/asciidoc/_chapters/developer.adoc                         | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/96989510/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
index d47b213..a41e383 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
@@ -61,7 +61,7 @@ public final class HBaseClassTestRule implements TestRule {
     for (Class<?> c : categories[0].value()) {
       if (c == SmallTests.class || c == MediumTests.class || c == LargeTests.class) {
         // All tests have a 10minute timeout.
-        return TimeUnit.MINUTES.toSeconds(10);
+        return TimeUnit.MINUTES.toSeconds(13);
       }
     }
     throw new IllegalArgumentException(

http://git-wip-us.apache.org/repos/asf/hbase/blob/96989510/src/main/asciidoc/_chapters/developer.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc
index 3d8275f..3dbfe74 100644
--- a/src/main/asciidoc/_chapters/developer.adoc
+++ b/src/main/asciidoc/_chapters/developer.adoc
@@ -999,7 +999,7 @@ is that of the local test; i.e. in the TestTimeout Test Class, you'd
 pass `TestTimeout.class` to the `CLASS_RULE` instead of the
 `TestHRegionInfo.class` we have above. The `CLASS_RULE`
 is where we'll enforce timeouts (currently set at a hard-limit of
-ten minutes for all tests) and other cross-unit test facility.
+thirteen! minutes for all tests -- 780 seconds) and other cross-unit test facility.
 The test is in the `SmallTest` Category.
 
 Categories can be arbitrary and provided as a list but each test MUST


[12/50] [abbrv] hbase git commit: HBASE-18294 Reduce global heap pressure: flush based on heap occupancy

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java
index 7689fcd..1c627f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java
@@ -31,17 +31,17 @@ import org.apache.hadoop.hbase.util.Pair;
 
 /**
  * RegionServerAccounting keeps record of some basic real time information about
- * the Region Server. Currently, it keeps record the global memstore size and global memstore heap
- * overhead. It also tracks the replay edits per region.
+ * the Region Server. Currently, it keeps record the global memstore size and global memstore
+ * on-heap and off-heap overhead. It also tracks the replay edits per region.
  */
 @InterfaceAudience.Private
 public class RegionServerAccounting {
   // memstore data size
-  private final LongAdder globalMemstoreDataSize = new LongAdder();
-  // memstore heap size. When off heap MSLAB in place, this will be only heap overhead of the Cell
-  // POJOs and entry overhead of them onto memstore. When on heap MSLAB, this will be include heap
-  // overhead as well as the cell data size. Ya cell data is in on heap area only then.
-  private final LongAdder globalMemstoreHeapSize = new LongAdder();
+  private final LongAdder globalMemStoreDataSize = new LongAdder();
+  // memstore heap size.
+  private final LongAdder globalMemStoreHeapSize = new LongAdder();
+  // memstore off-heap size.
+  private final LongAdder globalMemStoreOffHeapSize = new LongAdder();
 
   // Store the edits size during replaying WAL. Use this to roll back the
   // global memstore size once a region opening failed.
@@ -114,14 +114,21 @@ public class RegionServerAccounting {
    * @return the global Memstore data size in the RegionServer
    */
   public long getGlobalMemStoreDataSize() {
-    return globalMemstoreDataSize.sum();
+    return globalMemStoreDataSize.sum();
   }
 
   /**
    * @return the global memstore heap size in the RegionServer
    */
   public long getGlobalMemStoreHeapSize() {
-    return this.globalMemstoreHeapSize.sum();
+    return this.globalMemStoreHeapSize.sum();
+  }
+
+  /**
+   * @return the global memstore heap size in the RegionServer
+   */
+  public long getGlobalMemStoreOffHeapSize() {
+    return this.globalMemStoreOffHeapSize.sum();
   }
 
   /**
@@ -129,13 +136,15 @@ public class RegionServerAccounting {
    *        the global Memstore size
    */
   public void incGlobalMemStoreSize(MemStoreSize memStoreSize) {
-    globalMemstoreDataSize.add(memStoreSize.getDataSize());
-    globalMemstoreHeapSize.add(memStoreSize.getHeapSize());
+    globalMemStoreDataSize.add(memStoreSize.getDataSize());
+    globalMemStoreHeapSize.add(memStoreSize.getHeapSize());
+    globalMemStoreOffHeapSize.add(memStoreSize.getOffHeapSize());
   }
 
   public void decGlobalMemStoreSize(MemStoreSize memStoreSize) {
-    globalMemstoreDataSize.add(-memStoreSize.getDataSize());
-    globalMemstoreHeapSize.add(-memStoreSize.getHeapSize());
+    globalMemStoreDataSize.add(-memStoreSize.getDataSize());
+    globalMemStoreHeapSize.add(-memStoreSize.getHeapSize());
+    globalMemStoreOffHeapSize.add(-memStoreSize.getOffHeapSize());
   }
 
   /**
@@ -151,13 +160,13 @@ public class RegionServerAccounting {
       }
     } else {
       // If the configured memstore is offheap, check for two things
-      // 1) If the global memstore data size is greater than the configured
+      // 1) If the global memstore off-heap size is greater than the configured
       // 'hbase.regionserver.offheap.global.memstore.size'
       // 2) If the global memstore heap size is greater than the configured onheap
       // global memstore limit 'hbase.regionserver.global.memstore.size'.
       // We do this to avoid OOME incase of scenarios where the heap is occupied with
       // lot of onheap references to the cells in memstore
-      if (getGlobalMemStoreDataSize() >= globalMemStoreLimit) {
+      if (getGlobalMemStoreOffHeapSize() >= globalMemStoreLimit) {
         // Indicates that global memstore size is above the configured
         // 'hbase.regionserver.offheap.global.memstore.size'
         return FlushType.ABOVE_OFFHEAP_HIGHER_MARK;
@@ -181,8 +190,8 @@ public class RegionServerAccounting {
         return FlushType.ABOVE_ONHEAP_LOWER_MARK;
       }
     } else {
-      if (getGlobalMemStoreDataSize() >= globalMemStoreLimitLowMark) {
-        // Indicates that the offheap memstore's data size is greater than the global memstore
+      if (getGlobalMemStoreOffHeapSize() >= globalMemStoreLimitLowMark) {
+        // Indicates that the offheap memstore's size is greater than the global memstore
         // lower limit
         return FlushType.ABOVE_OFFHEAP_LOWER_MARK;
       } else if (getGlobalMemStoreHeapSize() >= globalOnHeapMemstoreLimitLowMark) {
@@ -203,7 +212,7 @@ public class RegionServerAccounting {
     if (memType == MemoryType.HEAP) {
       return (getGlobalMemStoreHeapSize()) * 1.0 / globalMemStoreLimitLowMark;
     } else {
-      return Math.max(getGlobalMemStoreDataSize() * 1.0 / globalMemStoreLimitLowMark,
+      return Math.max(getGlobalMemStoreOffHeapSize() * 1.0 / globalMemStoreLimitLowMark,
           getGlobalMemStoreHeapSize() * 1.0 / globalOnHeapMemstoreLimitLowMark);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
index a1f5755..5b98a27 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
@@ -66,7 +66,7 @@ public class RegionServicesForStores {
   }
 
   public void addMemStoreSize(MemStoreSize size) {
-    region.addAndGetMemStoreSize(size);
+    region.incMemStoreSize(size);
   }
 
   public RegionInfo getRegionInfo() {
@@ -89,6 +89,6 @@ public class RegionServicesForStores {
 
   @VisibleForTesting
   long getMemStoreSize() {
-    return region.getMemStoreSize();
+    return region.getMemStoreDataSize();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
index 5bfab52..66a2ad5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
 import java.util.SortedSet;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.hbase.Cell;
@@ -48,9 +47,9 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 @InterfaceAudience.Private
 public abstract class Segment {
 
-  public final static long FIXED_OVERHEAD = ClassSize.align((long)ClassSize.OBJECT
-      + 6 * ClassSize.REFERENCE // cellSet, comparator, memStoreLAB, dataSize,
-                                // heapSize, and timeRangeTracker
+  public final static long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
+      + 5 * ClassSize.REFERENCE // cellSet, comparator, memStoreLAB, memStoreSizing,
+                                // and timeRangeTracker
       + Bytes.SIZEOF_LONG // minSequenceId
       + Bytes.SIZEOF_BOOLEAN); // tagsPresent
   public final static long DEEP_OVERHEAD = FIXED_OVERHEAD + ClassSize.ATOMIC_REFERENCE
@@ -62,8 +61,7 @@ public abstract class Segment {
   private MemStoreLAB memStoreLAB;
   // Sum of sizes of all Cells added to this Segment. Cell's heapSize is considered. This is not
   // including the heap overhead of this class.
-  protected final AtomicLong dataSize;
-  protected final AtomicLong heapSize;
+  protected final MemStoreSizing segmentSize;
   protected final TimeRangeTracker timeRangeTracker;
   protected volatile boolean tagsPresent;
 
@@ -71,8 +69,23 @@ public abstract class Segment {
   // and there is no need in true Segments state
   protected Segment(CellComparator comparator, TimeRangeTracker trt) {
     this.comparator = comparator;
-    this.dataSize = new AtomicLong(0);
-    this.heapSize = new AtomicLong(0);
+    this.segmentSize = new MemStoreSizing();
+    this.timeRangeTracker = trt;
+  }
+
+  protected Segment(CellComparator comparator, List<ImmutableSegment> segments,
+      TimeRangeTracker trt) {
+    long dataSize = 0;
+    long heapSize = 0;
+    long OffHeapSize = 0;
+    for (Segment segment : segments) {
+      MemStoreSize memStoreSize = segment.getMemStoreSize();
+      dataSize += memStoreSize.getDataSize();
+      heapSize += memStoreSize.getHeapSize();
+      OffHeapSize += memStoreSize.getOffHeapSize();
+    }
+    this.comparator = comparator;
+    this.segmentSize = new MemStoreSizing(dataSize, heapSize, OffHeapSize);
     this.timeRangeTracker = trt;
   }
 
@@ -82,8 +95,7 @@ public abstract class Segment {
     this.comparator = comparator;
     this.minSequenceId = Long.MAX_VALUE;
     this.memStoreLAB = memStoreLAB;
-    this.dataSize = new AtomicLong(0);
-    this.heapSize = new AtomicLong(0);
+    this.segmentSize = new MemStoreSizing();
     this.tagsPresent = false;
     this.timeRangeTracker = trt;
   }
@@ -93,8 +105,7 @@ public abstract class Segment {
     this.comparator = segment.getComparator();
     this.minSequenceId = segment.getMinSequenceId();
     this.memStoreLAB = segment.getMemStoreLAB();
-    this.dataSize = new AtomicLong(segment.keySize());
-    this.heapSize = new AtomicLong(segment.heapSize.get());
+    this.segmentSize = new MemStoreSizing(segment.getMemStoreSize());
     this.tagsPresent = segment.isTagsPresent();
     this.timeRangeTracker = segment.getTimeRangeTracker();
   }
@@ -134,17 +145,6 @@ public abstract class Segment {
   }
 
   /**
-   * @return the first cell in the segment that has equal or greater key than the given cell
-   */
-  public Cell getFirstAfter(Cell cell) {
-    SortedSet<Cell> snTailSet = tailSet(cell);
-    if (!snTailSet.isEmpty()) {
-      return snTailSet.first();
-    }
-    return null;
-  }
-
-  /**
    * Closing a segment before it is being discarded
    */
   public void close() {
@@ -221,27 +221,39 @@ public abstract class Segment {
     return this;
   }
 
+  public MemStoreSize getMemStoreSize() {
+    return this.segmentSize;
+  }
+
   /**
    * @return Sum of all cell's size.
    */
   public long keySize() {
-    return this.dataSize.get();
+    return this.segmentSize.getDataSize();
   }
 
   /**
    * @return The heap size of this segment.
    */
   public long heapSize() {
-    return this.heapSize.get();
+    return this.segmentSize.getHeapSize();
+  }
+
+  /**
+   * @return The off-heap size of this segment.
+   */
+  public long offHeapSize() {
+    return this.segmentSize.getOffHeapSize();
   }
 
   /**
    * Updates the size counters of the segment by the given delta
    */
   //TODO
-  protected void incSize(long delta, long heapOverhead) {
-    this.dataSize.addAndGet(delta);
-    this.heapSize.addAndGet(heapOverhead);
+  protected void incSize(long delta, long heapOverhead, long offHeapOverhead) {
+    synchronized (this) {
+      this.segmentSize.incMemStoreSize(delta, heapOverhead, offHeapOverhead);
+    }
   }
 
   public long getMinSequenceId() {
@@ -303,9 +315,10 @@ public abstract class Segment {
       cellSize = getCellLength(cellToAdd);
     }
     long heapSize = heapSizeChange(cellToAdd, succ);
-    incSize(cellSize, heapSize);
+    long offHeapSize = offHeapSizeChange(cellToAdd, succ);
+    incSize(cellSize, heapSize, offHeapSize);
     if (memstoreSizing != null) {
-      memstoreSizing.incMemStoreSize(cellSize, heapSize);
+      memstoreSizing.incMemStoreSize(cellSize, heapSize, offHeapSize);
     }
     getTimeRangeTracker().includeTimestamp(cellToAdd);
     minSequenceId = Math.min(minSequenceId, cellToAdd.getSequenceId());
@@ -327,10 +340,48 @@ public abstract class Segment {
    *         heap size itself and additional overhead because of addition on to CSLM.
    */
   protected long heapSizeChange(Cell cell, boolean succ) {
+    long res = 0;
     if (succ) {
-      return ClassSize
-          .align(indexEntrySize() + PrivateCellUtil.estimatedHeapSizeOf(cell));
+      boolean onHeap = true;
+      MemStoreLAB memStoreLAB = getMemStoreLAB();
+      if(memStoreLAB != null) {
+        onHeap = memStoreLAB.isOnHeap();
+      }
+      res += indexEntryOnHeapSize(onHeap);
+      if(onHeap) {
+        res += PrivateCellUtil.estimatedSizeOfCell(cell);
+      }
+      res = ClassSize.align(res);
     }
+    return res;
+  }
+
+  protected long offHeapSizeChange(Cell cell, boolean succ) {
+    long res = 0;
+    if (succ) {
+      boolean offHeap = false;
+      MemStoreLAB memStoreLAB = getMemStoreLAB();
+      if(memStoreLAB != null) {
+        offHeap = memStoreLAB.isOffHeap();
+      }
+      res += indexEntryOffHeapSize(offHeap);
+      if(offHeap) {
+        res += PrivateCellUtil.estimatedSizeOfCell(cell);
+      }
+      res = ClassSize.align(res);
+    }
+    return res;
+  }
+
+  protected long indexEntryOnHeapSize(boolean onHeap) {
+    // in most cases index is allocated on-heap
+    // override this method when it is not always the case, e.g., in CCM
+    return indexEntrySize();
+  }
+
+  protected long indexEntryOffHeapSize(boolean offHeap) {
+    // in most cases index is allocated on-heap
+    // override this method when it is not always the case, e.g., in CCM
     return 0;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 0b9b547..1624810 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -600,7 +600,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
             // Update the progress of the scanner context
             scannerContext.incrementSizeProgress(cellSize,
-              PrivateCellUtil.estimatedHeapSizeOf(cell));
+              PrivateCellUtil.estimatedSizeOfCell(cell));
             scannerContext.incrementBatchProgress(1);
 
             if (matcher.isUserScan() && totalBytesRead > maxRowSize) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/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 e7c7caf..1d4dc1b 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
@@ -168,7 +168,7 @@ public class WALEdit implements HeapSize {
   public long heapSize() {
     long ret = ClassSize.ARRAYLIST;
     for (Cell cell : cells) {
-      ret += PrivateCellUtil.estimatedHeapSizeOf(cell);
+      ret += PrivateCellUtil.estimatedSizeOfCell(cell);
     }
     return ret;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
index f66a828..2e2d978 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
@@ -99,7 +99,7 @@ public class TestGlobalMemStoreSize {
       long globalMemStoreSize = 0;
       for (RegionInfo regionInfo :
           ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
-        globalMemStoreSize += server.getRegion(regionInfo.getEncodedName()).getMemStoreSize();
+        globalMemStoreSize += server.getRegion(regionInfo.getEncodedName()).getMemStoreDataSize();
       }
       assertEquals(server.getRegionServerAccounting().getGlobalMemStoreDataSize(),
         globalMemStoreSize);
@@ -130,7 +130,7 @@ public class TestGlobalMemStoreSize {
         for (RegionInfo regionInfo :
             ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
           HRegion r = server.getRegion(regionInfo.getEncodedName());
-          long l = r.getMemStoreSize();
+          long l = r.getMemStoreDataSize();
           if (l > 0) {
             // Only meta could have edits at this stage.  Give it another flush
             // clear them.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index 3038744..965243f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -391,7 +391,7 @@ public class TestPartialResultsFromClientSide {
       // Estimate the cell heap size. One difference is that on server side, the KV Heap size is
       // estimated differently in case the cell is backed up by MSLAB byte[] (no overhead for
       // backing array). Thus below calculation is a bit brittle.
-      CELL_HEAP_SIZE = PrivateCellUtil.estimatedHeapSizeOf(result.rawCells()[0])
+      CELL_HEAP_SIZE = PrivateCellUtil.estimatedSizeOfCell(result.rawCells()[0])
           - (ClassSize.ARRAY+3);
       if (LOG.isInfoEnabled()) LOG.info("Cell heap size: " + CELL_HEAP_SIZE);
       scanner.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
index 162be35..5a3ba82 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.client.metrics.ServerSideScanMetrics;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
@@ -156,7 +155,7 @@ public class TestServerSideScanMetricsFromClientSide {
       assertTrue(result.rawCells() != null);
       assertTrue(result.rawCells().length == 1);
 
-      CELL_HEAP_SIZE = PrivateCellUtil.estimatedHeapSizeOf(result.rawCells()[0]);
+      CELL_HEAP_SIZE = PrivateCellUtil.estimatedSizeOfCell(result.rawCells()[0]);
       scanner.close();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
index fbb87bb..762dbd1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
@@ -215,30 +215,30 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
     ASYNC_CONN.getTable(tableName)
         .put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-1")))
         .join();
-    assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0);
+    assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreDataSize() > 0);
     // flush region and wait flush operation finished.
     LOG.info("flushing region: " + Bytes.toStringBinary(hri.getRegionName()));
     admin.flushRegion(hri.getRegionName()).get();
     LOG.info("blocking until flush is complete: " + Bytes.toStringBinary(hri.getRegionName()));
     Threads.sleepWithoutInterrupt(500);
-    while (regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0) {
+    while (regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreDataSize() > 0) {
       Threads.sleep(50);
     }
     // check the memstore.
-    assertEquals(0, regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize());
+    assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreDataSize(), 0);
 
     // write another put into the specific region
     ASYNC_CONN.getTable(tableName)
         .put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-2")))
         .join();
-    assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0);
+    assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreDataSize() > 0);
     admin.flush(tableName).get();
     Threads.sleepWithoutInterrupt(500);
-    while (regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0) {
+    while (regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreDataSize() > 0) {
       Threads.sleep(50);
     }
     // check the memstore.
-    assertEquals(0, regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize());
+    assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreDataSize(), 0);
   }
 
   private void waitUntilMobCompactionFinished(TableName tableName)

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
index 48d9a93..d6f32f5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
@@ -36,9 +36,7 @@ import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.MemStoreSize;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -67,7 +65,7 @@ public class TestClientPushback {
   private static final TableName tableName = TableName.valueOf("client-pushback");
   private static final byte[] family = Bytes.toBytes("f");
   private static final byte[] qualifier = Bytes.toBytes("q");
-  private static final long flushSizeBytes = 256;
+  private static final long flushSizeBytes = 512;
 
   @BeforeClass
   public static void setupCluster() throws Exception{
@@ -110,7 +108,7 @@ public class TestClientPushback {
     mutator.flush();
 
     // get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data
-    int load = (int) ((((HRegion) region).addAndGetMemStoreSize(new MemStoreSize(0, 0)) * 100)
+    int load = (int) ((region.getMemStoreHeapSize() * 100)
         / flushSizeBytes);
     LOG.debug("Done writing some data to "+tableName);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java
index dc3f8da..207e1fc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFlushFromClient.java
@@ -93,7 +93,7 @@ public class TestFlushFromClient {
       t.put(puts);
     }
     assertFalse(getRegionInfo().isEmpty());
-    assertTrue(getRegionInfo().stream().allMatch(r -> r.getMemStoreSize() != 0));
+    assertTrue(getRegionInfo().stream().allMatch(r -> r.getMemStoreDataSize() != 0));
   }
 
   @After
@@ -108,7 +108,7 @@ public class TestFlushFromClient {
   public void testFlushTable() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
       admin.flush(tableName);
-      assertFalse(getRegionInfo().stream().anyMatch(r -> r.getMemStoreSize() != 0));
+      assertFalse(getRegionInfo().stream().anyMatch(r -> r.getMemStoreDataSize() != 0));
     }
   }
 
@@ -116,7 +116,7 @@ public class TestFlushFromClient {
   public void testAsyncFlushTable() throws Exception {
     AsyncAdmin admin = asyncConn.getAdmin();
     admin.flush(tableName).get();
-    assertFalse(getRegionInfo().stream().anyMatch(r -> r.getMemStoreSize() != 0));
+    assertFalse(getRegionInfo().stream().anyMatch(r -> r.getMemStoreDataSize() != 0));
   }
 
   @Test
@@ -125,7 +125,7 @@ public class TestFlushFromClient {
       for (HRegion r : getRegionInfo()) {
         admin.flushRegion(r.getRegionInfo().getRegionName());
         TimeUnit.SECONDS.sleep(1);
-        assertEquals(0, r.getMemStoreSize());
+        assertEquals(0, r.getMemStoreDataSize());
       }
     }
   }
@@ -136,7 +136,7 @@ public class TestFlushFromClient {
     for (HRegion r : getRegionInfo()) {
       admin.flushRegion(r.getRegionInfo().getRegionName()).get();
       TimeUnit.SECONDS.sleep(1);
-      assertEquals(0, r.getMemStoreSize());
+      assertEquals(0, r.getMemStoreDataSize());
     }
   }
 
@@ -148,7 +148,7 @@ public class TestFlushFromClient {
             .stream().map(JVMClusterUtil.RegionServerThread::getRegionServer)
             .collect(Collectors.toList())) {
         admin.flushRegionServer(rs.getServerName());
-        assertFalse(getRegionInfo(rs).stream().anyMatch(r -> r.getMemStoreSize() != 0));
+        assertFalse(getRegionInfo(rs).stream().anyMatch(r -> r.getMemStoreDataSize() != 0));
       }
     }
   }
@@ -161,7 +161,7 @@ public class TestFlushFromClient {
       .stream().map(JVMClusterUtil.RegionServerThread::getRegionServer)
       .collect(Collectors.toList())) {
       admin.flushRegionServer(rs.getServerName()).get();
-      assertFalse(getRegionInfo(rs).stream().anyMatch(r -> r.getMemStoreSize() != 0));
+      assertFalse(getRegionInfo(rs).stream().anyMatch(r -> r.getMemStoreDataSize() != 0));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
index a336274..cadce8a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
@@ -53,7 +53,7 @@ public class TestSizeFailures {
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   protected static int SLAVES = 1;
   private static TableName TABLENAME;
-  private static final int NUM_ROWS = 1000 * 1000, NUM_COLS = 10;
+  private static final int NUM_ROWS = 1000 * 1000, NUM_COLS = 9;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -129,7 +129,7 @@ public class TestSizeFailures {
       long rowsObserved = entry.getKey();
       long entriesObserved = entry.getValue();
 
-      // Verify that we see 1M rows and 10M cells
+      // Verify that we see 1M rows and 9M cells
       assertEquals(NUM_ROWS, rowsObserved);
       assertEquals(NUM_ROWS * NUM_COLS, entriesObserved);
     }
@@ -152,7 +152,7 @@ public class TestSizeFailures {
       long rowsObserved = entry.getKey();
       long entriesObserved = entry.getValue();
 
-      // Verify that we see 1M rows and 10M cells
+      // Verify that we see 1M rows and 9M cells
       assertEquals(NUM_ROWS, rowsObserved);
       assertEquals(NUM_ROWS * NUM_COLS, entriesObserved);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java
index 336d342..4a92d4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.MemStoreSize;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -111,7 +110,7 @@ public class TestNegativeMemStoreSizeWithSlowCoprocessor {
 
       if (Bytes.equals(put.getRow(), Bytes.toBytes("row2"))) {
         region.flush(false);
-        Assert.assertTrue(region.addAndGetMemStoreSize(new MemStoreSize()) >= 0);
+        Assert.assertTrue(region.getMemStoreDataSize() >= 0);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index d68191c..505c2f0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -699,7 +699,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     int totalCellsLen1 = addRowsByKeys(memstore, keys1);// Adding 4 cells.
     int oneCellOnCSLMHeapSize = 120;
     int oneCellOnCAHeapSize = 88;
-    assertEquals(totalCellsLen1, region.getMemStoreSize());
+    assertEquals(totalCellsLen1, region.getMemStoreDataSize());
     long totalHeapSize = MutableSegment.DEEP_OVERHEAD + 4 * oneCellOnCSLMHeapSize;
     assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize());
     ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
@@ -780,7 +780,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
 
     int totalCellsLen1 = addRowsByKeys(memstore, keys1);// Adding 3 cells.
     int oneCellOnCSLMHeapSize = 120;
-    assertEquals(totalCellsLen1, region.getMemStoreSize());
+    assertEquals(totalCellsLen1, region.getMemStoreDataSize());
     long totalHeapSize = MutableSegment.DEEP_OVERHEAD + 3 * oneCellOnCSLMHeapSize;
     assertEquals(totalHeapSize, memstore.heapSize());
 
@@ -838,7 +838,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       LOG.debug("added kv: " + kv.getKeyString() + ", timestamp:" + kv.getTimestamp());
     }
     regionServicesForStores.addMemStoreSize(new MemStoreSize(hmc.getActive().keySize() - size,
-        hmc.getActive().heapSize() - heapOverhead));
+        hmc.getActive().heapSize() - heapOverhead, 0));
     return totalLen;
   }
 
@@ -859,7 +859,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       LOG.debug("added kv: " + kv.getKeyString() + ", timestamp:" + kv.getTimestamp());
     }
     regionServicesForStores.addMemStoreSize(new MemStoreSize(hmc.getActive().keySize() - size,
-            hmc.getActive().heapSize() - heapOverhead));
+            hmc.getActive().heapSize() - heapOverhead, 0));
     return totalLen;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java
index a8561de..12b078a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java
@@ -221,7 +221,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
     long cellBeforeFlushSize = cellBeforeFlushSize();
     long cellAfterFlushSize = cellAfterFlushSize();
     long totalHeapSize1 = MutableSegment.DEEP_OVERHEAD + 4 * cellBeforeFlushSize;
-    assertEquals(totalCellsLen1, region.getMemStoreSize());
+    assertEquals(totalCellsLen1, region.getMemStoreDataSize());
     assertEquals(totalHeapSize1, ((CompactingMemStore) memstore).heapSize());
 
     MemStoreSize size = memstore.getFlushableSize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
index 2c12341..b0302f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
@@ -321,7 +321,7 @@ public class TestEndToEndSplitTransaction {
     admin.flushRegion(regionName);
     log("blocking until flush is complete: " + Bytes.toStringBinary(regionName));
     Threads.sleepWithoutInterrupt(500);
-    while (rs.getOnlineRegion(regionName).getMemStoreSize() > 0) {
+    while (rs.getOnlineRegion(regionName).getMemStoreDataSize() > 0) {
       Threads.sleep(50);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 88e1aa2..31dfa2a 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
@@ -310,7 +310,7 @@ public class TestHRegion {
     region.put(put);
     // Close with something in memstore and something in the snapshot.  Make sure all is cleared.
     region.close();
-    assertEquals(0, region.getMemStoreSize());
+    assertEquals(0, region.getMemStoreDataSize());
     HBaseTestingUtility.closeRegionAndWAL(region);
   }
 
@@ -391,14 +391,14 @@ public class TestHRegion {
     HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog,
         COLUMN_FAMILY_BYTES);
     HStore store = region.getStore(COLUMN_FAMILY_BYTES);
-    assertEquals(0, region.getMemStoreSize());
+    assertEquals(0, region.getMemStoreDataSize());
 
     // Put one value
     byte [] value = Bytes.toBytes(method);
     Put put = new Put(value);
     put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
     region.put(put);
-    long onePutSize = region.getMemStoreSize();
+    long onePutSize = region.getMemStoreDataSize();
     assertTrue(onePutSize > 0);
 
     RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
@@ -414,7 +414,7 @@ public class TestHRegion {
     } catch (IOException expected) {
     }
     long expectedSize = onePutSize * 2;
-    assertEquals("memstoreSize should be incremented", expectedSize, region.getMemStoreSize());
+    assertEquals("memstoreSize should be incremented", expectedSize, region.getMemStoreDataSize());
     assertEquals("flushable size should be incremented", expectedSize,
         store.getFlushableSize().getDataSize());
 
@@ -459,13 +459,13 @@ public class TestHRegion {
           // Initialize region
           region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, wal,
               COLUMN_FAMILY_BYTES);
-          long size = region.getMemStoreSize();
+          long size = region.getMemStoreDataSize();
           Assert.assertEquals(0, size);
           // Put one item into memstore.  Measure the size of one item in memstore.
           Put p1 = new Put(row);
           p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[]) null));
           region.put(p1);
-          final long sizeOfOnePut = region.getMemStoreSize();
+          final long sizeOfOnePut = region.getMemStoreDataSize();
           // Fail a flush which means the current memstore will hang out as memstore 'snapshot'.
           try {
             LOG.info("Flushing");
@@ -478,7 +478,7 @@ public class TestHRegion {
           // Make it so all writes succeed from here on out
           ffs.fault.set(false);
           // Check sizes.  Should still be the one entry.
-          Assert.assertEquals(sizeOfOnePut, region.getMemStoreSize());
+          Assert.assertEquals(sizeOfOnePut, region.getMemStoreDataSize());
           // Now add two entries so that on this next flush that fails, we can see if we
           // subtract the right amount, the snapshot size only.
           Put p2 = new Put(row);
@@ -486,13 +486,13 @@ public class TestHRegion {
           p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null));
           region.put(p2);
           long expectedSize = sizeOfOnePut * 3;
-          Assert.assertEquals(expectedSize, region.getMemStoreSize());
+          Assert.assertEquals(expectedSize, region.getMemStoreDataSize());
           // Do a successful flush.  It will clear the snapshot only.  Thats how flushes work.
           // If already a snapshot, we clear it else we move the memstore to be snapshot and flush
           // it
           region.flush(true);
           // Make sure our memory accounting is right.
-          Assert.assertEquals(sizeOfOnePut * 2, region.getMemStoreSize());
+          Assert.assertEquals(sizeOfOnePut * 2, region.getMemStoreDataSize());
         } finally {
           HBaseTestingUtility.closeRegionAndWAL(region);
         }
@@ -524,7 +524,7 @@ public class TestHRegion {
           // Initialize region
           region = initHRegion(tableName, null, null, false,
               Durability.SYNC_WAL, wal, COLUMN_FAMILY_BYTES);
-          long size = region.getMemStoreSize();
+          long size = region.getMemStoreDataSize();
           Assert.assertEquals(0, size);
           // Put one item into memstore.  Measure the size of one item in memstore.
           Put p1 = new Put(row);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 44ff84d..26b57b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -366,7 +366,7 @@ public class TestHRegionReplayEvents {
         verifyData(secondaryRegion, 0, lastReplayed, cq, families);
         HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
         long storeMemstoreSize = store.getMemStoreSize().getHeapSize();
-        long regionMemstoreSize = secondaryRegion.getMemStoreSize();
+        long regionMemstoreSize = secondaryRegion.getMemStoreDataSize();
         long storeFlushableSize = store.getFlushableSize().getHeapSize();
         long storeSize = store.getSize();
         long storeSizeUncompressed = store.getStoreSizeUncompressed();
@@ -395,7 +395,7 @@ public class TestHRegionReplayEvents {
           assertTrue(storeFlushableSize > newFlushableSize);
 
           // assert that the region memstore is smaller now
-          long newRegionMemstoreSize = secondaryRegion.getMemStoreSize();
+          long newRegionMemstoreSize = secondaryRegion.getMemStoreDataSize();
           assertTrue(regionMemstoreSize > newRegionMemstoreSize);
 
           // assert that the store sizes are bigger
@@ -465,7 +465,7 @@ public class TestHRegionReplayEvents {
         // first verify that everything is replayed and visible before flush event replay
         HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
         long storeMemstoreSize = store.getMemStoreSize().getHeapSize();
-        long regionMemstoreSize = secondaryRegion.getMemStoreSize();
+        long regionMemstoreSize = secondaryRegion.getMemStoreDataSize();
         long storeFlushableSize = store.getFlushableSize().getHeapSize();
 
         if (flushDesc.getAction() == FlushAction.START_FLUSH) {
@@ -505,7 +505,7 @@ public class TestHRegionReplayEvents {
     assertNotNull(secondaryRegion.getPrepareFlushResult());
     assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId,
       startFlushDesc.getFlushSequenceNumber());
-    assertTrue(secondaryRegion.getMemStoreSize() > 0); // memstore is not empty
+    assertTrue(secondaryRegion.getMemStoreDataSize() > 0); // memstore is not empty
     verifyData(secondaryRegion, 0, numRows, cq, families);
 
     // Test case 2: replay a flush start marker with a smaller seqId
@@ -518,7 +518,7 @@ public class TestHRegionReplayEvents {
     assertNotNull(secondaryRegion.getPrepareFlushResult());
     assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId,
       startFlushDesc.getFlushSequenceNumber());
-    assertTrue(secondaryRegion.getMemStoreSize() > 0); // memstore is not empty
+    assertTrue(secondaryRegion.getMemStoreDataSize() > 0); // memstore is not empty
     verifyData(secondaryRegion, 0, numRows, cq, families);
 
     // Test case 3: replay a flush start marker with a larger seqId
@@ -531,7 +531,7 @@ public class TestHRegionReplayEvents {
     assertNotNull(secondaryRegion.getPrepareFlushResult());
     assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId,
       startFlushDesc.getFlushSequenceNumber());
-    assertTrue(secondaryRegion.getMemStoreSize() > 0); // memstore is not empty
+    assertTrue(secondaryRegion.getMemStoreDataSize() > 0); // memstore is not empty
     verifyData(secondaryRegion, 0, numRows, cq, families);
 
     LOG.info("-- Verifying edits from secondary");
@@ -600,7 +600,7 @@ public class TestHRegionReplayEvents {
     for (HStore s : secondaryRegion.getStores()) {
       assertEquals(expectedStoreFileCount, s.getStorefilesCount());
     }
-    long regionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long regionMemstoreSize = secondaryRegion.getMemStoreDataSize();
 
     // Test case 1: replay the a flush commit marker smaller than what we have prepared
     LOG.info("Testing replaying flush COMMIT " + commitFlushDesc + " on top of flush START"
@@ -620,7 +620,7 @@ public class TestHRegionReplayEvents {
     assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped
 
     // assert that the region memstore is same as before
-    long newRegionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long newRegionMemstoreSize = secondaryRegion.getMemStoreDataSize();
     assertEquals(regionMemstoreSize, newRegionMemstoreSize);
 
     assertNotNull(secondaryRegion.getPrepareFlushResult()); // not dropped
@@ -690,7 +690,7 @@ public class TestHRegionReplayEvents {
     for (HStore s : secondaryRegion.getStores()) {
       assertEquals(expectedStoreFileCount, s.getStorefilesCount());
     }
-    long regionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long regionMemstoreSize = secondaryRegion.getMemStoreDataSize();
 
     // Test case 1: replay the a flush commit marker larger than what we have prepared
     LOG.info("Testing replaying flush COMMIT " + commitFlushDesc + " on top of flush START"
@@ -710,7 +710,7 @@ public class TestHRegionReplayEvents {
     assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped
 
     // assert that the region memstore is smaller than before, but not empty
-    long newRegionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long newRegionMemstoreSize = secondaryRegion.getMemStoreDataSize();
     assertTrue(newRegionMemstoreSize > 0);
     assertTrue(regionMemstoreSize > newRegionMemstoreSize);
 
@@ -791,7 +791,7 @@ public class TestHRegionReplayEvents {
     for (HStore s : secondaryRegion.getStores()) {
       assertEquals(expectedStoreFileCount, s.getStorefilesCount());
     }
-    long regionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long regionMemstoreSize = secondaryRegion.getMemStoreDataSize();
 
     // Test case 1: replay a flush commit marker without start flush marker
     assertNull(secondaryRegion.getPrepareFlushResult());
@@ -820,7 +820,7 @@ public class TestHRegionReplayEvents {
     }
 
     // assert that the region memstore is same as before (we could not drop)
-    long newRegionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long newRegionMemstoreSize = secondaryRegion.getMemStoreDataSize();
     if (droppableMemstore) {
       assertTrue(0 == newRegionMemstoreSize);
     } else {
@@ -890,7 +890,7 @@ public class TestHRegionReplayEvents {
     for (HStore s : secondaryRegion.getStores()) {
       assertEquals(expectedStoreFileCount, s.getStorefilesCount());
     }
-    long regionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long regionMemstoreSize = secondaryRegion.getMemStoreDataSize();
     assertTrue(regionMemstoreSize == 0);
 
     // now replay the region open event that should contain new file locations
@@ -907,7 +907,7 @@ public class TestHRegionReplayEvents {
     assertTrue(newFlushableSize == MutableSegment.DEEP_OVERHEAD);
 
     // assert that the region memstore is empty
-    long newRegionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long newRegionMemstoreSize = secondaryRegion.getMemStoreDataSize();
     assertTrue(newRegionMemstoreSize == 0);
 
     assertNull(secondaryRegion.getPrepareFlushResult()); //prepare snapshot should be dropped if any
@@ -986,7 +986,7 @@ public class TestHRegionReplayEvents {
     assertTrue(newSnapshotSize.getDataSize() == 0);
 
     // assert that the region memstore is empty
-    long newRegionMemstoreSize = secondaryRegion.getMemStoreSize();
+    long newRegionMemstoreSize = secondaryRegion.getMemStoreDataSize();
     assertTrue(newRegionMemstoreSize == 0);
 
     assertNull(secondaryRegion.getPrepareFlushResult()); //prepare snapshot should be dropped if any
@@ -1434,7 +1434,7 @@ public class TestHRegionReplayEvents {
     LOG.info("-- Replaying edits in secondary");
 
     // Test case 4: replay some edits, ensure that memstore is dropped.
-    assertTrue(secondaryRegion.getMemStoreSize() == 0);
+    assertTrue(secondaryRegion.getMemStoreDataSize() == 0);
     putDataWithFlushes(primaryRegion, 400, 400, 0);
     numRows = 400;
 
@@ -1452,11 +1452,11 @@ public class TestHRegionReplayEvents {
       }
     }
 
-    assertTrue(secondaryRegion.getMemStoreSize() > 0);
+    assertTrue(secondaryRegion.getMemStoreDataSize() > 0);
 
     secondaryRegion.refreshStoreFiles();
 
-    assertTrue(secondaryRegion.getMemStoreSize() == 0);
+    assertTrue(secondaryRegion.getMemStoreDataSize() == 0);
 
     LOG.info("-- Verifying edits from primary");
     verifyData(primaryRegion, 0, numRows, cq, families);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
index ea40200..9479890 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
@@ -265,7 +265,7 @@ public class TestHStore {
         MemStoreSizing kvSize = new MemStoreSizing();
         store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), kvSize);
         // add the heap size of active (mutable) segment
-        kvSize.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD);
+        kvSize.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD, 0);
         size = store.memstore.getFlushableSize();
         assertEquals(kvSize, size);
         // Flush.  Bug #1 from HBASE-10466.  Make sure size calculation on failed flush is right.
@@ -278,12 +278,12 @@ public class TestHStore {
         }
         // due to snapshot, change mutable to immutable segment
         kvSize.incMemStoreSize(0,
-            CSLMImmutableSegment.DEEP_OVERHEAD_CSLM-MutableSegment.DEEP_OVERHEAD);
+            CSLMImmutableSegment.DEEP_OVERHEAD_CSLM-MutableSegment.DEEP_OVERHEAD, 0);
         size = store.memstore.getFlushableSize();
         assertEquals(kvSize, size);
         MemStoreSizing kvSize2 = new MemStoreSizing();
         store.add(new KeyValue(row, family, qf2, 2, (byte[])null), kvSize2);
-        kvSize2.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD);
+        kvSize2.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD, 0);
         // Even though we add a new kv, we expect the flushable size to be 'same' since we have
         // not yet cleared the snapshot -- the above flush failed.
         assertEquals(kvSize, size);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
index 353cc28..fded9ba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
@@ -149,7 +149,7 @@ public class TestPerColumnFamilyFlush {
       }
     }
 
-    long totalMemstoreSize = region.getMemStoreSize();
+    long totalMemstoreSize = region.getMemStoreDataSize();
 
     // Find the smallest LSNs for edits wrt to each CF.
     long smallestSeqCF1 = region.getOldestSeqIdOfStore(FAMILY1);
@@ -192,7 +192,7 @@ public class TestPerColumnFamilyFlush {
     cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
     cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
     cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
-    totalMemstoreSize = region.getMemStoreSize();
+    totalMemstoreSize = region.getMemStoreDataSize();
     smallestSeqInRegionCurrentMemstore = getWAL(region)
         .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
 
@@ -230,7 +230,7 @@ public class TestPerColumnFamilyFlush {
     cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
     cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
     cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
-    totalMemstoreSize = region.getMemStoreSize();
+    totalMemstoreSize = region.getMemStoreDataSize();
     smallestSeqInRegionCurrentMemstore = getWAL(region)
         .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
 
@@ -265,7 +265,7 @@ public class TestPerColumnFamilyFlush {
 
     // Since we won't find any CF above the threshold, and hence no specific
     // store to flush, we should flush all the memstores.
-    assertEquals(0, region.getMemStoreSize());
+    assertEquals(0, region.getMemStoreDataSize());
     HBaseTestingUtility.closeRegionAndWAL(region);
   }
 
@@ -289,7 +289,7 @@ public class TestPerColumnFamilyFlush {
       }
     }
 
-    long totalMemstoreSize = region.getMemStoreSize();
+    long totalMemstoreSize = region.getMemStoreDataSize();
 
     // Find the sizes of the memstores of each CF.
     MemStoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
@@ -312,7 +312,7 @@ public class TestPerColumnFamilyFlush {
     cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
     cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
     cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
-    totalMemstoreSize = region.getMemStoreSize();
+    totalMemstoreSize = region.getMemStoreDataSize();
     long smallestSeqInRegionCurrentMemstore =
         region.getWAL().getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
 
@@ -381,7 +381,7 @@ public class TestPerColumnFamilyFlush {
 
       long totalMemstoreSize;
       long cf1MemstoreSize, cf2MemstoreSize, cf3MemstoreSize;
-      totalMemstoreSize = desiredRegion.getMemStoreSize();
+      totalMemstoreSize = desiredRegion.getMemStoreDataSize();
 
       // Find the sizes of the memstores of each CF.
       cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getMemStoreSize().getDataSize();
@@ -504,12 +504,12 @@ public class TestPerColumnFamilyFlush {
 
         @Override
         public boolean evaluate() throws Exception {
-          return desiredRegion.getMemStoreSize() == 0;
+          return desiredRegion.getMemStoreDataSize() == 0;
         }
 
         @Override
         public String explainFailure() throws Exception {
-          long memstoreSize = desiredRegion.getMemStoreSize();
+          long memstoreSize = desiredRegion.getMemStoreDataSize();
           if (memstoreSize > 0) {
             return "Still have unflushed entries in memstore, memstore size is " + memstoreSize;
           }
@@ -551,7 +551,7 @@ public class TestPerColumnFamilyFlush {
       put.addColumn(FAMILY3, qf, value3);
       table.put(put);
       // slow down to let regionserver flush region.
-      while (region.getMemStoreSize() > memstoreFlushSize) {
+      while (region.getMemStoreHeapSize() > memstoreFlushSize) {
         Thread.sleep(100);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java
index ede9cae..7bd9e16 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java
@@ -42,7 +42,7 @@ public class TestRegionServerAccounting {
     // try for default cases
     RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf);
     MemStoreSize memstoreSize =
-        new MemStoreSize(3L * 1024 * 1024 * 1024, 1L * 1024 * 1024 * 1024);
+        new MemStoreSize((3L * 1024L * 1024L * 1024L), (1L * 1024L * 1024L * 1024L), 0);
     regionServerAccounting.incGlobalMemStoreSize(memstoreSize);
     assertEquals(FlushType.ABOVE_ONHEAP_HIGHER_MARK,
       regionServerAccounting.isAboveHighWaterMark());
@@ -55,7 +55,7 @@ public class TestRegionServerAccounting {
     // try for default cases
     RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf);
     MemStoreSize memstoreSize =
-        new MemStoreSize(3L * 1024 * 1024 * 1024, 1L * 1024 * 1024 * 1024);
+        new MemStoreSize((3L * 1024L * 1024L * 1024L), (1L * 1024L * 1024L * 1024L), 0);
     regionServerAccounting.incGlobalMemStoreSize(memstoreSize);
     assertEquals(FlushType.ABOVE_ONHEAP_LOWER_MARK,
       regionServerAccounting.isAboveLowWaterMark());
@@ -65,12 +65,12 @@ public class TestRegionServerAccounting {
   public void testOffheapMemstoreHigherWaterMarkLimitsDueToDataSize() {
     Configuration conf = HBaseConfiguration.create();
     // setting 1G as offheap data size
-    conf.setLong(MemorySizeUtil.OFFHEAP_MEMSTORE_SIZE_KEY, (1L * 1024));
+    conf.setLong(MemorySizeUtil.OFFHEAP_MEMSTORE_SIZE_KEY, (1L * 1024L));
     // try for default cases
     RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf);
     // this will breach offheap limit as data size is higher and not due to heap size
     MemStoreSize memstoreSize =
-        new MemStoreSize(3L * 1024 * 1024 * 1024, 1L * 1024 * 1024 * 1024);
+        new MemStoreSize((3L * 1024L * 1024L * 1024L), 0, (1L * 1024L * 1024L * 1024L));
     regionServerAccounting.incGlobalMemStoreSize(memstoreSize);
     assertEquals(FlushType.ABOVE_OFFHEAP_HIGHER_MARK,
       regionServerAccounting.isAboveHighWaterMark());
@@ -81,12 +81,12 @@ public class TestRegionServerAccounting {
     Configuration conf = HBaseConfiguration.create();
     conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.2f);
     // setting 1G as offheap data size
-    conf.setLong(MemorySizeUtil.OFFHEAP_MEMSTORE_SIZE_KEY, (1L * 1024));
+    conf.setLong(MemorySizeUtil.OFFHEAP_MEMSTORE_SIZE_KEY, (1L * 1024L));
     // try for default cases
     RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf);
     // this will breach higher limit as heap size is higher and not due to offheap size
     MemStoreSize memstoreSize =
-        new MemStoreSize(3L * 1024 * 1024, 2L * 1024 * 1024 * 1024);
+        new MemStoreSize((3L * 1024L * 1024L), (2L * 1024L * 1024L * 1024L), 0);
     regionServerAccounting.incGlobalMemStoreSize(memstoreSize);
     assertEquals(FlushType.ABOVE_ONHEAP_HIGHER_MARK,
       regionServerAccounting.isAboveHighWaterMark());
@@ -96,12 +96,12 @@ public class TestRegionServerAccounting {
   public void testOffheapMemstoreLowerWaterMarkLimitsDueToDataSize() {
     Configuration conf = HBaseConfiguration.create();
     // setting 1G as offheap data size
-    conf.setLong(MemorySizeUtil.OFFHEAP_MEMSTORE_SIZE_KEY, (1L * 1024));
+    conf.setLong(MemorySizeUtil.OFFHEAP_MEMSTORE_SIZE_KEY, (1L * 1024L));
     // try for default cases
     RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf);
     // this will breach offheap limit as data size is higher and not due to heap size
     MemStoreSize memstoreSize =
-        new MemStoreSize(3L * 1024 * 1024 * 1024, 1L * 1024 * 1024 * 1024);
+        new MemStoreSize((3L * 1024L * 1024L * 1024L), 0, (1L * 1024L * 1024L * 1024L));
     regionServerAccounting.incGlobalMemStoreSize(memstoreSize);
     assertEquals(FlushType.ABOVE_OFFHEAP_LOWER_MARK,
       regionServerAccounting.isAboveLowWaterMark());
@@ -112,12 +112,12 @@ public class TestRegionServerAccounting {
     Configuration conf = HBaseConfiguration.create();
     conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.2f);
     // setting 1G as offheap data size
-    conf.setLong(MemorySizeUtil.OFFHEAP_MEMSTORE_SIZE_KEY, (1L * 1024));
+    conf.setLong(MemorySizeUtil.OFFHEAP_MEMSTORE_SIZE_KEY, (1L * 1024L));
     // try for default cases
     RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf);
     // this will breach higher limit as heap size is higher and not due to offheap size
     MemStoreSize memstoreSize =
-        new MemStoreSize(3L * 1024 * 1024, 2L * 1024 * 1024 * 1024);
+        new MemStoreSize((3L * 1024L * 1024L), (2L * 1024L * 1024L * 1024L), 0);
     regionServerAccounting.incGlobalMemStoreSize(memstoreSize);
     assertEquals(FlushType.ABOVE_ONHEAP_LOWER_MARK,
       regionServerAccounting.isAboveLowWaterMark());

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index ca65914..5eb8fa8 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
@@ -264,8 +264,8 @@ public class TestWALLockup {
         @Override
         public void run() {
           try {
-            if (region.getMemStoreSize() <= 0) {
-              throw new IOException("memstore size=" + region.getMemStoreSize());
+            if (region.getMemStoreDataSize() <= 0) {
+              throw new IOException("memstore size=" + region.getMemStoreDataSize());
             }
             region.flush(false);
           } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
index 3f73d37..15bf2a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
@@ -172,7 +172,7 @@ public class TestWalAndCompactingMemStoreFlush {
       region.put(createPut(2, i));
     }
 
-    long totalMemstoreSize = region.getMemStoreSize();
+    long totalMemstoreSize = region.getMemStoreDataSize();
 
     // Find the smallest LSNs for edits wrt to each CF.
     long smallestSeqCF1PhaseI = region.getOldestSeqIdOfStore(FAMILY1);
@@ -363,13 +363,13 @@ public class TestWalAndCompactingMemStoreFlush {
     s = s + "----AFTER THIRD AND FORTH FLUSH, The smallest sequence in region WAL is: "
         + smallestSeqInRegionCurrentMemstorePhaseV
         + ". After additional inserts and last flush, the entire region size is:" + region
-        .getMemStoreSize()
+        .getMemStoreDataSize()
         + "\n----------------------------------\n";
 
     // Since we won't find any CF above the threshold, and hence no specific
     // store to flush, we should flush all the memstores
     // Also compacted memstores are flushed to disk.
-    assertEquals(0, region.getMemStoreSize());
+    assertEquals(0, region.getMemStoreDataSize());
     System.out.println(s);
     HBaseTestingUtility.closeRegionAndWAL(region);
   }
@@ -411,7 +411,7 @@ public class TestWalAndCompactingMemStoreFlush {
     /*------------------------------------------------------------------------------*/
     /*------------------------------------------------------------------------------*/
     /* PHASE I - collect sizes */
-    long totalMemstoreSizePhaseI = region.getMemStoreSize();
+    long totalMemstoreSizePhaseI = region.getMemStoreDataSize();
     // Find the smallest LSNs for edits wrt to each CF.
     long smallestSeqCF1PhaseI = region.getOldestSeqIdOfStore(FAMILY1);
     long smallestSeqCF2PhaseI = region.getOldestSeqIdOfStore(FAMILY2);
@@ -474,7 +474,7 @@ public class TestWalAndCompactingMemStoreFlush {
         .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
     // Find the smallest LSNs for edits wrt to each CF.
     long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3);
-    long totalMemstoreSizePhaseII = region.getMemStoreSize();
+    long totalMemstoreSizePhaseII = region.getMemStoreDataSize();
 
     /*------------------------------------------------------------------------------*/
     /* PHASE II - validation */
@@ -517,7 +517,7 @@ public class TestWalAndCompactingMemStoreFlush {
     /* PHASE III - collect sizes */
     // How much does the CF1 memstore occupy now? Will be used later.
     MemStoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize();
-    long totalMemstoreSizePhaseIII = region.getMemStoreSize();
+    long totalMemstoreSizePhaseIII = region.getMemStoreDataSize();
 
     /*------------------------------------------------------------------------------*/
     /* PHASE III - validation */
@@ -575,7 +575,7 @@ public class TestWalAndCompactingMemStoreFlush {
     MemStoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize();
     long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region)
         .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
-    long totalMemstoreSizePhaseV = region.getMemStoreSize();
+    long totalMemstoreSizePhaseV = region.getMemStoreDataSize();
 
     /*------------------------------------------------------------------------------*/
     /* PHASE V - validation */
@@ -663,7 +663,7 @@ public class TestWalAndCompactingMemStoreFlush {
     ((CompactingMemStore) ((HStore) region.getStore(FAMILY1)).memstore).setCompositeSnapshot(false);
     ((CompactingMemStore) ((HStore) region.getStore(FAMILY3)).memstore).setCompositeSnapshot(false);
 
-    long totalMemstoreSize = region.getMemStoreSize();
+    long totalMemstoreSize = region.getMemStoreDataSize();
 
     // Find the sizes of the memstores of each CF.
     MemStoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
@@ -794,7 +794,7 @@ public class TestWalAndCompactingMemStoreFlush {
       region.put(createPut(2, i));
     }
 
-    long totalMemstoreSize = region.getMemStoreSize();
+    long totalMemstoreSize = region.getMemStoreDataSize();
 
     // test in-memory flashing into CAM here
     ((CompactingMemStore) ((HStore)region.getStore(FAMILY1)).memstore).setIndexType(


[27/50] [abbrv] hbase git commit: HBASE-19953 Ensure post DDL hooks are only called after successful operations

Posted by zh...@apache.org.
HBASE-19953 Ensure post DDL hooks are only called after successful operations

The 1.x functionality of Master DDL operations is that "post" observer hooks
are only invoked when the DDL action was successful. With the async-ness of
ProcV2, we find ourselves in a case where the post-hook may be invoked before
the Procedure runs and fails. We need to introduce some blocking to wait and
see if the Procedure is going to fail on a precondition before invoking the hook.

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

Branch: refs/heads/HBASE-19064
Commit: d9b8dcc1d300ae114febc22dbc71866088387111
Parents: a27ef55
Author: Josh Elser <el...@apache.org>
Authored: Thu Feb 15 18:00:09 2018 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Feb 21 14:02:49 2018 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/master/ClusterSchema.java      |  10 +-
 .../hbase/master/ClusterSchemaServiceImpl.java  |  16 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  43 ++-
 .../hbase/master/TableNamespaceManager.java     |   3 +-
 .../AbstractStateMachineNamespaceProcedure.java |  13 +
 .../procedure/CreateNamespaceProcedure.java     |  22 +-
 .../procedure/DeleteNamespaceProcedure.java     |  34 +-
 .../procedure/ModifyNamespaceProcedure.java     |  30 +-
 .../master/procedure/ProcedurePrepareLatch.java |  14 +
 .../procedure/TestMasterObserverPostCalls.java  | 368 +++++++++++++++++++
 10 files changed, 518 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
index 746f9ad..56a1f33 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
@@ -22,6 +22,7 @@ import java.io.InterruptedIOException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.NonceKey;
 
@@ -79,32 +80,35 @@ public interface ClusterSchema {
    * Create a new Namespace.
    * @param namespaceDescriptor descriptor for new Namespace
    * @param nonceKey A unique identifier for this operation from the client or process.
+   * @param latch A latch to block on for precondition validation
    * @return procedure id
    * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
    *    as well as {@link IOException}
    */
-  long createNamespace(NamespaceDescriptor namespaceDescriptor, NonceKey nonceKey)
+  long createNamespace(NamespaceDescriptor namespaceDescriptor, NonceKey nonceKey, ProcedurePrepareLatch latch)
   throws IOException;
 
   /**
    * Modify an existing Namespace.
    * @param nonceKey A unique identifier for this operation from the client or process.
+   * @param latch A latch to block on for precondition validation
    * @return procedure id
    * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
    *    as well as {@link IOException}
    */
-  long modifyNamespace(NamespaceDescriptor descriptor, NonceKey nonceKey)
+  long modifyNamespace(NamespaceDescriptor descriptor, NonceKey nonceKey, ProcedurePrepareLatch latch)
   throws IOException;
 
   /**
    * Delete an existing Namespace.
    * Only empty Namespaces (no tables) can be removed.
    * @param nonceKey A unique identifier for this operation from the client or process.
+   * @param latch A latch to block on for precondition validation
    * @return procedure id
    * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
    *    as well as {@link IOException}
    */
-  long deleteNamespace(String name, NonceKey nonceKey)
+  long deleteNamespace(String name, NonceKey nonceKey, ProcedurePrepareLatch latch)
   throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
index 4dd8de0..7ad5b56 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AbstractService;
@@ -88,26 +89,27 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS
   }
 
   @Override
-  public long createNamespace(NamespaceDescriptor namespaceDescriptor, final NonceKey nonceKey)
+  public long createNamespace(NamespaceDescriptor namespaceDescriptor, final NonceKey nonceKey,
+      final ProcedurePrepareLatch latch)
       throws IOException {
     return submitProcedure(new CreateNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor, latch),
         nonceKey);
   }
 
   @Override
-  public long modifyNamespace(NamespaceDescriptor namespaceDescriptor, final NonceKey nonceKey)
-      throws IOException {
+  public long modifyNamespace(NamespaceDescriptor namespaceDescriptor, final NonceKey nonceKey,
+      final ProcedurePrepareLatch latch) throws IOException {
     return submitProcedure(new ModifyNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor, latch),
         nonceKey);
   }
 
   @Override
-  public long deleteNamespace(String name, final NonceKey nonceKey)
+  public long deleteNamespace(String name, final NonceKey nonceKey, final ProcedurePrepareLatch latch)
       throws IOException {
     return submitProcedure(new DeleteNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), name),
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), name, latch),
       nonceKey);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/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 b639503..818a0ed 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
@@ -117,6 +117,7 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
@@ -124,6 +125,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
+import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.RecoverMetaProcedure;
@@ -1743,7 +1745,10 @@ public class HMaster extends HRegionServer implements MasterServices {
 
         // TODO: We can handle/merge duplicate requests, and differentiate the case of
         //       TableExistsException by saying if the schema is the same or not.
-        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
+        //
+        // We need to wait for the procedure to potentially fail due to "prepare" sanity
+        // checks. This will block only the beginning of the procedure. See HBASE-19953.
+        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
         submitProcedure(new CreateTableProcedure(
             procedureExecutor.getEnvironment(), tableDescriptor, newRegions, latch));
         latch.await();
@@ -2089,7 +2094,10 @@ public class HMaster extends HRegionServer implements MasterServices {
         LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
 
         // TODO: We can handle/merge duplicate request
-        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
+        //
+        // We need to wait for the procedure to potentially fail due to "prepare" sanity
+        // checks. This will block only the beginning of the procedure. See HBASE-19953.
+        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
         submitProcedure(new DeleteTableProcedure(procedureExecutor.getEnvironment(),
             tableName, latch));
         latch.await();
@@ -2276,7 +2284,10 @@ public class HMaster extends HRegionServer implements MasterServices {
         // we want to make sure that the table is prepared to be
         // enabled (the table is locked and the table state is set).
         // Note: if the procedure throws exception, we will catch it and rethrow.
-        final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
+        //
+        // We need to wait for the procedure to potentially fail due to "prepare" sanity
+        // checks. This will block only the beginning of the procedure. See HBASE-19953.
+        final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createBlockingLatch();
         submitProcedure(new DisableTableProcedure(procedureExecutor.getEnvironment(),
             tableName, false, prepareLatch));
         prepareLatch.await();
@@ -2339,7 +2350,10 @@ public class HMaster extends HRegionServer implements MasterServices {
         LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
 
         // Execute the operation synchronously - wait for the operation completes before continuing.
-        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
+        //
+        // We need to wait for the procedure to potentially fail due to "prepare" sanity
+        // checks. This will block only the beginning of the procedure. See HBASE-19953.
+        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
         submitProcedure(new ModifyTableProcedure(procedureExecutor.getEnvironment(),
             descriptor, latch));
         latch.await();
@@ -2931,10 +2945,14 @@ public class HMaster extends HRegionServer implements MasterServices {
       @Override
       protected void run() throws IOException {
         getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor);
+        // We need to wait for the procedure to potentially fail due to "prepare" sanity
+        // checks. This will block only the beginning of the procedure. See HBASE-19953.
+        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
         LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
         // Execute the operation synchronously - wait for the operation to complete before
         // continuing.
-        setProcId(getClusterSchema().createNamespace(namespaceDescriptor, getNonceKey()));
+        setProcId(getClusterSchema().createNamespace(namespaceDescriptor, getNonceKey(), latch));
+        latch.await();
         getMaster().getMasterCoprocessorHost().postCreateNamespace(namespaceDescriptor);
       }
 
@@ -2963,10 +2981,14 @@ public class HMaster extends HRegionServer implements MasterServices {
       @Override
       protected void run() throws IOException {
         getMaster().getMasterCoprocessorHost().preModifyNamespace(namespaceDescriptor);
+        // We need to wait for the procedure to potentially fail due to "prepare" sanity
+        // checks. This will block only the beginning of the procedure. See HBASE-19953.
+        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
         LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
         // Execute the operation synchronously - wait for the operation to complete before
         // continuing.
-        setProcId(getClusterSchema().modifyNamespace(namespaceDescriptor, getNonceKey()));
+        setProcId(getClusterSchema().modifyNamespace(namespaceDescriptor, getNonceKey(), latch));
+        latch.await();
         getMaster().getMasterCoprocessorHost().postModifyNamespace(namespaceDescriptor);
       }
 
@@ -2996,7 +3018,14 @@ public class HMaster extends HRegionServer implements MasterServices {
         LOG.info(getClientIdAuditPrefix() + " delete " + name);
         // Execute the operation synchronously - wait for the operation to complete before
         // continuing.
-        setProcId(getClusterSchema().deleteNamespace(name, getNonceKey()));
+        //
+        // We need to wait for the procedure to potentially fail due to "prepare" sanity
+        // checks. This will block only the beginning of the procedure. See HBASE-19953.
+        ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
+        setProcId(submitProcedure(
+              new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name, latch)));
+        latch.await();
+        // Will not be invoked in the face of Exception thrown by the Procedure's execution
         getMaster().getMasterCoprocessorHost().postDeleteNamespace(name);
       }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index 47b27f4..f334d32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -228,7 +229,7 @@ public class TableNamespaceManager implements Stoppable {
   private void blockingCreateNamespace(final NamespaceDescriptor namespaceDescriptor)
       throws IOException {
     ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
-    long procId = clusterSchema.createNamespace(namespaceDescriptor, null);
+    long procId = clusterSchema.createNamespace(namespaceDescriptor, null, ProcedurePrepareLatch.getNoopLatch());
     block(this.masterServices, procId);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
index 2c6ae34..574706a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
@@ -31,12 +31,21 @@ public abstract class AbstractStateMachineNamespaceProcedure<TState>
     extends StateMachineProcedure<MasterProcedureEnv, TState>
     implements TableProcedureInterface {
 
+  private final ProcedurePrepareLatch syncLatch;
+
   protected AbstractStateMachineNamespaceProcedure() {
     // Required by the Procedure framework to create the procedure on replay
+    syncLatch = null;
   }
 
   protected AbstractStateMachineNamespaceProcedure(final MasterProcedureEnv env) {
+    this(env, null);
+  }
+
+  protected AbstractStateMachineNamespaceProcedure(final MasterProcedureEnv env,
+      final ProcedurePrepareLatch latch) {
     this.setOwner(env.getRequestUser());
+    this.syncLatch = latch;
   }
 
   protected abstract String getNamespaceName();
@@ -69,4 +78,8 @@ public abstract class AbstractStateMachineNamespaceProcedure<TState>
   protected void releaseLock(final MasterProcedureEnv env) {
     env.getProcedureScheduler().wakeNamespaceExclusiveLock(this, getNamespaceName());
   }
+
+  protected void releaseSyncLatch() {
+    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index 5eb56f5..c63f420 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -50,7 +50,12 @@ public class CreateNamespaceProcedure
 
   public CreateNamespaceProcedure(final MasterProcedureEnv env,
       final NamespaceDescriptor nsDescriptor) {
-    super(env);
+    this(env, nsDescriptor, null);
+  }
+
+  public CreateNamespaceProcedure(final MasterProcedureEnv env,
+      final NamespaceDescriptor nsDescriptor, ProcedurePrepareLatch latch) {
+    super(env, latch);
     this.nsDescriptor = nsDescriptor;
     this.traceEnabled = null;
   }
@@ -64,7 +69,12 @@ public class CreateNamespaceProcedure
     try {
       switch (state) {
       case CREATE_NAMESPACE_PREPARE:
-        prepareCreate(env);
+        boolean success = prepareCreate(env);
+        releaseSyncLatch();
+        if (!success) {
+          assert isFailed() : "createNamespace should have an exception here";
+          return Flow.NO_MORE_STATE;
+        }
         setNextState(CreateNamespaceState.CREATE_NAMESPACE_CREATE_DIRECTORY);
         break;
       case CREATE_NAMESPACE_CREATE_DIRECTORY:
@@ -102,6 +112,7 @@ public class CreateNamespaceProcedure
     if (state == CreateNamespaceState.CREATE_NAMESPACE_PREPARE) {
       // nothing to rollback, pre-create is just state checks.
       // TODO: coprocessor rollback semantic is still undefined.
+      releaseSyncLatch();
       return;
     }
     // The procedure doesn't have a rollback. The execution will succeed, at some point.
@@ -190,11 +201,14 @@ public class CreateNamespaceProcedure
    * @param env MasterProcedureEnv
    * @throws IOException
    */
-  private void prepareCreate(final MasterProcedureEnv env) throws IOException {
+  private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
     if (getTableNamespaceManager(env).doesNamespaceExist(nsDescriptor.getName())) {
-      throw new NamespaceExistException("Namespace " + nsDescriptor.getName() + " already exists");
+      setFailure("master-create-namespace",
+          new NamespaceExistException("Namespace " + nsDescriptor.getName() + " already exists"));
+      return false;
     }
     getTableNamespaceManager(env).validateTableAndRegionCount(nsDescriptor);
+    return true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
index 1c587eb..5f7959e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -57,7 +57,12 @@ public class DeleteNamespaceProcedure
   }
 
   public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName) {
-    super(env);
+    this(env, namespaceName, null);
+  }
+
+  public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName,
+      final ProcedurePrepareLatch latch) {
+    super(env, latch);
     this.namespaceName = namespaceName;
     this.nsDescriptor = null;
     this.traceEnabled = null;
@@ -74,7 +79,12 @@ public class DeleteNamespaceProcedure
     try {
       switch (state) {
       case DELETE_NAMESPACE_PREPARE:
-        prepareDelete(env);
+        boolean present = prepareDelete(env);
+        releaseSyncLatch();
+        if (!present) {
+          assert isFailed() : "Delete namespace should have an exception here";
+          return Flow.NO_MORE_STATE;
+        }
         setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_FROM_NS_TABLE);
         break;
       case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE:
@@ -113,6 +123,7 @@ public class DeleteNamespaceProcedure
       // nothing to rollback, pre is just table-state checks.
       // We can fail if the table does not exist or is not disabled.
       // TODO: coprocessor rollback semantic is still undefined.
+      releaseSyncLatch();
       return;
     }
 
@@ -188,27 +199,34 @@ public class DeleteNamespaceProcedure
    * @param env MasterProcedureEnv
    * @throws IOException
    */
-  private void prepareDelete(final MasterProcedureEnv env) throws IOException {
+  private boolean prepareDelete(final MasterProcedureEnv env) throws IOException {
     if (getTableNamespaceManager(env).doesNamespaceExist(namespaceName) == false) {
-      throw new NamespaceNotFoundException(namespaceName);
+      setFailure("master-delete-namespace", new NamespaceNotFoundException(namespaceName));
+      return false;
     }
     if (NamespaceDescriptor.RESERVED_NAMESPACES.contains(namespaceName)) {
-      throw new ConstraintException("Reserved namespace "+ namespaceName +" cannot be removed.");
+      setFailure("master-delete-namespace", new ConstraintException(
+          "Reserved namespace "+ namespaceName +" cannot be removed."));
+      return false;
     }
 
     int tableCount = 0;
     try {
       tableCount = env.getMasterServices().listTableDescriptorsByNamespace(namespaceName).size();
     } catch (FileNotFoundException fnfe) {
-      throw new NamespaceNotFoundException(namespaceName);
+      setFailure("master-delete-namespace", new NamespaceNotFoundException(namespaceName));
+      return false;
     }
     if (tableCount > 0) {
-      throw new ConstraintException("Only empty namespaces can be removed. " +
-          "Namespace "+ namespaceName + " has "+ tableCount +" tables");
+      setFailure("master-delete-namespace", new ConstraintException(
+          "Only empty namespaces can be removed. Namespace "+ namespaceName + " has "
+          + tableCount +" tables"));
+      return false;
     }
 
     // This is used for rollback
     nsDescriptor = getTableNamespaceManager(env).get(namespaceName);
+    return true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
index 2a7dc5b..d3f982f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,7 +51,12 @@ public class ModifyNamespaceProcedure
 
   public ModifyNamespaceProcedure(final MasterProcedureEnv env,
       final NamespaceDescriptor newNsDescriptor) {
-    super(env);
+    this(env, newNsDescriptor, null);
+  }
+
+  public ModifyNamespaceProcedure(final MasterProcedureEnv env,
+      final NamespaceDescriptor newNsDescriptor, final ProcedurePrepareLatch latch) {
+    super(env, latch);
     this.oldNsDescriptor = null;
     this.newNsDescriptor = newNsDescriptor;
     this.traceEnabled = null;
@@ -66,7 +72,12 @@ public class ModifyNamespaceProcedure
     try {
       switch (state) {
       case MODIFY_NAMESPACE_PREPARE:
-        prepareModify(env);
+        boolean success = prepareModify(env);
+        releaseSyncLatch();
+        if (!success) {
+          assert isFailed() : "Modify namespace should have an exception here";
+          return Flow.NO_MORE_STATE;
+        }
         setNextState(ModifyNamespaceState.MODIFY_NAMESPACE_UPDATE_NS_TABLE);
         break;
       case MODIFY_NAMESPACE_UPDATE_NS_TABLE:
@@ -96,6 +107,7 @@ public class ModifyNamespaceProcedure
     if (state == ModifyNamespaceState.MODIFY_NAMESPACE_PREPARE) {
       // nothing to rollback, pre-modify is just checks.
       // TODO: coprocessor rollback semantic is still undefined.
+      releaseSyncLatch();
       return;
     }
 
@@ -173,14 +185,22 @@ public class ModifyNamespaceProcedure
    * @param env MasterProcedureEnv
    * @throws IOException
    */
-  private void prepareModify(final MasterProcedureEnv env) throws IOException {
+  private boolean prepareModify(final MasterProcedureEnv env) throws IOException {
     if (getTableNamespaceManager(env).doesNamespaceExist(newNsDescriptor.getName()) == false) {
-      throw new NamespaceNotFoundException(newNsDescriptor.getName());
+      setFailure("master-modify-namespace", new NamespaceNotFoundException(
+            newNsDescriptor.getName()));
+      return false;
+    }
+    try {
+      getTableNamespaceManager(env).validateTableAndRegionCount(newNsDescriptor);
+    } catch (ConstraintException e) {
+      setFailure("master-modify-namespace", e);
+      return false;
     }
-    getTableNamespaceManager(env).validateTableAndRegionCount(newNsDescriptor);
 
     // This is used for rollback
     oldNsDescriptor = getTableNamespaceManager(env).get(newNsDescriptor.getName());
+    return true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
index f572cef..2011c0b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
@@ -57,6 +57,20 @@ public abstract class ProcedurePrepareLatch {
     return hasProcedureSupport(major, minor) ? noopLatch : new CompatibilityLatch();
   }
 
+  /**
+   * Creates a latch which blocks.
+   */
+  public static ProcedurePrepareLatch createBlockingLatch() {
+    return new CompatibilityLatch();
+  }
+
+  /**
+   * Returns the singleton latch which does nothing.
+   */
+  public static ProcedurePrepareLatch getNoopLatch() {
+    return noopLatch;
+  }
+
   private static boolean hasProcedureSupport(int major, int minor) {
     return VersionInfoUtil.currentClientHasMinimumVersion(major, minor);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterObserverPostCalls.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterObserverPostCalls.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterObserverPostCalls.java
new file mode 100644
index 0000000..65033a3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterObserverPostCalls.java
@@ -0,0 +1,368 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.procedure;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests class that validates that "post" observer hook methods are only invoked when the operation was successful.
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestMasterObserverPostCalls {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterObserverPostCalls.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestMasterObserverPostCalls.class);
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(1);
+  }
+
+  private static void setupConf(Configuration conf) {
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+    conf.set(MasterCoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+        MasterObserverForTest.class.getName());
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  public static class MasterObserverForTest implements MasterCoprocessor, MasterObserver {
+    private AtomicInteger postHookCalls = null;
+
+    @Override
+    public Optional<MasterObserver> getMasterObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void start(@SuppressWarnings("rawtypes") CoprocessorEnvironment ctx) throws IOException {
+      this.postHookCalls = new AtomicInteger(0);
+    }
+
+    @Override
+    public void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        String namespace) {
+      postHookCalls.incrementAndGet();
+    }
+
+    @Override
+    public void postModifyNamespace(
+        ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor desc) {
+      postHookCalls.incrementAndGet();
+    }
+
+    @Override
+    public void postCreateNamespace(
+        ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor desc) {
+      postHookCalls.incrementAndGet();
+    }
+
+    @Override
+    public void postCreateTable(
+        ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor td,
+        RegionInfo[] regions) {
+      postHookCalls.incrementAndGet();
+    }
+
+    @Override
+    public void postModifyTable(
+        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tn, TableDescriptor td) {
+      postHookCalls.incrementAndGet();
+    }
+
+    @Override
+    public void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tn) {
+      postHookCalls.incrementAndGet();
+    }
+
+    @Override
+    public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tn) {
+      postHookCalls.incrementAndGet();
+    }
+  }
+
+  @Test
+  public void testPostDeleteNamespace() throws IOException {
+    final Admin admin = UTIL.getAdmin();
+    final String ns = "postdeletens";
+    final TableName tn1 = TableName.valueOf(ns, "table1");
+
+    admin.createNamespace(NamespaceDescriptor.create(ns).build());
+    admin.createTable(TableDescriptorBuilder.newBuilder(tn1)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f1")).build())
+        .build());
+
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterObserverForTest observer = master.getMasterCoprocessorHost().findCoprocessor(
+        MasterObserverForTest.class);
+    int preCount = observer.postHookCalls.get();
+    try {
+      admin.deleteNamespace(ns);
+      fail("Deleting a non-empty namespace should be disallowed");
+    } catch (IOException e) {
+      // Pass
+    }
+    int postCount = observer.postHookCalls.get();
+    assertEquals("Expected no invocations of postDeleteNamespace when the operation fails",
+        preCount, postCount);
+
+    // Disable and delete the table so that we can delete the NS.
+    admin.disableTable(tn1);
+    admin.deleteTable(tn1);
+
+    // Validate that the postDeletNS hook is invoked
+    preCount = observer.postHookCalls.get();
+    admin.deleteNamespace(ns);
+    postCount = observer.postHookCalls.get();
+    assertEquals("Expected 1 invocation of postDeleteNamespace", preCount + 1, postCount);
+  }
+
+  @Test
+  public void testPostModifyNamespace() throws IOException {
+    final Admin admin = UTIL.getAdmin();
+    final String ns = "postmodifyns";
+
+    NamespaceDescriptor nsDesc = NamespaceDescriptor.create(ns).build();
+    admin.createNamespace(nsDesc);
+
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterObserverForTest observer = master.getMasterCoprocessorHost().findCoprocessor(
+        MasterObserverForTest.class);
+    int preCount = observer.postHookCalls.get();
+    try {
+      admin.modifyNamespace(NamespaceDescriptor.create("nonexistent").build());
+      fail("Modifying a missing namespace should fail");
+    } catch (IOException e) {
+      // Pass
+    }
+    int postCount = observer.postHookCalls.get();
+    assertEquals("Expected no invocations of postModifyNamespace when the operation fails",
+        preCount, postCount);
+
+    // Validate that the postDeletNS hook is invoked
+    preCount = observer.postHookCalls.get();
+    admin.modifyNamespace(
+        NamespaceDescriptor.create(nsDesc).addConfiguration("foo", "bar").build());
+    postCount = observer.postHookCalls.get();
+    assertEquals("Expected 1 invocation of postModifyNamespace", preCount + 1, postCount);
+  }
+
+  @Test
+  public void testPostCreateNamespace() throws IOException {
+    final Admin admin = UTIL.getAdmin();
+    final String ns = "postcreatens";
+
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterObserverForTest observer = master.getMasterCoprocessorHost().findCoprocessor(
+        MasterObserverForTest.class);
+
+    // Validate that the post hook is called
+    int preCount = observer.postHookCalls.get();
+    NamespaceDescriptor nsDesc = NamespaceDescriptor.create(ns).build();
+    admin.createNamespace(nsDesc);
+    int postCount = observer.postHookCalls.get();
+    assertEquals("Expected 1 invocation of postModifyNamespace", preCount + 1, postCount);
+
+    // Then, validate that it's not called when the call fails
+    preCount = observer.postHookCalls.get();
+    try {
+      admin.createNamespace(nsDesc);
+      fail("Creating an already present namespace should fail");
+    } catch (IOException e) {
+      // Pass
+    }
+    postCount = observer.postHookCalls.get();
+    assertEquals("Expected no invocations of postModifyNamespace when the operation fails",
+        preCount, postCount);
+  }
+
+  @Test
+  public void testPostCreateTable() throws IOException {
+    final Admin admin = UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("postcreatetable");
+    final TableDescriptor td = TableDescriptorBuilder.newBuilder(tn).addColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f1")).build()).build();
+
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterObserverForTest observer = master.getMasterCoprocessorHost().findCoprocessor(
+        MasterObserverForTest.class);
+
+    // Validate that the post hook is called
+    int preCount = observer.postHookCalls.get();
+    admin.createTable(td);
+    int postCount = observer.postHookCalls.get();
+    assertEquals("Expected 1 invocation of postCreateTable", preCount + 1, postCount);
+
+    // Then, validate that it's not called when the call fails
+    preCount = observer.postHookCalls.get();
+    try {
+      admin.createTable(td);
+      fail("Creating an already present table should fail");
+    } catch (IOException e) {
+      // Pass
+    }
+    postCount = observer.postHookCalls.get();
+    assertEquals("Expected no invocations of postCreateTable when the operation fails",
+        preCount, postCount);
+  }
+
+  @Test
+  public void testPostModifyTable() throws IOException {
+    final Admin admin = UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("postmodifytable");
+    final TableDescriptor td = TableDescriptorBuilder.newBuilder(tn).addColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f1")).build()).build();
+
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterObserverForTest observer = master.getMasterCoprocessorHost().findCoprocessor(
+        MasterObserverForTest.class);
+
+    // Create the table
+    admin.createTable(td);
+
+    // Validate that the post hook is called
+    int preCount = observer.postHookCalls.get();
+    admin.modifyTable(td);
+    int postCount = observer.postHookCalls.get();
+    assertEquals("Expected 1 invocation of postModifyTable", preCount + 1, postCount);
+
+    // Then, validate that it's not called when the call fails
+    preCount = observer.postHookCalls.get();
+    try {
+      admin.modifyTable(TableDescriptorBuilder.newBuilder(TableName.valueOf("missing"))
+          .addColumnFamily(td.getColumnFamily(Bytes.toBytes("f1"))).build());
+      fail("Modifying a missing table should fail");
+    } catch (IOException e) {
+      // Pass
+    }
+    postCount = observer.postHookCalls.get();
+    assertEquals("Expected no invocations of postModifyTable when the operation fails",
+        preCount, postCount);
+  }
+
+  @Test
+  public void testPostDisableTable() throws IOException {
+    final Admin admin = UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("postdisabletable");
+    final TableDescriptor td = TableDescriptorBuilder.newBuilder(tn).addColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f1")).build()).build();
+
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterObserverForTest observer = master.getMasterCoprocessorHost().findCoprocessor(
+        MasterObserverForTest.class);
+
+    // Create the table and disable it
+    admin.createTable(td);
+
+    // Validate that the post hook is called
+    int preCount = observer.postHookCalls.get();
+    admin.disableTable(td.getTableName());
+    int postCount = observer.postHookCalls.get();
+    assertEquals("Expected 1 invocation of postDisableTable", preCount + 1, postCount);
+
+    // Then, validate that it's not called when the call fails
+    preCount = observer.postHookCalls.get();
+    try {
+      admin.disableTable(TableName.valueOf("Missing"));
+      fail("Disabling a missing table should fail");
+    } catch (IOException e) {
+      // Pass
+    }
+    postCount = observer.postHookCalls.get();
+    assertEquals("Expected no invocations of postDisableTable when the operation fails",
+        preCount, postCount);
+  }
+
+  @Test
+  public void testPostDeleteTable() throws IOException {
+    final Admin admin = UTIL.getAdmin();
+    final TableName tn = TableName.valueOf("postdeletetable");
+    final TableDescriptor td = TableDescriptorBuilder.newBuilder(tn).addColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f1")).build()).build();
+
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterObserverForTest observer = master.getMasterCoprocessorHost().findCoprocessor(
+        MasterObserverForTest.class);
+
+    // Create the table and disable it
+    admin.createTable(td);
+    admin.disableTable(td.getTableName());
+
+    // Validate that the post hook is called
+    int preCount = observer.postHookCalls.get();
+    admin.deleteTable(td.getTableName());
+    int postCount = observer.postHookCalls.get();
+    assertEquals("Expected 1 invocation of postDeleteTable", preCount + 1, postCount);
+
+    // Then, validate that it's not called when the call fails
+    preCount = observer.postHookCalls.get();
+    try {
+      admin.deleteTable(TableName.valueOf("missing"));
+      fail("Deleting a missing table should fail");
+    } catch (IOException e) {
+      // Pass
+    }
+    postCount = observer.postHookCalls.get();
+    assertEquals("Expected no invocations of postDeleteTable when the operation fails",
+        preCount, postCount);
+  }
+}


[37/50] [abbrv] hbase git commit: HBASE-20038 TestLockProcedure.testTimeout is flakey

Posted by zh...@apache.org.
HBASE-20038 TestLockProcedure.testTimeout is flakey


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

Branch: refs/heads/HBASE-19064
Commit: b328807d25ec0d2537371dd51d8ad79c841c3cec
Parents: a090085
Author: zhangduo <zh...@apache.org>
Authored: Wed Feb 21 21:20:24 2018 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Feb 21 17:24:19 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/locking/TestLockProcedure.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b328807d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
index 85b00d0..c985fa7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java
@@ -284,7 +284,7 @@ public class TestLockProcedure {
     sendHeartbeatAndCheckLocked(procId, true);
     Thread.sleep(HEARTBEAT_TIMEOUT / 2);
     sendHeartbeatAndCheckLocked(procId, true);
-    Thread.sleep(2 * HEARTBEAT_TIMEOUT);
+    Thread.sleep(4 * HEARTBEAT_TIMEOUT);
     sendHeartbeatAndCheckLocked(procId, false);
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);


[24/50] [abbrv] hbase git commit: HBASE-20017 BufferedMutatorImpl submit the same mutation repeatedly

Posted by zh...@apache.org.
HBASE-20017 BufferedMutatorImpl submit the same mutation repeatedly

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


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

Branch: refs/heads/HBASE-19064
Commit: 79d9403a79cca60e614834659e3d9005d5482cac
Parents: 0068b95
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Sun Feb 18 21:45:04 2018 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Feb 20 16:59:48 2018 -0800

----------------------------------------------------------------------
 .../hbase/client/BufferedMutatorImpl.java       | 47 +++++++++++++++-----
 .../hadoop/hbase/client/TestAsyncProcess.java   | 44 ++++++++++++++++++
 .../hbase/regionserver/wal/ReaderBase.java      |  2 +-
 3 files changed, 81 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/79d9403a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index 9d24b4d..d4bc811 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -295,7 +295,7 @@ public class BufferedMutatorImpl implements BufferedMutator {
         break;
       }
       AsyncRequestFuture asf;
-      try (QueueRowAccess access = new QueueRowAccess()) {
+      try (QueueRowAccess access = createQueueRowAccess()) {
         if (access.isEmpty()) {
           // It means someone has gotten the ticker to run the flush.
           break;
@@ -406,16 +406,46 @@ public class BufferedMutatorImpl implements BufferedMutator {
     return currentWriteBufferSize.get();
   }
 
+  /**
+   * Count the mutations which haven't been processed.
+   * @return count of undealt mutation
+   */
   @VisibleForTesting
   int size() {
     return undealtMutationCount.get();
   }
 
-  private class QueueRowAccess implements RowAccess<Row>, Closeable {
+  /**
+   * Count the mutations which haven't been flushed
+   * @return count of unflushed mutation
+   */
+  @VisibleForTesting
+  int getUnflushedSize() {
+    return writeAsyncBuffer.size();
+  }
+
+  @VisibleForTesting
+  QueueRowAccess createQueueRowAccess() {
+    return new QueueRowAccess();
+  }
+
+  @VisibleForTesting
+  class QueueRowAccess implements RowAccess<Row>, Closeable {
     private int remainder = undealtMutationCount.getAndSet(0);
+    private Mutation last = null;
+
+    private void restoreLastMutation() {
+      // restore the last mutation since it isn't submitted
+      if (last != null) {
+        writeAsyncBuffer.add(last);
+        currentWriteBufferSize.addAndGet(last.heapSize());
+        last = null;
+      }
+    }
 
     @Override
     public void close() {
+      restoreLastMutation();
       if (remainder > 0) {
         undealtMutationCount.addAndGet(remainder);
         remainder = 0;
@@ -425,25 +455,22 @@ public class BufferedMutatorImpl implements BufferedMutator {
     @Override
     public Iterator<Row> iterator() {
       return new Iterator<Row>() {
-        private final Iterator<Mutation> iter = writeAsyncBuffer.iterator();
         private int countDown = remainder;
-        private Mutation last = null;
         @Override
         public boolean hasNext() {
-          if (countDown <= 0) {
-            return false;
-          }
-          return iter.hasNext();
+          return countDown > 0;
         }
         @Override
         public Row next() {
+          restoreLastMutation();
           if (!hasNext()) {
             throw new NoSuchElementException();
           }
-          last = iter.next();
+          last = writeAsyncBuffer.poll();
           if (last == null) {
             throw new NoSuchElementException();
           }
+          currentWriteBufferSize.addAndGet(-last.heapSize());
           --countDown;
           return last;
         }
@@ -452,8 +479,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
           if (last == null) {
             throw new IllegalStateException();
           }
-          iter.remove();
-          currentWriteBufferSize.addAndGet(-last.heapSize());
           --remainder;
           last = null;
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/79d9403a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 2979dcd..4a2ed8d 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -1795,4 +1795,48 @@ public class TestAsyncProcess {
     LOG.debug("Expected to sleep " + expectedSleep + "ms, actually slept " + actualSleep + "ms");
     Assert.assertTrue("Slept for too long: " + actualSleep + "ms", actualSleep <= expectedSleep);
   }
+
+  @Test
+  public void testQueueRowAccess() throws Exception {
+    ClusterConnection conn = createHConnection();
+    BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null,
+      new BufferedMutatorParams(DUMMY_TABLE).writeBufferSize(100000));
+    Put p0 = new Put(DUMMY_BYTES_1).addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1);
+    Put p1 = new Put(DUMMY_BYTES_2).addColumn(DUMMY_BYTES_2, DUMMY_BYTES_2, DUMMY_BYTES_2);
+    mutator.mutate(p0);
+    BufferedMutatorImpl.QueueRowAccess ra0 = mutator.createQueueRowAccess();
+    // QueueRowAccess should take all undealt mutations
+    assertEquals(0, mutator.size());
+    mutator.mutate(p1);
+    assertEquals(1, mutator.size());
+    BufferedMutatorImpl.QueueRowAccess ra1 = mutator.createQueueRowAccess();
+    // QueueRowAccess should take all undealt mutations
+    assertEquals(0, mutator.size());
+    assertEquals(1, ra0.size());
+    assertEquals(1, ra1.size());
+    Iterator<Row> iter0 = ra0.iterator();
+    Iterator<Row> iter1 = ra1.iterator();
+    assertTrue(iter0.hasNext());
+    assertTrue(iter1.hasNext());
+    // the next() will poll the mutation from inner buffer and update the buffer count
+    assertTrue(iter0.next() == p0);
+    assertEquals(1, mutator.getUnflushedSize());
+    assertEquals(p1.heapSize(), mutator.getCurrentWriteBufferSize());
+    assertTrue(iter1.next() == p1);
+    assertEquals(0, mutator.getUnflushedSize());
+    assertEquals(0, mutator.getCurrentWriteBufferSize());
+    assertFalse(iter0.hasNext());
+    assertFalse(iter1.hasNext());
+    // ra0 doest handle the mutation so the mutation won't be pushed back to buffer
+    iter0.remove();
+    ra0.close();
+    assertEquals(0, mutator.size());
+    assertEquals(0, mutator.getUnflushedSize());
+    assertEquals(0, mutator.getCurrentWriteBufferSize());
+    // ra1 doesn't handle the mutation so the mutation will be pushed back to buffer
+    ra1.close();
+    assertEquals(1, mutator.size());
+    assertEquals(1, mutator.getUnflushedSize());
+    assertEquals(p1.heapSize(), mutator.getCurrentWriteBufferSize());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/79d9403a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
index f242cef..4338f6d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
@@ -139,7 +139,7 @@ public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
    * Initializes the compression after the shared stuff has been initialized. Called once.
    */
   protected abstract void initAfterCompression() throws IOException;
-  
+
   /**
    * Initializes the compression after the shared stuff has been initialized. Called once.
    * @param cellCodecClsName class name of cell Codec


[07/50] [abbrv] hbase git commit: HBASE-20011 Disable TestRestoreSnapshotFromClientWithRegionReplicas; it is flakey. Needs attention.

Posted by zh...@apache.org.
HBASE-20011 Disable TestRestoreSnapshotFromClientWithRegionReplicas; it is flakey. Needs attention.


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

Branch: refs/heads/HBASE-19064
Commit: abf7de702cb1851e70eef4afb849469a322102fa
Parents: 0d6acfa
Author: Michael Stack <st...@apache.org>
Authored: Fri Feb 16 14:45:55 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Feb 16 14:46:24 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/regionserver/HRegion.java  | 4 ----
 .../client/TestRestoreSnapshotFromClientWithRegionReplicas.java  | 2 ++
 2 files changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/abf7de70/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 6170888..9464fdb 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
@@ -892,10 +892,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     if (this.getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
       status.setStatus("Writing region info on filesystem");
       fs.checkRegionInfoOnFilesystem();
-    } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Skipping creation of .regioninfo file for " + this.getRegionInfo());
-      }
     }
 
     // Initialize all the HStores

http://git-wip-us.apache.org/repos/asf/hbase/blob/abf7de70/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java
index d301098..839542a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java
@@ -25,10 +25,12 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.ClassRule;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 
+@Ignore // Disabled because flakey. See HBASE-20006.
 @Category({LargeTests.class, ClientTests.class})
 public class TestRestoreSnapshotFromClientWithRegionReplicas extends
     TestRestoreSnapshotFromClient {


[49/50] [abbrv] 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/fb268839
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/fb268839
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/fb268839

Branch: refs/heads/HBASE-19064
Commit: fb268839f5392b4a20f192b4c784fa6b52da45f8
Parents: fbab0eb
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 26 16:50:48 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 2018 +0800

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


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

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb268839/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/fb268839/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 044f38e..228d67b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -1878,10 +1878,9 @@ public final class RequestConverter {
   }
 
   public static TransitReplicationPeerSyncReplicationStateRequest
-    buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
-      SyncReplicationState state) {
+      buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
+          SyncReplicationState state) {
     return TransitReplicationPeerSyncReplicationStateRequest.newBuilder().setPeerId(peerId)
-      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
-      .build();
+        .setSyncReplicationState(ReplicationPeerConfigUtil.toSyncReplicationState(state)).build();
   }
 }

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb268839/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
index 338ce3f..909daa0 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
@@ -17,12 +17,12 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 
 /**
@@ -96,7 +95,7 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
       ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
         enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES),
       ZKUtilOp.createAndFailSilent(getSyncReplicationStateNode(peerId),
-        Bytes.toBytes(syncReplicationState.ordinal())));
+        SyncReplicationState.toByteArray(syncReplicationState)));
     try {
       ZKUtil.createWithParents(zookeeper, peersZNode);
       ZKUtil.multiOrSequential(zookeeper, multiOps, false);
@@ -179,29 +178,27 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
   }
 
   @Override
-  public void setPeerSyncReplicationState(String peerId, SyncReplicationState clusterState)
+  public void setPeerSyncReplicationState(String peerId, SyncReplicationState state)
       throws ReplicationException {
-    byte[] clusterStateBytes = Bytes.toBytes(clusterState.ordinal());
     try {
-      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId), clusterStateBytes);
+      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId),
+        SyncReplicationState.toByteArray(state));
     } catch (KeeperException e) {
       throw new ReplicationException(
-          "Unable to change the cluster state for the synchronous replication peer with id=" +
-              peerId,
-          e);
+        "Unable to change the cluster state for the synchronous replication peer with id=" + peerId,
+        e);
     }
   }
 
   @Override
   public SyncReplicationState getPeerSyncReplicationState(String peerId)
       throws ReplicationException {
-    byte[] data;
     try {
-      data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
-    } catch (KeeperException | InterruptedException e) {
+      byte[] data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
+      return SyncReplicationState.parseFrom(data);
+    } catch (KeeperException | InterruptedException | IOException e) {
       throw new ReplicationException(
-          "Error getting cluster state for the synchronous replication peer with id=" + peerId, e);
+        "Error getting cluster state for the synchronous replication peer with id=" + peerId, e);
     }
-    return SyncReplicationState.valueOf(Bytes.toInt(data));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb268839/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 f22765e..d05f83a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -83,7 +83,6 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
@@ -1942,13 +1941,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/fb268839/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/fb268839/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 1204c9b..3d7cdaf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -174,7 +174,7 @@ public abstract class TestReplicationSourceManager {
       ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/sync-rep-state");
     ZKUtil.setData(zkw, "/hbase/replication/peers/1/sync-rep-state",
-      Bytes.toBytes(SyncReplicationState.NONE.ordinal()));
+      SyncReplicationState.toByteArray(SyncReplicationState.NONE));
     ZKUtil.createWithParents(zkw, "/hbase/replication/state");
     ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
 


[35/50] [abbrv] hbase git commit: HBASE-20042 TestRegionServerAbort flakey; ADDENDUM, RETRY

Posted by zh...@apache.org.
HBASE-20042 TestRegionServerAbort flakey; ADDENDUM, RETRY


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

Branch: refs/heads/HBASE-19064
Commit: 13223c217ca6cb84a96f3c70b8c38ec19eca729f
Parents: 61b5516
Author: Michael Stack <st...@apache.org>
Authored: Wed Feb 21 15:26:21 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Feb 21 15:26:21 2018 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/TestRegionServerAbort.java  | 17 +++--------------
 1 file changed, 3 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/13223c21/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
index 0c778fb..d5cbad0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java
@@ -144,21 +144,10 @@ public class TestRegionServerAbort {
     put.addColumn(FAMILY_BYTES, Bytes.toBytes("c"), new byte[]{});
     put.setAttribute(StopBlockingRegionObserver.DO_ABORT, new byte[]{1});
 
-    table.put(put);
-    // should have triggered an abort due to FileNotFoundException
-
-    // verify that the regionserver is stopped
-    List<HRegion> regions = null;
-    do {
-      regions = cluster.findRegionsForTable(tableName);
-      if (regions != null && regions.size() > 0) {
-        break;
-      }
-      LOG.warn("Waiting on regions for {} to online");
-      Threads.sleep(100);
-    } while(true);
-
+    List<HRegion> regions = cluster.findRegionsForTable(tableName);
     HRegion firstRegion = cluster.findRegionsForTable(tableName).get(0);
+    table.put(put);
+    // Verify that the regionserver is stopped
     assertNotNull(firstRegion);
     assertNotNull(firstRegion.getRegionServerServices());
     LOG.info("isAborted = " + firstRegion.getRegionServerServices().isAborted());


[16/50] [abbrv] hbase git commit: HBASE-20020 Make sure we throw DoNotRetryIOException when ConnectionImplementation is closed

Posted by zh...@apache.org.
HBASE-20020 Make sure we throw DoNotRetryIOException when ConnectionImplementation is closed


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

Branch: refs/heads/HBASE-19064
Commit: b7685307e4cc72035fc1b8737faec2e06a7c97df
Parents: 391790d
Author: zhangduo <zh...@apache.org>
Authored: Tue Feb 20 16:50:03 2018 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Feb 20 06:12:13 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/MasterNotRunningException.java |  4 +-
 .../hadoop/hbase/client/ClusterConnection.java  | 30 ++-----
 .../hbase/client/ConnectionImplementation.java  | 86 +++++++++-----------
 .../hadoop/hbase/client/ConnectionUtils.java    |  8 +-
 .../org/apache/hadoop/hbase/client/HTable.java  |  5 +-
 .../hadoop/hbase/client/MasterCallable.java     | 11 ++-
 .../hbase/client/TestSnapshotFromAdmin.java     |  4 +-
 .../hbase/client/TestHBaseAdminNoCluster.java   |  4 +-
 8 files changed, 59 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
index 1ff17ac..35cdecb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
@@ -18,15 +18,13 @@
  */
 package org.apache.hadoop.hbase;
 
-import java.io.IOException;
-
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Thrown if the master is not running
  */
 @InterfaceAudience.Public
-public class MasterNotRunningException extends IOException {
+public class MasterNotRunningException extends HBaseIOException {
   private static final long serialVersionUID = (1L << 23) - 1L;
   /** default constructor */
   public MasterNotRunningException() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
index 7294559..3e055b0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MasterNotRunningException;
@@ -29,12 +28,12 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
 
 /** Internal methods on Connection that should not be used by user code. */
 @InterfaceAudience.Private
@@ -224,7 +223,7 @@ public interface ClusterConnection extends Connection {
   /**
    * Returns a {@link MasterKeepAliveConnection} to the active master
    */
-  MasterService.BlockingInterface getMaster() throws IOException;
+  MasterKeepAliveConnection getMaster() throws IOException;
 
   /**
    * Get the admin service for master.
@@ -258,9 +257,8 @@ public interface ClusterConnection extends Connection {
    * @return Location of row.
    * @throws IOException if a remote or network exception occurs
    */
-  HRegionLocation getRegionLocation(TableName tableName, byte [] row,
-    boolean reload)
-  throws IOException;
+  HRegionLocation getRegionLocation(TableName tableName, byte[] row, boolean reload)
+      throws IOException;
 
   /**
    * Clear any caches that pertain to server name <code>sn</code>.
@@ -269,24 +267,6 @@ public interface ClusterConnection extends Connection {
   void clearCaches(final ServerName sn);
 
   /**
-   * This function allows HBaseAdmin and potentially others to get a shared MasterService
-   * connection.
-   * @return The shared instance. Never returns null.
-   * @throws MasterNotRunningException if master is not running
-   * @deprecated Since 0.96.0
-   */
-  @Deprecated
-  MasterKeepAliveConnection getKeepAliveMasterService()
-  throws MasterNotRunningException;
-
-  /**
-   * @param serverName of server to check
-   * @return true if the server is known as dead, false otherwise.
-   * @deprecated internal method, do not use thru ClusterConnection */
-  @Deprecated
-  boolean isDeadServer(ServerName serverName);
-
-  /**
    * @return Nonce generator for this ClusterConnection; may be null if disabled in configuration.
    */
   NonceGenerator getNonceGenerator();

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/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 a5a0188..6408044 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
@@ -538,6 +538,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return this.conf;
   }
 
+  private void checkClosed() throws DoNotRetryIOException {
+    if (this.closed) {
+      throw new DoNotRetryIOException(toString() + " closed");
+    }
+  }
+
   /**
    * @return true if the master is running, throws an exception otherwise
    * @throws org.apache.hadoop.hbase.MasterNotRunningException - if the master is not running
@@ -545,21 +551,24 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
    */
   @Deprecated
   @Override
-  public boolean isMasterRunning()
-  throws MasterNotRunningException, ZooKeeperConnectionException {
+  public boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException {
     // When getting the master connection, we check it's running,
     // so if there is no exception, it means we've been able to get a
     // connection on a running master
-    MasterKeepAliveConnection m = getKeepAliveMasterService();
+    MasterKeepAliveConnection m;
+    try {
+      m = getKeepAliveMasterService();
+    } catch (IOException e) {
+      throw new MasterNotRunningException(e);
+    }
     m.close();
     return true;
   }
 
   @Override
-  public HRegionLocation getRegionLocation(final TableName tableName,
-      final byte [] row, boolean reload)
-  throws IOException {
-    return reload? relocateRegion(tableName, row): locateRegion(tableName, row);
+  public HRegionLocation getRegionLocation(final TableName tableName, final byte[] row,
+      boolean reload) throws IOException {
+    return reload ? relocateRegion(tableName, row) : locateRegion(tableName, row);
   }
 
 
@@ -576,9 +585,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   @Override
   public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys)
       throws IOException {
-    if (this.closed) {
-      throw new IOException(toString() + " closed");
-    }
+    checkClosed();
     try {
       if (!isTableEnabled(tableName)) {
         LOG.debug("Table " + tableName + " not enabled");
@@ -641,8 +648,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return locations == null ? null : locations.getRegionLocation();
   }
 
-  @Override
-  public boolean isDeadServer(ServerName sn) {
+  private boolean isDeadServer(ServerName sn) {
     if (clusterStatusListener == null) {
       return false;
     } else {
@@ -679,19 +685,19 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   }
 
   @Override
-  public HRegionLocation locateRegion(
-      final TableName tableName, final byte[] row) throws IOException{
+  public HRegionLocation locateRegion(final TableName tableName, final byte[] row)
+      throws IOException {
     RegionLocations locations = locateRegion(tableName, row, true, true);
     return locations == null ? null : locations.getRegionLocation();
   }
 
   @Override
-  public HRegionLocation relocateRegion(final TableName tableName,
-      final byte [] row) throws IOException{
-    RegionLocations locations =  relocateRegion(tableName, row,
-      RegionReplicaUtil.DEFAULT_REPLICA_ID);
-    return locations == null ? null :
-      locations.getRegionLocation(RegionReplicaUtil.DEFAULT_REPLICA_ID);
+  public HRegionLocation relocateRegion(final TableName tableName, final byte[] row)
+      throws IOException {
+    RegionLocations locations =
+      relocateRegion(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID);
+    return locations == null ? null
+      : locations.getRegionLocation(RegionReplicaUtil.DEFAULT_REPLICA_ID);
   }
 
   @Override
@@ -708,22 +714,17 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   }
 
   @Override
-  public RegionLocations locateRegion(final TableName tableName,
-    final byte [] row, boolean useCache, boolean retry)
-  throws IOException {
+  public RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache,
+      boolean retry) throws IOException {
     return locateRegion(tableName, row, useCache, retry, RegionReplicaUtil.DEFAULT_REPLICA_ID);
   }
 
   @Override
-  public RegionLocations locateRegion(final TableName tableName,
-    final byte [] row, boolean useCache, boolean retry, int replicaId)
-  throws IOException {
-    if (this.closed) {
-      throw new DoNotRetryIOException(toString() + " closed");
-    }
-    if (tableName== null || tableName.getName().length == 0) {
-      throw new IllegalArgumentException(
-          "table name cannot be null or zero length");
+  public RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache,
+      boolean retry, int replicaId) throws IOException {
+    checkClosed();
+    if (tableName == null || tableName.getName().length == 0) {
+      throw new IllegalArgumentException("table name cannot be null or zero length");
     }
     if (tableName.equals(TableName.META_TABLE_NAME)) {
       return locateMeta(tableName, useCache, replicaId);
@@ -1170,6 +1171,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   @Override
   public AdminProtos.AdminService.BlockingInterface getAdmin(ServerName serverName)
       throws IOException {
+    checkClosed();
     if (isDeadServer(serverName)) {
       throw new RegionServerStoppedException(serverName + " is dead.");
     }
@@ -1184,6 +1186,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
 
   @Override
   public BlockingInterface getClient(ServerName serverName) throws IOException {
+    checkClosed();
     if (isDeadServer(serverName)) {
       throw new RegionServerStoppedException(serverName + " is dead.");
     }
@@ -1199,7 +1202,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   final MasterServiceState masterServiceState = new MasterServiceState(this);
 
   @Override
-  public MasterProtos.MasterService.BlockingInterface getMaster() throws MasterNotRunningException {
+  public MasterKeepAliveConnection getMaster() throws IOException {
     return getKeepAliveMasterService();
   }
 
@@ -1207,20 +1210,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     mss.userCount++;
   }
 
-  @Override
-  public MasterKeepAliveConnection getKeepAliveMasterService()
-  throws MasterNotRunningException {
+  private MasterKeepAliveConnection getKeepAliveMasterService() throws IOException {
     synchronized (masterLock) {
       if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) {
         MasterServiceStubMaker stubMaker = new MasterServiceStubMaker();
-        try {
-          this.masterServiceState.stub = stubMaker.makeStub();
-        } catch (MasterNotRunningException ex) {
-          throw ex;
-        } catch (IOException e) {
-          // rethrow as MasterNotRunningException so that we can keep the method sig
-          throw new MasterNotRunningException(e);
-        }
+        this.masterServiceState.stub = stubMaker.makeStub();
       }
       resetMasterServiceState(this.masterServiceState);
     }
@@ -1955,9 +1949,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
 
   @Override
   public TableState getTableState(TableName tableName) throws IOException {
-    if (this.closed) {
-      throw new IOException(toString() + " closed");
-    }
+    checkClosed();
     TableState tableState = MetaTableAccessor.getTableState(this, tableName);
     if (tableState == null) {
       throw new TableNotFoundException(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index c9e994f..63ef865 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -31,12 +31,10 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -51,9 +49,11 @@ import org.apache.hadoop.net.DNS;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
@@ -155,11 +155,11 @@ public final class ConnectionUtils {
     }
 
     @Override
-    public MasterKeepAliveConnection getKeepAliveMasterService() throws MasterNotRunningException {
+    public MasterKeepAliveConnection getMaster() throws IOException {
       if (this.localHostClient instanceof MasterService.BlockingInterface) {
         return new ShortCircuitMasterConnection((MasterService.BlockingInterface)this.localHostClient);
       }
-      return super.getKeepAliveMasterService();
+      return super.getMaster();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index a4289e9..1a11979 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -162,10 +162,7 @@ public class HTable implements Table {
       final RpcRetryingCallerFactory rpcCallerFactory,
       final RpcControllerFactory rpcControllerFactory,
       final ExecutorService pool) {
-    if (connection == null || connection.isClosed()) {
-      throw new IllegalArgumentException("Connection is null or closed.");
-    }
-    this.connection = connection;
+    this.connection = Preconditions.checkNotNull(connection, "connection is null");
     this.configuration = connection.getConfiguration();
     this.connConfiguration = connection.getConnectionConfiguration();
     if (pool == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
index f56ebd5..7ae9731 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
@@ -20,15 +20,14 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.Closeable;
 import java.io.IOException;
-
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 /**
  * A RetryingCallable for Master RPC operations.
  * Implement the #rpcCall method. It will be retried on error. See its javadoc and the javadoc of
@@ -55,7 +54,7 @@ abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
 
   @Override
   public void prepare(boolean reload) throws IOException {
-    this.master = this.connection.getKeepAliveMasterService();
+    this.master = this.connection.getMaster();
   }
 
   @Override
@@ -139,7 +138,7 @@ abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
   }
 
   private static boolean isMetaRegion(final byte[] regionName) {
-    return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
-        || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
+    return Bytes.equals(regionName, RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
+      Bytes.equals(regionName, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
index 387e9dd..dba1a36 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
@@ -92,7 +92,7 @@ public class TestSnapshotFromAdmin {
     // mock the master admin to our mock
     MasterKeepAliveConnection mockMaster = Mockito.mock(MasterKeepAliveConnection.class);
     Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
-    Mockito.when(mockConnection.getKeepAliveMasterService()).thenReturn(mockMaster);
+    Mockito.when(mockConnection.getMaster()).thenReturn(mockMaster);
     // we need a real retrying caller
     RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
     RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
@@ -163,7 +163,7 @@ public class TestSnapshotFromAdmin {
 
     // mock the master connection
     MasterKeepAliveConnection master = Mockito.mock(MasterKeepAliveConnection.class);
-    Mockito.when(mockConnection.getKeepAliveMasterService()).thenReturn(master);
+    Mockito.when(mockConnection.getMaster()).thenReturn(master);
     SnapshotResponse response = SnapshotResponse.newBuilder().setExpectedTimeout(0).build();
     Mockito.when(
       master.snapshot((RpcController) Mockito.any(), Mockito.any()))

http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
index bab0fe9..a1026a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
@@ -102,7 +102,7 @@ public class TestHBaseAdminNoCluster {
     Mockito.when(masterAdmin.createTable((RpcController)Mockito.any(),
       (CreateTableRequest)Mockito.any())).
         thenThrow(new ServiceException("Test fail").initCause(new PleaseHoldException("test")));
-    Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin);
+    Mockito.when(connection.getMaster()).thenReturn(masterAdmin);
     Admin admin = new HBaseAdmin(connection);
     try {
       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
@@ -304,7 +304,7 @@ public class TestHBaseAdminNoCluster {
             throw new MasterNotRunningException(); // all methods will throw an exception
           }
         });
-    Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin);
+    Mockito.when(connection.getMaster()).thenReturn(masterAdmin);
     RpcControllerFactory rpcControllerFactory = Mockito.mock(RpcControllerFactory.class);
     Mockito.when(connection.getRpcControllerFactory()).thenReturn(rpcControllerFactory);
     Mockito.when(rpcControllerFactory.newController()).thenReturn(


[38/50] [abbrv] hbase git commit: Revert "HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs" Applied prematurely.

Posted by zh...@apache.org.
Revert "HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs"
Applied prematurely.

This reverts commit 61b55166bf7fe9edc4e8105f217463ed6e693d17.


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

Branch: refs/heads/HBASE-19064
Commit: 2440f807bf7d077def819c616d4afa97a4e2539e
Parents: b328807
Author: Michael Stack <st...@apache.org>
Authored: Wed Feb 21 18:02:25 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Feb 21 18:02:25 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegionServer.java    |  2 +-
 .../apache/hadoop/hbase/regionserver/HStore.java    |  4 ++--
 .../compactions/CompactionProgress.java             | 16 ++++------------
 .../hbase/regionserver/TestMajorCompaction.java     |  2 +-
 4 files changed, 8 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/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 a76dec2..49b7b80 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
@@ -1623,7 +1623,7 @@ public class HRegionServer extends HasThread implements
       storefileIndexSizeKB += store.getStorefilesRootLevelIndexSize() / 1024;
       CompactionProgress progress = store.getCompactionProgress();
       if (progress != null) {
-        totalCompactingKVs += progress.getTotalCompactingKVs();
+        totalCompactingKVs += progress.totalCompactingKVs;
         currentCompactedKVs += progress.currentCompactedKVs;
       }
       rootLevelIndexSizeKB += (int) (store.getStorefilesRootLevelIndexSize() / 1024);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index c0ef3ef..bef50b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1373,10 +1373,10 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
     writeCompactionWalRecord(filesToCompact, sfs);
     replaceStoreFiles(filesToCompact, sfs);
     if (cr.isMajor()) {
-      majorCompactedCellsCount += getCompactionProgress().getTotalCompactingKVs();
+      majorCompactedCellsCount += getCompactionProgress().totalCompactingKVs;
       majorCompactedCellsSize += getCompactionProgress().totalCompactedSize;
     } else {
-      compactedCellsCount += getCompactionProgress().getTotalCompactingKVs();
+      compactedCellsCount += getCompactionProgress().totalCompactingKVs;
       compactedCellsSize += getCompactionProgress().totalCompactedSize;
     }
     long outputBytes = getTotalSize(sfs);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
index 577276e..d40651a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
@@ -20,8 +20,6 @@
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * This class holds information relevant for tracking the progress of a
@@ -34,10 +32,9 @@ import org.slf4j.LoggerFactory;
  */
 @InterfaceAudience.Private
 public class CompactionProgress {
-  private static final Logger LOG = LoggerFactory.getLogger(CompactionProgress.class);
 
   /** the total compacting key values in currently running compaction */
-  private long totalCompactingKVs;
+  public long totalCompactingKVs;
   /** the completed count of key values in currently running compaction */
   public long currentCompactedKVs = 0;
   /** the total size of data processed by the currently running compaction, in bytes */
@@ -54,7 +51,7 @@ public class CompactionProgress {
    * @return float
    */
   public float getProgressPct() {
-    return (float)currentCompactedKVs / getTotalCompactingKVs();
+    return (float)currentCompactedKVs / totalCompactingKVs;
   }
 
   /**
@@ -75,12 +72,7 @@ public class CompactionProgress {
   /**
    * @return the total compacting key values in currently running compaction
    */
-  public long getTotalCompactingKVs() {
-    if (totalCompactingKVs < currentCompactedKVs) {
-      LOG.warn("totalCompactingKVs={} less than currentCompactedKVs={}",
-          totalCompactingKVs, currentCompactedKVs);
-      return currentCompactedKVs;
-    }
+  public long getTotalCompactingKvs() {
     return totalCompactingKVs;
   }
 
@@ -100,7 +92,7 @@ public class CompactionProgress {
 
   @Override
   public String toString() {
-    return String.format("%d/%d (%.2f%%)", currentCompactedKVs, getTotalCompactingKVs(),
+    return String.format("%d/%d (%.2f%%)", currentCompactedKVs, totalCompactingKVs,
       100 * getProgressPct());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
index 4fdd6b3..ee717f9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
@@ -221,7 +221,7 @@ public class TestMajorCompaction {
       if( progress != null ) {
         ++storeCount;
         assertTrue(progress.currentCompactedKVs > 0);
-        assertTrue(progress.getTotalCompactingKVs() > 0);
+        assertTrue(progress.totalCompactingKVs > 0);
       }
       assertTrue(storeCount > 0);
     }


[15/50] [abbrv] hbase git commit: HBASE-19978 The keepalive logic is incomplete in ProcedureExecutor

Posted by zh...@apache.org.
HBASE-19978 The keepalive logic is incomplete in ProcedureExecutor


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

Branch: refs/heads/HBASE-19064
Commit: 391790ddb05f0db0ffd67e0c2f2f961157af7be3
Parents: a9a6eed
Author: zhangduo <zh...@apache.org>
Authored: Mon Feb 19 08:08:06 2018 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Feb 19 17:13:47 2018 -0800

----------------------------------------------------------------------
 .../hbase/procedure2/DelayedProcedure.java      |  28 ++
 .../hadoop/hbase/procedure2/InlineChore.java    |  42 +++
 .../hbase/procedure2/ProcedureExecutor.java     | 284 +++++--------------
 .../hbase/procedure2/StoppableThread.java       |  54 ++++
 .../hbase/procedure2/TimeoutExecutorThread.java | 140 +++++++++
 .../org/apache/hadoop/hbase/master/HMaster.java |  11 +-
 .../master/procedure/TestProcedurePriority.java | 180 ++++++++++++
 7 files changed, 522 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java
new file mode 100644
index 0000000..fcec0b7
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2;
+
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+class DelayedProcedure extends DelayedUtil.DelayedContainerWithTimestamp<Procedure<?>> {
+  public DelayedProcedure(Procedure<?> procedure) {
+    super(procedure, procedure.getTimeoutTimestamp());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/InlineChore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/InlineChore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/InlineChore.java
new file mode 100644
index 0000000..32b4922
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/InlineChore.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2;
+
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Inline Chores (executors internal chores).
+ */
+@InterfaceAudience.Private
+abstract class InlineChore extends DelayedUtil.DelayedObject implements Runnable {
+
+  private long timeout;
+
+  public abstract int getTimeoutInterval();
+
+  protected void refreshTimeout() {
+    this.timeout = EnvironmentEdgeManager.currentTime() + getTimeoutInterval();
+  }
+
+  @Override
+  public long getTimeout() {
+    return timeout;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index b1ff426..af0a61b 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -18,9 +18,6 @@
 
 package org.apache.hadoop.hbase.procedure2;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -30,35 +27,35 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.DelayQueue;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.procedure2.Procedure.LockState;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
-import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
-import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedWithTimeout;
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.hbase.util.Threads;
+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.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 
 /**
  * Thread Pool that executes the submitted procedures.
@@ -83,7 +80,7 @@ public class ProcedureExecutor<TEnvironment> {
 
   public static final String WORKER_KEEP_ALIVE_TIME_CONF_KEY =
       "hbase.procedure.worker.keep.alive.time.msec";
-  private static final long DEFAULT_WORKER_KEEP_ALIVE_TIME = Long.MAX_VALUE;
+  private static final long DEFAULT_WORKER_KEEP_ALIVE_TIME = TimeUnit.MINUTES.toMillis(1);
 
   Testing testing = null;
   public static class Testing {
@@ -272,8 +269,9 @@ public class ProcedureExecutor<TEnvironment> {
   private CopyOnWriteArrayList<WorkerThread> workerThreads;
   private TimeoutExecutorThread timeoutExecutor;
   private int corePoolSize;
+  private int maxPoolSize;
 
-  private volatile long keepAliveTime = Long.MAX_VALUE;
+  private volatile long keepAliveTime;
 
   /**
    * Scheduler/Queue that contains runnable procedures.
@@ -501,7 +499,7 @@ public class ProcedureExecutor<TEnvironment> {
    *          a corrupted procedure is found on replay. otherwise false.
    */
   public void start(int numThreads, boolean abortOnCorruption) throws IOException {
-    if (running.getAndSet(true)) {
+    if (!running.compareAndSet(false, true)) {
       LOG.warn("Already running");
       return;
     }
@@ -509,13 +507,15 @@ public class ProcedureExecutor<TEnvironment> {
     // We have numThreads executor + one timer thread used for timing out
     // procedures and triggering periodic procedures.
     this.corePoolSize = numThreads;
-    LOG.info("Starting {} Workers (bigger of cpus/4 or 16)", corePoolSize);
+    this.maxPoolSize = 10 * numThreads;
+    LOG.info("Starting {} core workers (bigger of cpus/4 or 16) with max (burst) worker count={}",
+        corePoolSize, maxPoolSize);
 
     // Create the Thread Group for the executors
     threadGroup = new ThreadGroup("PEWorkerGroup");
 
     // Create the timeout executor
-    timeoutExecutor = new TimeoutExecutorThread(threadGroup);
+    timeoutExecutor = new TimeoutExecutorThread(this, threadGroup);
 
     // Create the workers
     workerId.set(0);
@@ -530,8 +530,8 @@ public class ProcedureExecutor<TEnvironment> {
     st = EnvironmentEdgeManager.currentTime();
     store.recoverLease();
     et = EnvironmentEdgeManager.currentTime();
-    LOG.info(String.format("Recovered %s lease in %s",
-      store.getClass().getSimpleName(), StringUtils.humanTimeDiff(et - st)));
+    LOG.info("Recovered {} lease in {}", store.getClass().getSimpleName(),
+      StringUtils.humanTimeDiff(et - st));
 
     // start the procedure scheduler
     scheduler.start();
@@ -544,13 +544,11 @@ public class ProcedureExecutor<TEnvironment> {
     st = EnvironmentEdgeManager.currentTime();
     load(abortOnCorruption);
     et = EnvironmentEdgeManager.currentTime();
-    LOG.info(String.format("Loaded %s in %s, um pid=",
-      store.getClass().getSimpleName(), StringUtils.humanTimeDiff(et - st)));
+    LOG.info("Loaded {} in {}", store.getClass().getSimpleName(),
+      StringUtils.humanTimeDiff(et - st));
 
     // Start the executors. Here we must have the lastProcId set.
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Start workers " + workerThreads.size());
-    }
+    LOG.trace("Start workers {}", workerThreads.size());
     timeoutExecutor.start();
     for (WorkerThread worker: workerThreads) {
       worker.start();
@@ -645,7 +643,7 @@ public class ProcedureExecutor<TEnvironment> {
     return this.store;
   }
 
-  protected ProcedureScheduler getScheduler() {
+  ProcedureScheduler getScheduler() {
     return scheduler;
   }
 
@@ -1152,7 +1150,7 @@ public class ProcedureExecutor<TEnvironment> {
     return procedures.keySet();
   }
 
-  private Long getRootProcedureId(Procedure proc) {
+  Long getRootProcedureId(Procedure proc) {
     return Procedure.getRootProcedureId(procedures, proc);
   }
 
@@ -1699,15 +1697,23 @@ public class ProcedureExecutor<TEnvironment> {
     sendProcedureFinishedNotification(proc.getProcId());
   }
 
+  RootProcedureState getProcStack(long rootProcId) {
+    return rollbackStack.get(rootProcId);
+  }
+
   // ==========================================================================
   //  Worker Thread
   // ==========================================================================
-  private final class WorkerThread extends StoppableThread {
+  private class WorkerThread extends StoppableThread {
     private final AtomicLong executionStartTime = new AtomicLong(Long.MAX_VALUE);
-    private Procedure activeProcedure;
+    private volatile Procedure<?> activeProcedure;
 
-    public WorkerThread(final ThreadGroup group) {
-      super(group, "PEWorker-" + workerId.incrementAndGet());
+    public WorkerThread(ThreadGroup group) {
+      this(group, "PEWorker-");
+    }
+
+    protected WorkerThread(ThreadGroup group, String prefix) {
+      super(group, prefix + workerId.incrementAndGet());
       setDaemon(true);
     }
 
@@ -1721,34 +1727,33 @@ public class ProcedureExecutor<TEnvironment> {
       long lastUpdate = EnvironmentEdgeManager.currentTime();
       try {
         while (isRunning() && keepAlive(lastUpdate)) {
-          this.activeProcedure = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
-          if (this.activeProcedure == null) continue;
+          Procedure<?> proc = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
+          if (proc == null) {
+            continue;
+          }
+          this.activeProcedure = proc;
           int activeCount = activeExecutorCount.incrementAndGet();
           int runningCount = store.setRunningProcedureCount(activeCount);
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Execute pid=" + this.activeProcedure.getProcId() +
-                " runningCount=" + runningCount + ", activeCount=" + activeCount);
-          }
+          LOG.trace("Execute pid={} runningCount={}, activeCount={}", proc.getProcId(),
+            runningCount, activeCount);
           executionStartTime.set(EnvironmentEdgeManager.currentTime());
           try {
-            executeProcedure(this.activeProcedure);
+            executeProcedure(proc);
           } catch (AssertionError e) {
-            LOG.info("ASSERT pid=" + this.activeProcedure.getProcId(), e);
+            LOG.info("ASSERT pid=" + proc.getProcId(), e);
             throw e;
           } finally {
             activeCount = activeExecutorCount.decrementAndGet();
             runningCount = store.setRunningProcedureCount(activeCount);
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Halt pid=" + this.activeProcedure.getProcId() +
-                  " runningCount=" + runningCount + ", activeCount=" + activeCount);
-            }
+            LOG.trace("Halt pid={} runningCount={}, activeCount={}", proc.getProcId(),
+              runningCount, activeCount);
             this.activeProcedure = null;
             lastUpdate = EnvironmentEdgeManager.currentTime();
             executionStartTime.set(Long.MAX_VALUE);
           }
         }
       } catch (Throwable t) {
-        LOG.warn("Worker terminating UNNATURALLY " + this.activeProcedure, t);
+        LOG.warn("Worker terminating UNNATURALLY {}", this.activeProcedure, t);
       } finally {
         LOG.trace("Worker terminated.");
       }
@@ -1768,169 +1773,23 @@ public class ProcedureExecutor<TEnvironment> {
       return EnvironmentEdgeManager.currentTime() - executionStartTime.get();
     }
 
-    private boolean keepAlive(final long lastUpdate) {
-      if (workerThreads.size() <= corePoolSize) return true;
-      return (EnvironmentEdgeManager.currentTime() - lastUpdate) < keepAliveTime;
+    // core worker never timeout
+    protected boolean keepAlive(long lastUpdate) {
+      return true;
     }
   }
 
-  /**
-   * Runs task on a period such as check for stuck workers.
-   * @see InlineChore
-   */
-  private final class TimeoutExecutorThread extends StoppableThread {
-    private final DelayQueue<DelayedWithTimeout> queue = new DelayQueue<>();
+  // A worker thread which can be added when core workers are stuck. Will timeout after
+  // keepAliveTime if there is no procedure to run.
+  private final class KeepAliveWorkerThread extends WorkerThread {
 
-    public TimeoutExecutorThread(final ThreadGroup group) {
-      super(group, "ProcExecTimeout");
-      setDaemon(true);
+    public KeepAliveWorkerThread(ThreadGroup group) {
+      super(group, "KeepAlivePEWorker-");
     }
 
     @Override
-    public void sendStopSignal() {
-      queue.add(DelayedUtil.DELAYED_POISON);
-    }
-
-    @Override
-    public void run() {
-      final boolean traceEnabled = LOG.isTraceEnabled();
-      while (isRunning()) {
-        final DelayedWithTimeout task = DelayedUtil.takeWithoutInterrupt(queue);
-        if (task == null || task == DelayedUtil.DELAYED_POISON) {
-          // the executor may be shutting down,
-          // and the task is just the shutdown request
-          continue;
-        }
-
-        if (traceEnabled) {
-          LOG.trace("Executing " + task);
-        }
-
-        // execute the task
-        if (task instanceof InlineChore) {
-          execInlineChore((InlineChore)task);
-        } else if (task instanceof DelayedProcedure) {
-          execDelayedProcedure((DelayedProcedure)task);
-        } else {
-          LOG.error("CODE-BUG unknown timeout task type " + task);
-        }
-      }
-    }
-
-    public void add(final InlineChore chore) {
-      chore.refreshTimeout();
-      queue.add(chore);
-    }
-
-    public void add(final Procedure procedure) {
-      assert procedure.getState() == ProcedureState.WAITING_TIMEOUT;
-      LOG.info("ADDED " + procedure + "; timeout=" + procedure.getTimeout() +
-          ", timestamp=" + procedure.getTimeoutTimestamp());
-      queue.add(new DelayedProcedure(procedure));
-    }
-
-    public boolean remove(final Procedure procedure) {
-      return queue.remove(new DelayedProcedure(procedure));
-    }
-
-    private void execInlineChore(final InlineChore chore) {
-      chore.run();
-      add(chore);
-    }
-
-    private void execDelayedProcedure(final DelayedProcedure delayed) {
-      // TODO: treat this as a normal procedure, add it to the scheduler and
-      // let one of the workers handle it.
-      // Today we consider ProcedureInMemoryChore as InlineChores
-      final Procedure procedure = delayed.getObject();
-      if (procedure instanceof ProcedureInMemoryChore) {
-        executeInMemoryChore((ProcedureInMemoryChore)procedure);
-        // if the procedure is in a waiting state again, put it back in the queue
-        procedure.updateTimestamp();
-        if (procedure.isWaiting()) {
-          delayed.setTimeout(procedure.getTimeoutTimestamp());
-          queue.add(delayed);
-        }
-      } else {
-        executeTimedoutProcedure(procedure);
-      }
-    }
-
-    private void executeInMemoryChore(final ProcedureInMemoryChore chore) {
-      if (!chore.isWaiting()) return;
-
-      // The ProcedureInMemoryChore is a special case, and it acts as a chore.
-      // instead of bringing the Chore class in, we reuse this timeout thread for
-      // this special case.
-      try {
-        chore.periodicExecute(getEnvironment());
-      } catch (Throwable e) {
-        LOG.error("Ignoring " + chore + " exception: " + e.getMessage(), e);
-      }
-    }
-
-    private void executeTimedoutProcedure(final Procedure proc) {
-      // The procedure received a timeout. if the procedure itself does not handle it,
-      // call abort() and add the procedure back in the queue for rollback.
-      if (proc.setTimeoutFailure(getEnvironment())) {
-        long rootProcId = Procedure.getRootProcedureId(procedures, proc);
-        RootProcedureState procStack = rollbackStack.get(rootProcId);
-        procStack.abort();
-        store.update(proc);
-        scheduler.addFront(proc);
-      }
-    }
-  }
-
-  private static final class DelayedProcedure
-      extends DelayedUtil.DelayedContainerWithTimestamp<Procedure> {
-    public DelayedProcedure(final Procedure procedure) {
-      super(procedure, procedure.getTimeoutTimestamp());
-    }
-  }
-
-  private static abstract class StoppableThread extends Thread {
-    public StoppableThread(final ThreadGroup group, final String name) {
-      super(group, name);
-    }
-
-    public abstract void sendStopSignal();
-
-    public void awaitTermination() {
-      try {
-        final long startTime = EnvironmentEdgeManager.currentTime();
-        for (int i = 0; isAlive(); ++i) {
-          sendStopSignal();
-          join(250);
-          // Log every two seconds; send interrupt too.
-          if (i > 0 && (i % 8) == 0) {
-            LOG.warn("Waiting termination of thread " + getName() + ", " +
-              StringUtils.humanTimeDiff(EnvironmentEdgeManager.currentTime() - startTime) +
-            "; sending interrupt");
-            interrupt();
-          }
-        }
-      } catch (InterruptedException e) {
-        LOG.warn(getName() + " join wait got interrupted", e);
-      }
-    }
-  }
-
-  // ==========================================================================
-  //  Inline Chores (executors internal chores)
-  // ==========================================================================
-  private static abstract class InlineChore extends DelayedUtil.DelayedObject implements Runnable {
-    private long timeout;
-
-    public abstract int getTimeoutInterval();
-
-    protected void refreshTimeout() {
-      this.timeout = EnvironmentEdgeManager.currentTime() + getTimeoutInterval();
-    }
-
-    @Override
-    public long getTimeout() {
-      return timeout;
+    protected boolean keepAlive(long lastUpdate) {
+      return EnvironmentEdgeManager.currentTime() - lastUpdate < keepAliveTime;
     }
   }
 
@@ -1975,32 +1834,35 @@ public class ProcedureExecutor<TEnvironment> {
     private int checkForStuckWorkers() {
       // check if any of the worker is stuck
       int stuckCount = 0;
-      for (WorkerThread worker: workerThreads) {
+      for (WorkerThread worker : workerThreads) {
         if (worker.getCurrentRunTime() < stuckThreshold) {
           continue;
         }
 
         // WARN the worker is stuck
         stuckCount++;
-        LOG.warn("Worker stuck " + worker +
-            " run time " + StringUtils.humanTimeDiff(worker.getCurrentRunTime()));
+        LOG.warn("Worker stuck {} run time {}", worker,
+          StringUtils.humanTimeDiff(worker.getCurrentRunTime()));
       }
       return stuckCount;
     }
 
     private void checkThreadCount(final int stuckCount) {
       // nothing to do if there are no runnable tasks
-      if (stuckCount < 1 || !scheduler.hasRunnables()) return;
+      if (stuckCount < 1 || !scheduler.hasRunnables()) {
+        return;
+      }
 
       // add a new thread if the worker stuck percentage exceed the threshold limit
       // and every handler is active.
-      final float stuckPerc = ((float)stuckCount) / workerThreads.size();
-      if (stuckPerc >= addWorkerStuckPercentage &&
-          activeExecutorCount.get() == workerThreads.size()) {
-        final WorkerThread worker = new WorkerThread(threadGroup);
+      final float stuckPerc = ((float) stuckCount) / workerThreads.size();
+      // let's add new worker thread more aggressively, as they will timeout finally if there is no
+      // work to do.
+      if (stuckPerc >= addWorkerStuckPercentage && workerThreads.size() < maxPoolSize) {
+        final KeepAliveWorkerThread worker = new KeepAliveWorkerThread(threadGroup);
         workerThreads.add(worker);
         worker.start();
-        LOG.debug("Added new worker thread " + worker);
+        LOG.debug("Added new worker thread {}", worker);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StoppableThread.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StoppableThread.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StoppableThread.java
new file mode 100644
index 0000000..b58b571
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StoppableThread.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2;
+
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+abstract class StoppableThread extends Thread {
+
+  private static final Logger LOG = LoggerFactory.getLogger(StoppableThread.class);
+
+  public StoppableThread(final ThreadGroup group, final String name) {
+    super(group, name);
+  }
+
+  public abstract void sendStopSignal();
+
+  public void awaitTermination() {
+    try {
+      final long startTime = EnvironmentEdgeManager.currentTime();
+      for (int i = 0; isAlive(); ++i) {
+        sendStopSignal();
+        join(250);
+        // Log every two seconds; send interrupt too.
+        if (i > 0 && (i % 8) == 0) {
+          LOG.warn("Waiting termination of thread {}, {}; sending interrupt", getName(),
+            StringUtils.humanTimeDiff(EnvironmentEdgeManager.currentTime() - startTime));
+          interrupt();
+        }
+      }
+    } catch (InterruptedException e) {
+      LOG.warn("{} join wait got interrupted", getName(), e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
new file mode 100644
index 0000000..e5e3230
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2;
+
+import java.util.concurrent.DelayQueue;
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedWithTimeout;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
+
+/**
+ * Runs task on a period such as check for stuck workers.
+ * @see InlineChore
+ */
+@InterfaceAudience.Private
+class TimeoutExecutorThread extends StoppableThread {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TimeoutExecutorThread.class);
+
+  private final ProcedureExecutor<?> executor;
+
+  private final DelayQueue<DelayedWithTimeout> queue = new DelayQueue<>();
+
+  public TimeoutExecutorThread(ProcedureExecutor<?> executor, ThreadGroup group) {
+    super(group, "ProcExecTimeout");
+    setDaemon(true);
+    this.executor = executor;
+  }
+
+  @Override
+  public void sendStopSignal() {
+    queue.add(DelayedUtil.DELAYED_POISON);
+  }
+
+  @Override
+  public void run() {
+    while (executor.isRunning()) {
+      final DelayedWithTimeout task = DelayedUtil.takeWithoutInterrupt(queue);
+      if (task == null || task == DelayedUtil.DELAYED_POISON) {
+        // the executor may be shutting down,
+        // and the task is just the shutdown request
+        continue;
+      }
+      LOG.trace("Executing {}", task);
+
+      // execute the task
+      if (task instanceof InlineChore) {
+        execInlineChore((InlineChore) task);
+      } else if (task instanceof DelayedProcedure) {
+        execDelayedProcedure((DelayedProcedure) task);
+      } else {
+        LOG.error("CODE-BUG unknown timeout task type {}", task);
+      }
+    }
+  }
+
+  public void add(InlineChore chore) {
+    chore.refreshTimeout();
+    queue.add(chore);
+  }
+
+  public void add(Procedure<?> procedure) {
+    assert procedure.getState() == ProcedureState.WAITING_TIMEOUT;
+    LOG.info("ADDED {}; timeout={}, timestamp={}", procedure, procedure.getTimeout(),
+      procedure.getTimeoutTimestamp());
+    queue.add(new DelayedProcedure(procedure));
+  }
+
+  public boolean remove(Procedure<?> procedure) {
+    return queue.remove(new DelayedProcedure(procedure));
+  }
+
+  private void execInlineChore(InlineChore chore) {
+    chore.run();
+    add(chore);
+  }
+
+  private void execDelayedProcedure(DelayedProcedure delayed) {
+    // TODO: treat this as a normal procedure, add it to the scheduler and
+    // let one of the workers handle it.
+    // Today we consider ProcedureInMemoryChore as InlineChores
+    Procedure<?> procedure = delayed.getObject();
+    if (procedure instanceof ProcedureInMemoryChore) {
+      executeInMemoryChore((ProcedureInMemoryChore) procedure);
+      // if the procedure is in a waiting state again, put it back in the queue
+      procedure.updateTimestamp();
+      if (procedure.isWaiting()) {
+        delayed.setTimeout(procedure.getTimeoutTimestamp());
+        queue.add(delayed);
+      }
+    } else {
+      executeTimedoutProcedure(procedure);
+    }
+  }
+
+  private void executeInMemoryChore(ProcedureInMemoryChore chore) {
+    if (!chore.isWaiting()) {
+      return;
+    }
+
+    // The ProcedureInMemoryChore is a special case, and it acts as a chore.
+    // instead of bringing the Chore class in, we reuse this timeout thread for
+    // this special case.
+    try {
+      chore.periodicExecute(executor.getEnvironment());
+    } catch (Throwable e) {
+      LOG.error("Ignoring {} exception: {}", chore, e.getMessage(), e);
+    }
+  }
+
+  private void executeTimedoutProcedure(Procedure proc) {
+    // The procedure received a timeout. if the procedure itself does not handle it,
+    // call abort() and add the procedure back in the queue for rollback.
+    if (proc.setTimeoutFailure(executor.getEnvironment())) {
+      long rootProcId = executor.getRootProcedureId(proc);
+      RootProcedureState procStack = executor.getProcStack(rootProcId);
+      procStack.abort();
+      executor.getStore().update(proc);
+      executor.getScheduler().addFront(proc);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/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 b949aa9..b639503 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
@@ -1213,11 +1213,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     configurationManager.registerObserver(procEnv);
 
     int cpus = Runtime.getRuntime().availableProcessors();
-    final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
-        Math.max((cpus > 0? cpus/4: 0),
-            MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
-    final boolean abortOnCorruption = conf.getBoolean(
-        MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
+    final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, Math.max(
+      (cpus > 0 ? cpus / 4 : 0), MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
+    final boolean abortOnCorruption =
+      conf.getBoolean(MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
         MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
     procedureStore.start(numThreads);
     procedureExecutor.start(numThreads, abortOnCorruption);
@@ -3522,7 +3521,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   public boolean recoverMeta() throws IOException {
     ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
     LOG.info("Running RecoverMetaProcedure to ensure proper hbase:meta deploy.");
-    long procId = procedureExecutor.submitProcedure(new RecoverMetaProcedure(null, true, latch));
+    procedureExecutor.submitProcedure(new RecoverMetaProcedure(null, true, latch));
     latch.await();
     LOG.info("hbase:meta deployed at=" +
         getMetaTableLocator().getMetaRegionLocation(getZooKeeper()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java
new file mode 100644
index 0000000..05d8976
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.hbase.Cell;
+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;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Durability;
+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.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+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.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test to ensure that the priority for procedures and stuck checker can partially solve the problem
+ * describe in HBASE-19976, that is, RecoverMetaProcedure can finally be executed within a certain
+ * period of time.
+ */
+@Category({ MasterTests.class, LargeTests.class })
+public class TestProcedurePriority {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestProcedurePriority.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String TABLE_NAME_PREFIX = "TestProcedurePriority-";
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  private static byte[] CQ = Bytes.toBytes("cq");
+
+  private static int CORE_POOL_SIZE;
+
+  private static int TABLE_COUNT;
+
+  private static volatile boolean FAIL = false;
+
+  public static final class MyCP implements RegionObserver, RegionCoprocessor {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
+        List<Cell> result) throws IOException {
+      if (FAIL && c.getEnvironment().getRegionInfo().isMetaRegion()) {
+        throw new IOException("Inject error");
+      }
+    }
+
+    @Override
+    public void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit,
+        Durability durability) throws IOException {
+      if (FAIL && c.getEnvironment().getRegionInfo().isMetaRegion()) {
+        throw new IOException("Inject error");
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setLong(ProcedureExecutor.WORKER_KEEP_ALIVE_TIME_CONF_KEY, 5000);
+    UTIL.getConfiguration().setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 4);
+    UTIL.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, MyCP.class.getName());
+    UTIL.startMiniCluster(3);
+    CORE_POOL_SIZE =
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getCorePoolSize();
+    TABLE_COUNT = 50 * CORE_POOL_SIZE;
+    List<Future<?>> futures = new ArrayList<>();
+    for (int i = 0; i < TABLE_COUNT; i++) {
+      futures.add(UTIL.getAdmin().createTableAsync(
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE_NAME_PREFIX + i))
+          .addColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build(),
+        null));
+    }
+    for (Future<?> future : futures) {
+      future.get(1, TimeUnit.MINUTES);
+    }
+    UTIL.getAdmin().balance(true);
+    UTIL.waitUntilNoRegionsInTransition();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    RegionServerThread rsWithMetaThread = UTIL.getMiniHBaseCluster().getRegionServerThreads()
+      .stream().filter(t -> !t.getRegionServer().getRegions(TableName.META_TABLE_NAME).isEmpty())
+      .findAny().get();
+    HRegionServer rsNoMeta = UTIL.getOtherRegionServer(rsWithMetaThread.getRegionServer());
+    FAIL = true;
+    UTIL.getMiniHBaseCluster().killRegionServer(rsNoMeta.getServerName());
+    // wait until all the worker thread are stuck, which means that the stuck checker will start to
+    // add new worker thread.
+    ProcedureExecutor<?> executor =
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    UTIL.waitFor(60000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return executor.getWorkerThreadCount() > CORE_POOL_SIZE;
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Stuck checker does not add new worker thread";
+      }
+    });
+    UTIL.getMiniHBaseCluster().killRegionServer(rsWithMetaThread.getRegionServer().getServerName());
+    rsWithMetaThread.join();
+    FAIL = false;
+    // verify that the cluster is back
+    UTIL.waitUntilNoRegionsInTransition(60000);
+    for (int i = 0; i < TABLE_COUNT; i++) {
+      try (Table table = UTIL.getConnection().getTable(TableName.valueOf(TABLE_NAME_PREFIX + i))) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return executor.getWorkerThreadCount() == CORE_POOL_SIZE;
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "The new workers do not timeout";
+      }
+    });
+  }
+}


[28/50] [abbrv] hbase git commit: HBASE-20031 Unable to run integration test using mvn due to missing HBaseClassTestRule

Posted by zh...@apache.org.
HBASE-20031 Unable to run integration test using mvn due to missing HBaseClassTestRule


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

Branch: refs/heads/HBASE-19064
Commit: 401227ba6aeb3c7767e88d41a7fa2990b3717648
Parents: d9b8dcc
Author: tedyu <yu...@gmail.com>
Authored: Wed Feb 21 12:03:32 2018 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Feb 21 12:03:32 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/HBaseClassTestRule.java  |  4 ++++
 .../apache/hadoop/hbase/HBaseClassTestRuleChecker.java    | 10 ++++++++++
 src/main/asciidoc/_chapters/developer.adoc                |  3 ++-
 3 files changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/401227ba/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
index a41e383..c3bef0f 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase;
 
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -63,6 +64,9 @@ public final class HBaseClassTestRule implements TestRule {
         // All tests have a 10minute timeout.
         return TimeUnit.MINUTES.toSeconds(13);
       }
+      if (c == IntegrationTests.class) {
+        return TimeUnit.MINUTES.toSeconds(Long.MAX_VALUE);
+      }
     }
     throw new IllegalArgumentException(
         clazz.getName() + " does not have SmallTests/MediumTests/LargeTests in @Category");

http://git-wip-us.apache.org/repos/asf/hbase/blob/401227ba/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java
index 97c657f..c374903 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java
@@ -22,8 +22,12 @@ import static org.junit.Assert.fail;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
 import org.junit.runner.Description;
 import org.junit.runner.notification.RunListener;
 import org.junit.runner.notification.RunListener.ThreadSafe;
@@ -37,6 +41,12 @@ public class HBaseClassTestRuleChecker extends RunListener {
 
   @Override
   public void testStarted(Description description) throws Exception {
+    Category[] categories = description.getTestClass().getAnnotationsByType(Category.class);
+    for (Class<?> c : categories[0].value()) {
+      if (c == IntegrationTests.class) {
+        return;
+      }
+    }
     for (Field field : description.getTestClass().getFields()) {
       if (Modifier.isStatic(field.getModifiers()) && field.getType() == HBaseClassTestRule.class &&
         field.isAnnotationPresent(ClassRule.class)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/401227ba/src/main/asciidoc/_chapters/developer.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc
index 3dbfe74..c752cd7 100644
--- a/src/main/asciidoc/_chapters/developer.adoc
+++ b/src/main/asciidoc/_chapters/developer.adoc
@@ -1014,7 +1014,7 @@ The first three categories, `small`, `medium`, and `large`, are for test cases w
 type `$ mvn test`.
 In other words, these three categorizations are for HBase unit tests.
 The `integration` category is not for unit tests, but for integration tests.
-These are run when you invoke `$ mvn verify`.
+These are normally run when you invoke `$ mvn verify`.
 Integration tests are described in <<integration.tests,integration.tests>>.
 
 Keep reading to figure which annotation of the set `small`, `medium`, and `large`
@@ -1041,6 +1041,7 @@ Large Tests (((LargeTests)))::
 Integration Tests (((IntegrationTests)))::
   _Integration_ tests are system level tests.
   See <<integration.tests,integration.tests>> for more info.
+  If you invoke `$ mvn test` on integration tests, there is no timeout for the test.
 
 [[hbase.unittests.cmds]]
 === Running tests


[17/50] [abbrv] hbase git commit: HBASE-20021 TestFromClientSideWithCoprocessor is flakey

Posted by zh...@apache.org.
HBASE-20021 TestFromClientSideWithCoprocessor is flakey


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

Branch: refs/heads/HBASE-19064
Commit: 69d2becc73deeb0217a816fe923d4bcd70f3d65f
Parents: b768530
Author: zhangduo <zh...@apache.org>
Authored: Tue Feb 20 20:07:27 2018 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Feb 20 06:24:42 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/TestFromClientSide.java    | 17 ++++++++++-------
 .../client/TestFromClientSideWithCoprocessor.java  | 17 +++++------------
 2 files changed, 15 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/69d2becc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index b028880..29d3439 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -139,24 +139,27 @@ public class TestFromClientSide {
   @Rule
   public TestName name = new TestName();
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  protected static final void initialize(Class<?>... cps) throws Exception {
     // Uncomment the following lines if more verbosity is needed for
     // debugging (see HBASE-12285 for details).
-    //((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
-    //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
-    //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
+    // ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
+    // ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
+    // ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
     // make sure that we do not get the same ts twice, see HBASE-19731 for more details.
     EnvironmentEdgeManager.injectEdge(new NonRepeatedEnvironmentEdge());
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
-        MultiRowMutationEndpoint.class.getName());
+      Arrays.stream(cps).map(Class::getName).toArray(String[]::new));
     conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
-    conf.setLong(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 6000000);
     // We need more than one region server in this test
     TEST_UTIL.startMiniCluster(SLAVES);
   }
 
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    initialize(MultiRowMutationEndpoint.class);
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/69d2becc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
index 5bf70c4..37d0135 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
@@ -17,9 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -29,23 +27,18 @@ import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
 /**
- * Test all client operations with a coprocessor that
- * just implements the default flush/compact/scan policy.
+ * Test all client operations with a coprocessor that just implements the default flush/compact/scan
+ * policy.
  */
-@Category({LargeTests.class, ClientTests.class})
+@Category({ LargeTests.class, ClientTests.class })
 public class TestFromClientSideWithCoprocessor extends TestFromClientSide {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestFromClientSideWithCoprocessor.class);
+    HBaseClassTestRule.forClass(TestFromClientSideWithCoprocessor.class);
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
-        MultiRowMutationEndpoint.class.getName(), NoOpScanPolicyObserver.class.getName());
-    conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
-    // We need more than one region server in this test
-    TEST_UTIL.startMiniCluster(SLAVES);
+    initialize(MultiRowMutationEndpoint.class, NoOpScanPolicyObserver.class);
   }
 }


[40/50] [abbrv] 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/c4573964
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c4573964
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c4573964

Branch: refs/heads/HBASE-19064
Commit: c457396424c304cc91e55385baf4430e2dc68b0a
Parents: c5a0c7e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Jan 22 11:44:49 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index b8546fa..167d6f3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -2648,6 +2649,44 @@ public interface Admin extends Abortable, Closeable {
   List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException;
 
   /**
+   * Transit current cluster to a new state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @param state a new state of current cluster
+   * @throws IOException if a remote or network exception occurs
+   */
+  void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws IOException;
+
+  /**
+   * Transit current cluster to a new state in a synchronous replication peer. But does not block
+   * and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param state a new state of current cluster
+   * @throws IOException if a remote or network exception occurs
+   */
+  Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
+      SyncReplicationState state) throws IOException;
+
+  /**
+   * Get the current cluster state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @return the current cluster state
+   * @throws IOException if a remote or network exception occurs
+   */
+  default SyncReplicationState getReplicationPeerSyncReplicationState(String peerId)
+      throws IOException {
+    List<ReplicationPeerDescription> peers = listReplicationPeers(Pattern.compile(peerId));
+    if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) {
+      throw new IOException("Replication peer " + peerId + " does not exist");
+    }
+    return peers.get(0).getSyncReplicationState();
+  }
+
+  /**
    * Mark region server(s) as decommissioned to prevent additional regions from getting
    * assigned to them. Optionally unload the regions on the servers. If there are multiple servers
    * to be decommissioned, decommissioning them at the same time can prevent wasteful region

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/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/c4573964/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/c4573964/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 6408044..82edd85 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -123,6 +123,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 
@@ -1718,6 +1720,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/c4573964/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/c4573964/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/c4573964/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/c4573964/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index 642149b..86b49ea 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -391,25 +392,28 @@ public final class ReplicationPeerConfigUtil {
     return ProtobufUtil.prependPBMagic(bytes);
   }
 
-  public static ReplicationPeerDescription toReplicationPeerDescription(
-      ReplicationProtos.ReplicationPeerDescription desc) {
-    boolean enabled = ReplicationProtos.ReplicationState.State.ENABLED == desc.getState()
-        .getState();
+  public static ReplicationPeerDescription
+      toReplicationPeerDescription(ReplicationProtos.ReplicationPeerDescription desc) {
+    boolean enabled =
+        ReplicationProtos.ReplicationState.State.ENABLED == desc.getState().getState();
     ReplicationPeerConfig config = convert(desc.getConfig());
-    return new ReplicationPeerDescription(desc.getId(), enabled, config);
+    return new ReplicationPeerDescription(desc.getId(), enabled, config,
+        SyncReplicationState.valueOf(desc.getSyncReplicationState().getNumber()));
   }
 
-  public static ReplicationProtos.ReplicationPeerDescription toProtoReplicationPeerDescription(
-      ReplicationPeerDescription desc) {
+  public static ReplicationProtos.ReplicationPeerDescription
+      toProtoReplicationPeerDescription(ReplicationPeerDescription desc) {
     ReplicationProtos.ReplicationPeerDescription.Builder builder =
         ReplicationProtos.ReplicationPeerDescription.newBuilder();
     builder.setId(desc.getPeerId());
-    ReplicationProtos.ReplicationState.Builder stateBuilder = ReplicationProtos.ReplicationState
-        .newBuilder();
-    stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED
-        : ReplicationProtos.ReplicationState.State.DISABLED);
+    ReplicationProtos.ReplicationState.Builder stateBuilder =
+        ReplicationProtos.ReplicationState.newBuilder();
+    stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED :
+        ReplicationProtos.ReplicationState.State.DISABLED);
     builder.setState(stateBuilder.build());
     builder.setConfig(convert(desc.getPeerConfig()));
+    builder.setSyncReplicationState(
+      ReplicationProtos.SyncReplicationState.forNumber(desc.getSyncReplicationState().ordinal()));
     return builder.build();
   }
 

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/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/c4573964/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
index 11507aa..d633be9 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/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 b4f0faf..f22765e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
@@ -286,6 +287,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;
@@ -1938,6 +1941,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/c4573964/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 0e552d6..5bd149a 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/c4573964/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/c4573964/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index d715e2e..9336fbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.master.replication;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.EnumMap;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -43,6 +45,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -61,6 +64,16 @@ public class ReplicationPeerManager {
 
   private final ConcurrentMap<String, ReplicationPeerDescription> peers;
 
+  private final EnumMap<SyncReplicationState, EnumSet<SyncReplicationState>> allowedTransition =
+    new EnumMap<SyncReplicationState, EnumSet<SyncReplicationState>>(SyncReplicationState.class) {
+      {
+        put(SyncReplicationState.ACTIVE, EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE));
+        put(SyncReplicationState.STANDBY, EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE));
+        put(SyncReplicationState.DOWNGRADE_ACTIVE,
+          EnumSet.of(SyncReplicationState.STANDBY, SyncReplicationState.ACTIVE));
+      }
+    };
+
   ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage,
       ConcurrentMap<String, ReplicationPeerDescription> peers) {
     this.peerStorage = peerStorage;
@@ -163,6 +176,17 @@ public class ReplicationPeerManager {
     }
   }
 
+  public void preTransitPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    SyncReplicationState fromState = desc.getSyncReplicationState();
+    EnumSet<SyncReplicationState> allowedToStates = allowedTransition.get(fromState);
+    if (allowedToStates == null || !allowedToStates.contains(state)) {
+      throw new DoNotRetryIOException("Can not transit current cluster state from " + fromState +
+          " to " + state + " for peer id=" + peerId);
+    }
+  }
+
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
     if (peers.containsKey(peerId)) {
@@ -170,8 +194,12 @@ public class ReplicationPeerManager {
       return;
     }
     ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build();
-    peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
+    SyncReplicationState syncReplicationState =
+        StringUtils.isBlank(peerConfig.getRemoteWALDir()) ? SyncReplicationState.NONE
+            : SyncReplicationState.DOWNGRADE_ACTIVE;
+    peerStorage.addPeer(peerId, copiedPeerConfig, enabled, syncReplicationState);
+    peers.put(peerId,
+      new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState));
   }
 
   public void removePeer(String peerId) throws ReplicationException {
@@ -190,7 +218,8 @@ public class ReplicationPeerManager {
       return;
     }
     peerStorage.setPeerState(peerId, enabled);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig()));
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig(),
+      desc.getSyncReplicationState()));
   }
 
   public void enablePeer(String peerId) throws ReplicationException {
@@ -215,7 +244,8 @@ public class ReplicationPeerManager {
     newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
     ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build();
     peerStorage.updatePeerConfig(peerId, newPeerConfig);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
+    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig,
+      desc.getSyncReplicationState()));
   }
 
   public List<ReplicationPeerDescription> listPeers(Pattern pattern) {
@@ -231,6 +261,14 @@ public class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
+  public void transitPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws ReplicationException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    peerStorage.setPeerSyncReplicationState(peerId, state);
+    peers.put(peerId,
+      new ReplicationPeerDescription(peerId, desc.isEnabled(), desc.getPeerConfig(), state));
+  }
+
   public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
     // Here we need two passes to address the problem of claimQueue. Maybe a claimQueue is still
     // on-going when the refresh peer config procedure is done, if a RS which has already been
@@ -350,10 +388,11 @@ public class ReplicationPeerManager {
     for (String peerId : peerStorage.listPeerIds()) {
       ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
       boolean enabled = peerStorage.isPeerEnabled(peerId);
-      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig));
+      SyncReplicationState state = peerStorage.getPeerSyncReplicationState(peerId);
+      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig, state));
     }
     return new ReplicationPeerManager(peerStorage,
-        ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
+      ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/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/c4573964/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 5fef8ab..00912bc 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;
@@ -2502,6 +2503,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/c4573964/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/c4573964/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 48e9e8d..b7d54d6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -54,11 +54,10 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
-import com.google.protobuf.Service;
-
 public class MockNoopMasterServices implements MasterServices {
   private final Configuration conf;
   private final MetricsMaster metricsMaster;
@@ -481,4 +480,10 @@ public class MockNoopMasterServices implements MasterServices {
   public ReplicationPeerManager getReplicationPeerManager() {
     return null;
   }
-}
+
+  @Override
+  public long transitReplicationPeerSyncReplicationState(String peerId,
+    SyncReplicationState clusterState) throws ReplicationException, IOException {
+    return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/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/c4573964/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/c4573964/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index a53cba3..db62b49 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -584,7 +585,7 @@ public abstract class TestReplicationSourceManager {
   private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig,
       final boolean waitForSource) throws Exception {
     final ReplicationPeers rp = manager.getReplicationPeers();
-    rp.getPeerStorage().addPeer(peerId, peerConfig, true);
+    rp.getPeerStorage().addPeer(peerId, peerConfig, true, SyncReplicationState.NONE);
     try {
       manager.addPeer(peerId);
     } catch (Exception e) {

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


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

Branch: refs/heads/HBASE-19064
Commit: 0ac76924616157a13a905817896549441c576683
Parents: 595ac44
Author: zhangduo <zh...@apache.org>
Authored: Wed Feb 14 16:01:16 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac76924/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/0ac76924/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 9d0c05d..9699e7b 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/0ac76924/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 a37fd4e..9bb8858 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;
@@ -208,7 +211,9 @@ public class MasterFileSystem {
   /**
    * @return HBase root log dir.
    */
-  public Path getWALRootDir() { return this.walRootDir; }
+  public Path getWALRootDir() {
+    return this.walRootDir;
+  }
 
   /**
    * @return HBase temp dir.
@@ -225,13 +230,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/0ac76924/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/0ac76924/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 e53abc0..3583fd8 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());
@@ -199,7 +203,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/0ac76924/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
index acddc4a..196019d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplication.java
@@ -19,7 +19,9 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -45,6 +47,7 @@ import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -154,8 +157,13 @@ public class TestSyncReplication {
 
   @Test
   public void testStandby() throws Exception {
+    MasterFileSystem mfs = UTIL2.getHBaseCluster().getMaster().getMasterFileSystem();
+    Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
+    Path remoteWALDirForPeer = new Path(remoteWALDir, PEER_ID);
+    assertFalse(mfs.getWALFileSystem().exists(remoteWALDirForPeer));
     UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
       SyncReplicationState.STANDBY);
+    assertTrue(mfs.getWALFileSystem().exists(remoteWALDirForPeer));
     try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
       assertDisallow(table, t -> t.get(new Get(Bytes.toBytes("row"))));
       assertDisallow(table,


[19/50] [abbrv] hbase git commit: HBASE-20024 TestMergeTableRegionsProcedure is STILL flakey

Posted by zh...@apache.org.
HBASE-20024 TestMergeTableRegionsProcedure is STILL flakey


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

Branch: refs/heads/HBASE-19064
Commit: 51cea3e2c3c75b82148f14ca7903a88daa3738d9
Parents: 2b19698
Author: Michael Stack <st...@apache.org>
Authored: Tue Feb 20 07:21:52 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Feb 20 11:08:27 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/procedure2/ProcedureExecutor.java     |  8 ++++----
 .../hadoop/hbase/procedure2/StateMachineProcedure.java |  9 ++-------
 .../master/assignment/MergeTableRegionsProcedure.java  |  8 ++++++++
 .../master/assignment/SplitTableRegionProcedure.java   | 13 ++++++++++---
 .../hbase/master/procedure/DeleteTableProcedure.java   |  4 ++--
 .../master/procedure/MasterProcedureScheduler.java     |  2 +-
 .../assignment/TestMergeTableRegionsProcedure.java     |  2 +-
 .../procedure/MasterProcedureTestingUtility.java       |  2 +-
 8 files changed, 29 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index af0a61b..665d223 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -931,18 +931,18 @@ public class ProcedureExecutor<TEnvironment> {
    * Send an abort notification the specified procedure.
    * Depending on the procedure implementation the abort can be considered or ignored.
    * @param procId the procedure to abort
-   * @return true if the procedure exist and has received the abort, otherwise false.
+   * @return true if the procedure exists and has received the abort, otherwise false.
    */
   public boolean abort(final long procId) {
     return abort(procId, true);
   }
 
   /**
-   * Send an abort notification the specified procedure.
-   * Depending on the procedure implementation the abort can be considered or ignored.
+   * Send an abort notification to the specified procedure.
+   * Depending on the procedure implementation, the abort can be considered or ignored.
    * @param procId the procedure to abort
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
-   * @return true if the procedure exist and has received the abort, otherwise false.
+   * @return true if the procedure exists and has received the abort, otherwise false.
    */
   public boolean abort(final long procId, final boolean mayInterruptIfRunning) {
     final Procedure proc = procedures.get(procId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
index 20bba58..c530386 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -209,18 +209,13 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
 
   @Override
   protected boolean abort(final TEnvironment env) {
-    final boolean isDebugEnabled = LOG.isDebugEnabled();
     final TState state = getCurrentState();
-    if (isDebugEnabled) {
-      LOG.debug("abort requested for " + this + " state=" + state);
-    }
-
+    LOG.debug("Abort requested for {}", this);
     if (hasMoreState()) {
       aborted.set(true);
       return true;
-    } else if (isDebugEnabled) {
-      LOG.debug("ignoring abort request on state=" + state + " for " + this);
     }
+    LOG.debug("Ignoring abort request on {}", this);
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index bd41208..c65dbe5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -795,4 +795,12 @@ public class MergeTableRegionsProcedure
   public RegionInfo getMergedRegion() {
     return this.mergedRegion;
   }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    // Abort means rollback. We can't rollback all steps. HBASE-18018 added abort to all
+    // Procedures. Here is a Procedure that has a PONR and cannot be aborted wants it enters this
+    // range of steps; what do we do for these should an operator want to cancel them? HBASE-20022.
+    return isRollbackSupported(getCurrentState())? super.abort(env): false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index e898d6a..f7f49bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -262,14 +262,13 @@ public class SplitTableRegionProcedure
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
       }
     } catch (IOException e) {
-      String msg = "Error trying to split region " + getParentRegion().getEncodedName() + " in the table "
-          + getTableName() + " (in state=" + state + ")";
+      String msg = "Error trying to split region " + getParentRegion().getEncodedName() +
+          " in the table " + getTableName() + " (in state=" + state + ")";
       if (!isRollbackSupported(state)) {
         // We reach a state that cannot be rolled back. We just need to keep retry.
         LOG.warn(msg, e);
       } else {
         LOG.error(msg, e);
-        setFailure(e);
         setFailure("master-split-regions", e);
       }
     }
@@ -831,4 +830,12 @@ public class SplitTableRegionProcedure
     }
     return traceEnabled;
   }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    // Abort means rollback. We can't rollback all steps. HBASE-18018 added abort to all
+    // Procedures. Here is a Procedure that has a PONR and cannot be aborted wants it enters this
+    // range of steps; what do we do for these should an operator want to cancel them? HBASE-20022.
+    return isRollbackSupported(getCurrentState())? super.abort(env): false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index c519fe6..487bb27 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -144,8 +144,8 @@ public class DeleteTableProcedure
 
   @Override
   protected boolean abort(MasterProcedureEnv env) {
-    // TODO: Current behavior is: with no rollback and no abort support, procedure may stuck
-    // looping in retrying failing step forever. Default behavior of abort is changed to support
+    // TODO: Current behavior is: with no rollback and no abort support, procedure may get stuck
+    // looping in retrying failing a step forever. Default behavior of abort is changed to support
     // aborting all procedures. Override the default wisely. Following code retains the current
     // behavior. Revisit it later.
     return isRollbackSupported(getCurrentState()) ? super.abort(env) : false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 5cc9298..120bba3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -633,7 +633,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       boolean hasLock = true;
       final LockAndQueue[] regionLocks = new LockAndQueue[regionInfo.length];
       for (int i = 0; i < regionInfo.length; ++i) {
-        LOG.info(procedure + ", table=" + table + ", " + regionInfo[i].getRegionNameAsString());
+        LOG.info(procedure + ", " + regionInfo[i].getRegionNameAsString());
         assert table != null;
         assert regionInfo[i] != null;
         assert regionInfo[i].getTable() != null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
index 3285d3d..094a5a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
@@ -276,7 +276,7 @@ public class TestMergeTableRegionsProcedure {
 
   @Test
   public void testMergeWithoutPONR() throws Exception {
-    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
+    final TableName tableName = TableName.valueOf("testMergeWithoutPONR");
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
 
     List<RegionInfo> tableRegions = createTable(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index fc8953b..84b0094 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -401,7 +401,7 @@ public class MasterProcedureTestingUtility {
    * idempotent. Use this version of the test when the order in which flow steps are executed is
    * not start to finish; where the procedure may vary the flow steps dependent on circumstance
    * found.
-   * @see #testRecoveryAndDoubleExecution(ProcedureExecutor, long, int)
+   * @see #testRecoveryAndDoubleExecution(ProcedureExecutor, long, int, boolean)
    */
   public static void testRecoveryAndDoubleExecution(
       final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId) throws Exception {


[44/50] [abbrv] 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/f044dc49
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f044dc49
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f044dc49

Branch: refs/heads/HBASE-19064
Commit: f044dc4961ed328a8ad09f2de7c12a3334503f06
Parents: 2440f80
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 11 21:08:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 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/f044dc49/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 d22d1ec..91fb5b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -606,12 +606,16 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  @Override
-  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+  protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException {
     return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup,
       channelClass);
   }
 
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    return createAsyncWriter(fs, path);
+  }
+
   private void waitForSafePoint() {
     consumeLock.lock();
     try {
@@ -631,13 +635,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);
         }
@@ -652,7 +655,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   protected void doReplaceWriter(Path oldPath, Path newPath, AsyncWriter nextWriter)
       throws IOException {
     waitForSafePoint();
-    long oldFileLen = closeWriter();
+    long oldFileLen = closeWriter(this.writer);
     logRollAndSetupWalProps(oldPath, newPath, oldFileLen);
     this.writer = nextWriter;
     if (nextWriter != null && nextWriter instanceof AsyncProtobufLogWriter) {
@@ -677,7 +680,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/f044dc49/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/f044dc49/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/f044dc49/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/f044dc49/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
new file mode 100644
index 0000000..ecd8e6c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.stream.IntStream;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hadoop.hbase.wal.WALProvider;
+
+/**
+ * Helper class for testing protobuf log.
+ */
+final class ProtobufLogTestHelper {
+
+  private ProtobufLogTestHelper() {
+  }
+
+  private static byte[] toValue(int prefix, int suffix) {
+    return Bytes.toBytes(prefix + "-" + suffix);
+  }
+
+  private static RegionInfo toRegionInfo(TableName tableName) {
+    return RegionInfoBuilder.newBuilder(tableName).setRegionId(1024).build();
+  }
+
+  public static void doWrite(WALProvider.Writer writer, boolean withTrailer, TableName tableName,
+      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+    RegionInfo hri = toRegionInfo(tableName);
+    for (int i = 0; i < recordCount; i++) {
+      WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, i, timestamp,
+          HConstants.DEFAULT_CLUSTER_ID);
+      WALEdit edit = new WALEdit();
+      int prefix = i;
+      IntStream.range(0, columnCount).mapToObj(j -> toValue(prefix, j))
+          .map(value -> new KeyValue(row, row, row, timestamp, value)).forEachOrdered(edit::add);
+      writer.append(new WAL.Entry(key, edit));
+    }
+    writer.sync();
+    if (withTrailer) {
+      writer.close();
+    }
+  }
+
+  public static void doRead(ProtobufLogReader reader, boolean withTrailer, TableName tableName,
+      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+    if (withTrailer) {
+      assertNotNull(reader.trailer);
+    } else {
+      assertNull(reader.trailer);
+    }
+    RegionInfo hri = toRegionInfo(tableName);
+    for (int i = 0; i < recordCount; ++i) {
+      WAL.Entry entry = reader.next();
+      assertNotNull(entry);
+      assertEquals(columnCount, entry.getEdit().size());
+      assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
+      assertEquals(tableName, entry.getKey().getTableName());
+      int idx = 0;
+      for (Cell val : entry.getEdit().getCells()) {
+        assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
+          val.getRowLength()));
+        assertArrayEquals(toValue(i, idx), CellUtil.cloneValue(val));
+        idx++;
+      }
+    }
+    assertNull(reader.next());
+  }
+}

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

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

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


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

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

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

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


[14/50] [abbrv] hbase git commit: HBASE-19950 Introduce a ColumnValueFilter

Posted by zh...@apache.org.
HBASE-19950 Introduce a ColumnValueFilter

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


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

Branch: refs/heads/HBASE-19064
Commit: a9a6eed372fa11dd2851635ac2c4bb6e1ca9dba7
Parents: f3bb9b9
Author: Reid Chan <re...@outlook.com>
Authored: Tue Feb 13 16:01:28 2018 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Tue Feb 20 04:56:13 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/ColumnValueFilter.java  | 241 +++++++++++++++++++
 .../hadoop/hbase/filter/CompareFilter.java      |  10 +-
 .../apache/hadoop/hbase/filter/ParseFilter.java |   4 +-
 .../src/main/protobuf/Filter.proto              |   7 +
 hbase-protocol/src/main/protobuf/Filter.proto   |   9 +-
 .../apache/hadoop/hbase/filter/TestFilter.java  | 155 ++++++++++++
 .../hbase/filter/TestFilterSerialization.java   |   9 +
 .../hadoop/hbase/filter/TestParseFilter.java    |  11 +
 8 files changed, 440 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
new file mode 100644
index 0000000..0795165
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
@@ -0,0 +1,241 @@
+/**
+ *
+ * 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.filter;
+
+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.CompareOperator;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+/**
+ * Different from {@link SingleColumnValueFilter} which returns an <b>entire</b> row
+ * when specified condition is matched, {@link ColumnValueFilter} return the matched cell only.
+ * <p>
+ * This filter is used to filter cells based on column and value.
+ * It takes a {@link org.apache.hadoop.hbase.CompareOperator} operator (<, <=, =, !=, >, >=), and
+ * and a {@link ByteArrayComparable} comparator.
+ */
+@InterfaceAudience.Public
+public class ColumnValueFilter extends FilterBase {
+  private final byte[] family;
+  private final byte[] qualifier;
+  private final CompareOperator op;
+  private final ByteArrayComparable comparator;
+
+  // This flag is used to speed up seeking cells when matched column is found, such that following
+  // columns in the same row can be skipped faster by NEXT_ROW instead of NEXT_COL.
+  private boolean columnFound = false;
+
+  public ColumnValueFilter(final byte[] family, final byte[] qualifier,
+                           final CompareOperator op, final byte[] value) {
+    this(family, qualifier, op, new BinaryComparator(value));
+  }
+
+  public ColumnValueFilter(final byte[] family, final byte[] qualifier,
+                           final CompareOperator op,
+                           final ByteArrayComparable comparator) {
+    this.family = Preconditions.checkNotNull(family, "family should not be null.");
+    this.qualifier = qualifier == null ? new byte[0] : qualifier;
+    this.op = Preconditions.checkNotNull(op, "CompareOperator should not be null");
+    this.comparator = Preconditions.checkNotNull(comparator, "Comparator should not be null");
+  }
+
+  /**
+   * @return operator
+   */
+  public CompareOperator getCompareOperator() {
+    return op;
+  }
+
+  /**
+   * @return the comparator
+   */
+  public ByteArrayComparable getComparator() {
+    return comparator;
+  }
+
+  /**
+   * @return the column family
+   */
+  public byte[] getFamily() {
+    return family;
+  }
+
+  /**
+   * @return the qualifier
+   */
+  public byte[] getQualifier() {
+    return qualifier;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    columnFound = false;
+  }
+
+  @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    return false;
+  }
+
+  @Override
+  public ReturnCode filterCell(Cell c) throws IOException {
+    // 1. Check column match
+    if (!CellUtil.matchingColumn(c, this.family, this.qualifier)) {
+      return columnFound ? ReturnCode.NEXT_ROW : ReturnCode.NEXT_COL;
+    }
+    // Column found
+    columnFound = true;
+    // 2. Check value match:
+    // True means filter out, just skip this cell, else include it.
+    return compareValue(getCompareOperator(), getComparator(), c) ?
+      ReturnCode.SKIP : ReturnCode.INCLUDE;
+  }
+
+  /**
+   * This method is used to determine a cell should be included or filtered out.
+   * @param op one of operators {@link CompareOperator}
+   * @param comparator comparator used to compare cells.
+   * @param cell cell to be compared.
+   * @return true means cell should be filtered out, included otherwise.
+   */
+  private boolean compareValue(final CompareOperator op, final ByteArrayComparable comparator,
+    final Cell cell) {
+    if (op == CompareOperator.NO_OP) {
+      return true;
+    }
+    int compareResult = PrivateCellUtil.compareValue(cell, comparator);
+    return CompareFilter.compare(op, compareResult);
+  }
+
+  /**
+   * Creating this filter by reflection, it is used by {@link ParseFilter},
+   * @param filterArguments arguments for creating a ColumnValueFilter
+   * @return a ColumnValueFilter
+   */
+  public static Filter createFilterFromArguments(ArrayList<byte[]> filterArguments) {
+    Preconditions.checkArgument(filterArguments.size() == 4,
+      "Expect 4 arguments: %s", filterArguments.size());
+    byte[] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+    byte[] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
+    CompareOperator operator = ParseFilter.createCompareOperator(filterArguments.get(2));
+    ByteArrayComparable comparator =
+      ParseFilter.createComparator(ParseFilter.removeQuotesFromByteArray(filterArguments.get(3)));
+
+    if (comparator instanceof RegexStringComparator ||
+        comparator instanceof SubstringComparator) {
+      if (operator != CompareOperator.EQUAL &&
+          operator != CompareOperator.NOT_EQUAL) {
+        throw new IllegalArgumentException("A regexstring comparator and substring comparator " +
+            "can only be used with EQUAL and NOT_EQUAL");
+      }
+    }
+
+    return new ColumnValueFilter(family, qualifier, operator, comparator);
+  }
+
+  /**
+   * @return A pb instance to represent this instance.
+   */
+  FilterProtos.ColumnValueFilter convert() {
+    FilterProtos.ColumnValueFilter.Builder builder =
+      FilterProtos.ColumnValueFilter.newBuilder();
+
+    builder.setFamily(UnsafeByteOperations.unsafeWrap(this.family));
+    builder.setQualifier(UnsafeByteOperations.unsafeWrap(this.qualifier));
+    builder.setCompareOp(HBaseProtos.CompareType.valueOf(this.op.name()));
+    builder.setComparator(ProtobufUtil.toComparator(this.comparator));
+
+    return builder.build();
+  }
+
+  /**
+   * Parse protobuf bytes to a ColumnValueFilter
+   * @param pbBytes pbBytes
+   * @return a ColumnValueFilter
+   * @throws DeserializationException deserialization exception
+   */
+  public static ColumnValueFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
+    FilterProtos.ColumnValueFilter proto;
+    try {
+      proto = FilterProtos.ColumnValueFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+
+    final CompareOperator compareOp = CompareOperator.valueOf(proto.getCompareOp().name());
+    final ByteArrayComparable comparator;
+    try {
+      comparator = ProtobufUtil.toComparator(proto.getComparator());
+    } catch (IOException ioe) {
+      throw new DeserializationException(ioe);
+    }
+
+    return new ColumnValueFilter(proto.getFamily().toByteArray(),
+      proto.getQualifier().toByteArray(), compareOp, comparator);
+  }
+
+  @Override
+  public byte[] toByteArray() throws IOException {
+    return convert().toByteArray();
+  }
+
+  @Override
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) {
+      return true;
+    } else if (!(o instanceof ColumnValueFilter)) {
+      return false;
+    }
+
+    ColumnValueFilter other = (ColumnValueFilter) o;
+    return Bytes.equals(this.getFamily(), other.getFamily()) &&
+      Bytes.equals(this.getQualifier(), other.getQualifier()) &&
+      this.getCompareOperator().equals(other.getCompareOperator()) &&
+      this.getComparator().areSerializedFieldsEqual(other.getComparator());
+  }
+
+  @Override
+  public boolean isFamilyEssential(byte[] name) throws IOException {
+    return Bytes.equals(name, this.family);
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s (%s, %s, %s, %s)",
+      getClass().getSimpleName(), Bytes.toStringBinary(this.family),
+      Bytes.toStringBinary(this.qualifier), this.op.name(),
+      Bytes.toStringBinary(this.comparator.getValue()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
index 6e6ca60..50924ec 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
@@ -25,23 +25,27 @@ import java.util.ArrayList;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
-import org.apache.hadoop.hbase.util.Bytes;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 /**
  * This is a generic filter to be used to filter by comparison.  It takes an
  * operator (equal, greater, not equal, etc) and a byte [] comparator.
  * <p>
  * To filter by row key, use {@link RowFilter}.
  * <p>
+ * To filter by column family, use {@link FamilyFilter}.
+ * <p>
  * To filter by column qualifier, use {@link QualifierFilter}.
  * <p>
- * To filter by value, use {@link SingleColumnValueFilter}.
+ * To filter by value, use {@link ValueFilter}.
  * <p>
  * These filters can be wrapped with {@link SkipFilter} and {@link WhileMatchFilter}
  * to add more control.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
index e984212..716322c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
@@ -93,6 +93,8 @@ public class ParseFilter {
                       "SingleColumnValueExcludeFilter");
     filterHashMap.put("DependentColumnFilter", ParseConstants.FILTER_PACKAGE + "." +
                       "DependentColumnFilter");
+    filterHashMap.put("ColumnValueFilter", ParseConstants.FILTER_PACKAGE + "." +
+                      "ColumnValueFilter");
 
     // Creates the operatorPrecedenceHashMap
     operatorPrecedenceHashMap = new HashMap<>();
@@ -769,8 +771,6 @@ public class ParseFilter {
 
   /**
    * Takes a compareOperator symbol as a byte array and returns the corresponding CompareOperator
-   * @deprecated Since 2.0
-   * <p>
    * @param compareOpAsByteArray the comparatorOperator symbol as a byte array
    * @return the Compare Operator
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-protocol-shaded/src/main/protobuf/Filter.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Filter.proto b/hbase-protocol-shaded/src/main/protobuf/Filter.proto
index 7434985..b0d6da6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Filter.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Filter.proto
@@ -170,3 +170,10 @@ message RowRange {
 message MultiRowRangeFilter {
   repeated RowRange row_range_list = 1;
 }
+
+message ColumnValueFilter {
+  required bytes family = 1;
+  required bytes qualifier = 2;
+  required CompareType compare_op = 3;
+  required Comparator comparator = 4;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-protocol/src/main/protobuf/Filter.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto
index 1fa6697..8a0a9bf 100644
--- a/hbase-protocol/src/main/protobuf/Filter.proto
+++ b/hbase-protocol/src/main/protobuf/Filter.proto
@@ -168,4 +168,11 @@ message RowRange {
 
 message MultiRowRangeFilter {
   repeated RowRange row_range_list = 1;
-}
\ No newline at end of file
+}
+
+message ColumnValueFilter {
+  required bytes family = 1;
+  required bytes qualifier = 2;
+  required CompareType compare_op = 3;
+  required Comparator comparator = 4;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
index 3c5be63..a3e3359 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
@@ -132,6 +132,10 @@ public class TestFilter {
     Bytes.toBytes("testQualifierFour-2"), Bytes.toBytes("testQualifierFour-3")
   };
 
+  private static final byte [][] QUALIFIERS_FIVE = {
+    Bytes.toBytes("testQualifierFive-0"), Bytes.toBytes("testQualifierFive-1")
+  };
+
   private static final byte [][] VALUES = {
     Bytes.toBytes("testValueOne"), Bytes.toBytes("testValueTwo")
   };
@@ -1663,6 +1667,157 @@ public class TestFilter {
 
   }
 
+  @Test
+  public void testColumnValueFilter() throws Exception {
+    // Prepare test rows:
+    for (int i = 0; i < 2; i++) {
+      for (int j = 0; j < ROWS_ONE.length; j++) {
+        Put p1 = new Put(ROWS_ONE[j]).setDurability(Durability.SKIP_WAL);
+        Put p2 = new Put(ROWS_TWO[j]).setDurability(Durability.SKIP_WAL);
+        for (byte[] q5 : QUALIFIERS_FIVE) {
+          p1.addColumn(FAMILIES[0], q5, VALUES[0 + i]).addColumn(FAMILIES[1], q5, VALUES[0 + i]);
+          p2.addColumn(FAMILIES[0], q5, VALUES[1 - i]).addColumn(FAMILIES[1], q5, VALUES[1 - i]);
+        }
+        this.region.put(p1);
+        this.region.put(p2);
+      }
+      this.region.flush(true);
+    }
+    // 1. Test = f[0]:q5[0]:v[1]
+    Scan scan = new Scan().setFilter(
+      new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.EQUAL, VALUES[1]));
+    KeyValue[] expectedEquals =
+      { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
+    verifyScanFull(scan, expectedEquals);
+    // 2. Test > f[0]:q5[0]:v[0]
+    scan.setFilter(
+      new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.GREATER, VALUES[0]));
+    KeyValue[] expectedGreater =
+      { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
+    verifyScanFull(scan, expectedGreater);
+    // 3. Test >= f[0]:q5[0]:v[0]
+    // also test readAllVersions(), since FAMILIES[0] allow multiple versions.
+    scan.readAllVersions().setFilter(new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0],
+      CompareOperator.GREATER_OR_EQUAL, VALUES[0]));
+    KeyValue[] expectedGreaterOrEqual =
+      { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
+    verifyScanFull(scan, expectedGreaterOrEqual);
+    // 4. Test < f[1]:q5[1]:v[1], FAMILIES[1] doesn't support multiple versions
+    scan.readVersions(1).setFilter(new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1],
+      CompareOperator.LESS, VALUES[1]));
+    KeyValue[] expectedLess =
+      { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
+    verifyScanFull(scan, expectedLess);
+    // 5. Test <= f[1]:q5[0]:v[1]
+    scan.setFilter(new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1],
+      CompareOperator.LESS_OR_EQUAL, VALUES[1]));
+    KeyValue[] expectedLessOrEqual =
+      { new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
+        new KeyValue(ROWS_ONE[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
+    verifyScanFull(scan, expectedLessOrEqual);
+    // 6. Test != f[1]:q5[1]:v[1]
+    scan.setFilter(
+      new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
+    KeyValue[] expectedNotEqual =
+      { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
+    verifyScanFull(scan, expectedNotEqual);
+    // 7. Test FilterList(MUST_PASS_ONE) combining ColumnValueFilter and QualifierFilter
+    // (ColumnValueFilter, != f[1]:q5[1]:v[1]) || (QualifierFilter, = q5[0])
+    List<Filter> orFilters = new ArrayList<>(2);
+    orFilters.add(
+      new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
+    orFilters.add(
+      new QualifierFilter(CompareOperator.EQUAL, new BinaryComparator(QUALIFIERS_FIVE[0])));
+    scan.setFilter(new FilterList(Operator.MUST_PASS_ONE, orFilters));
+    KeyValue[] expectedMustPassOne =
+      { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[1], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
+        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; // this pass scvf
+    verifyScanFull(scan, expectedMustPassOne);
+    // 8. Test FilterList(MUST_PASS_ALL) combining ColumnValueFilter and RowFilter
+    // (ColumnValueFilter, != f[1]:q5[1]:v[1]) && (RowFilter, = prefix:"testRow")
+    List<Filter> andFilters = new ArrayList<>(2);
+    andFilters.add(
+      new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
+    andFilters.add(new RowFilter(CompareOperator.EQUAL,
+      new BinaryPrefixComparator(Bytes.toBytes("testRow"))));
+    scan.setFilter(new FilterList(Operator.MUST_PASS_ALL, andFilters));
+    KeyValue[] expectedMustPassAll =
+      { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
+    verifyScanFull(scan, expectedMustPassAll);
+    // 9. Test specified columns with FilterList(MUST_PASS_ONE) which sused in case 7.
+    // Result is different from case 7, because column is strongly constrained by specified columns
+    Scan anotherScan = new Scan().addColumn(FAMILIES[1], QUALIFIERS_FIVE[1])
+      .setFilter(new FilterList(Operator.MUST_PASS_ONE, orFilters));
+    KeyValue[] expectedValues =
+      { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
+    verifyScanFull(anotherScan, expectedValues);
+  }
+
   private void verifyScan(Scan s, long expectedRows, long expectedKeys)
   throws IOException {
     InternalScanner scanner = this.region.getScanner(s);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java
index 2a0d1f4..b0d876b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java
@@ -341,4 +341,13 @@ public class TestFilterSerialization {
     assertTrue(multiRowRangeFilter.areSerializedFieldsEqual(
       ProtobufUtil.toFilter(ProtobufUtil.toFilter(multiRowRangeFilter))));
   }
+
+  @Test
+  public void testColumnValueFilter() throws Exception {
+    ColumnValueFilter columnValueFilter =
+        new ColumnValueFilter(Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
+            CompareOperator.EQUAL, Bytes.toBytes("value"));
+    assertTrue(columnValueFilter.areSerializedFieldsEqual(
+        ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnValueFilter))));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java
index d30958c..cee0fb3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java
@@ -685,4 +685,15 @@ public class TestParseFilter {
     assertEquals(clazz, filter.getClass());
     return clazz.cast(filter);
   }
+
+  @Test
+  public void testColumnValueFilter() throws IOException {
+    String filterString = "ColumnValueFilter ('family', 'qualifier', <, 'binaryprefix:value')";
+    ColumnValueFilter cvf = doTestFilter(filterString, ColumnValueFilter.class);
+    assertEquals("family", new String(cvf.getFamily(), StandardCharsets.UTF_8));
+    assertEquals("qualifier", new String(cvf.getQualifier(), StandardCharsets.UTF_8));
+    assertEquals(CompareOperator.LESS, cvf.getCompareOperator());
+    assertTrue(cvf.getComparator() instanceof BinaryPrefixComparator);
+    assertEquals("value", new String(cvf.getComparator().getValue(), StandardCharsets.UTF_8));
+  }
 }


[48/50] [abbrv] 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/595ac441
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/595ac441
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/595ac441

Branch: refs/heads/HBASE-19064
Commit: 595ac441f9bb87b1f7cb83fc5253cfbd709f7927
Parents: b54b94b
Author: zhangduo <zh...@apache.org>
Authored: Mon Feb 12 18:20:18 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 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       |  89 +++++++--
 .../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(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/595ac441/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 4664778..1cd6f89 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
@@ -1351,9 +1351,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/595ac441/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 8cc5c81..2bf634b 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -382,7 +382,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/595ac441/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 d633be9..9d0c05d 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/595ac441/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 c253bff..e53abc0 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
@@ -191,8 +191,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/595ac441/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 e26cc43..d2afbb4 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
@@ -4219,12 +4219,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/595ac441/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 6d81067..0c3e175 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
@@ -1788,7 +1788,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/595ac441/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 05bbb47..20d3259 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
@@ -116,6 +116,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;
@@ -1189,7 +1191,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
     }
     priority = createPriority();
-    String hostname = initialIsa.getHostName();
     // Using Address means we don't get the IP too. Shorten it more even to just the host name
     // w/o the domain.
     String name = rs.getProcessName() + "/" +
@@ -2375,6 +2376,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.
    *
@@ -2383,8 +2396,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;
@@ -2393,6 +2406,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();
@@ -2522,16 +2536,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) {
@@ -2571,17 +2614,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 {
@@ -2608,6 +2653,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
@@ -2623,7 +2677,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);
       }
     }
@@ -2682,8 +2736,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;
@@ -2703,6 +2757,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()) {
@@ -2867,6 +2922,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) {
@@ -2893,6 +2949,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/595ac441/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/595ac441/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 5a05660..ae072d6 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);
     }
   }
 
@@ -217,6 +217,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)) {
@@ -370,11 +372,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;
           }
         }
       }
@@ -390,14 +392,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);
       }
@@ -410,21 +410,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;
         }
       }
     }
@@ -437,9 +434,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/595ac441/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/595ac441/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/595ac441/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/595ac441/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/595ac441/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/595ac441/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());


[42/50] [abbrv] 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/b54b94ba
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b54b94ba
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b54b94ba

Branch: refs/heads/HBASE-19064
Commit: b54b94bab415926a07685341bb115a0dee0341a9
Parents: 3d72064
Author: zhangduo <zh...@apache.org>
Authored: Fri Feb 9 18:33:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 22 14:48:42 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   |  65 +++++--
 .../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 |  15 +-
 .../replication/RefreshPeerProcedure.java       |  18 +-
 .../replication/ReplicationPeerManager.java     | 107 +++++++-----
 ...ransitPeerSyncReplicationStateProcedure.java | 175 ++++++++++++-------
 .../hbase/regionserver/HRegionServer.java       |  35 ++--
 .../regionserver/ReplicationSourceService.java  |  11 +-
 .../regionserver/PeerActionListener.java        |   4 +-
 .../regionserver/PeerProcedureHandler.java      |  16 +-
 .../regionserver/PeerProcedureHandlerImpl.java  |  55 +++++-
 .../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, 752 insertions(+), 325 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b54b94ba/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 69565a7..79b3a1d 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/b54b94ba/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/b54b94ba/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/b54b94ba/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 1cd6f89..4664778 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
@@ -1351,6 +1351,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/b54b94ba/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 1dffd33..8cc5c81 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -374,6 +374,17 @@ enum PeerModificationState {
   POST_PEER_MODIFICATION = 4;
 }
 
+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;
 }
@@ -384,18 +395,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 {
@@ -412,5 +428,7 @@ message UpdatePeerConfigStateData {
 }
 
 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/b54b94ba/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/b54b94ba/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/b54b94ba/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/b54b94ba/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 909daa0..9241ce3 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
@@ -30,7 +30,9 @@ 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,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)
@@ -83,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
@@ -95,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);
     }
   }
@@ -134,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);
     }
   }
 
@@ -167,38 +179,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/b54b94ba/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/b54b94ba/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 f671363..f0ce8bc 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;
@@ -3378,7 +3378,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/b54b94ba/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 ba5e70a..bf60066 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
@@ -1525,22 +1525,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/b54b94ba/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/b54b94ba/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 83c5134..ac5d2af 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
@@ -105,8 +105,8 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
         return Flow.HAS_MORE_STATE;
       case REFRESH_PEER_ON_RS:
         addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
-            .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn))
-            .toArray(RefreshPeerProcedure[]::new));
+          .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn))
+          .toArray(RefreshPeerProcedure[]::new));
         setNextState(PeerModificationState.POST_PEER_MODIFICATION);
         return Flow.HAS_MORE_STATE;
       case POST_PEER_MODIFICATION:
@@ -128,17 +128,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/b54b94ba/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 1253ef9..b6825a5 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
@@ -52,6 +52,8 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
 
   private ServerName targetServer;
 
+  private int stage;
+
   private boolean dispatched;
 
   private ProcedureEvent<?> event;
@@ -62,9 +64,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
@@ -89,6 +97,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);
     }
@@ -106,6 +116,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);
     }
@@ -116,7 +128,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) {
@@ -191,7 +204,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
@@ -200,5 +213,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/b54b94ba/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 6bfd9c9..d80e9a4 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) {
@@ -147,44 +145,48 @@ public class ReplicationPeerManager {
     ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
     if (!isStringEquals(peerConfig.getClusterKey(), oldPeerConfig.getClusterKey())) {
       throw new DoNotRetryIOException(
-          "Changing the cluster key on an existing peer is not allowed. Existing key '" +
-              oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
-              peerConfig.getClusterKey() + "'");
+        "Changing the cluster key on an existing peer is not allowed. Existing key '" +
+          oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
+          peerConfig.getClusterKey() + "'");
     }
 
     if (!isStringEquals(peerConfig.getReplicationEndpointImpl(),
       oldPeerConfig.getReplicationEndpointImpl())) {
       throw new DoNotRetryIOException("Changing the replication endpoint implementation class " +
-          "on an existing peer is not allowed. Existing class '" +
-          oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
-          " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
+        "on an existing peer is not allowed. Existing class '" +
+        oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
+        " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
     }
 
     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()) {
       if (!ReplicationUtils.isKeyConfigEqual(oldPeerConfig, peerConfig)) {
         throw new DoNotRetryIOException(
-            "Changing the replicated namespace/table config on a synchronous replication "
-                + "peer(peerId: " + peerId + ") is not allowed.");
+          "Changing the replicated namespace/table config on a synchronous replication " +
+            "peer(peerId: " + peerId + ") is not allowed.");
       }
     }
   }
 
-  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)
@@ -195,8 +197,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));
@@ -236,7 +238,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());
@@ -253,7 +255,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) {
@@ -261,12 +263,23 @@ public class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
-  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 {
@@ -293,10 +306,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());
@@ -304,13 +317,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());
@@ -330,11 +343,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");
@@ -342,7 +355,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");
       }
     }
   }
@@ -386,7 +399,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) {
@@ -411,7 +424,7 @@ public class ReplicationPeerManager {
   public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
       throws ReplicationException {
     ReplicationPeerStorage peerStorage =
-        ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
+      ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
     ConcurrentMap<String, ReplicationPeerDescription> peers = new ConcurrentHashMap<>();
     for (String peerId : peerStorage.listPeerIds()) {
       ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b54b94ba/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..c253bff 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,148 @@ 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:
+        addChildProcedure(
+          env.getAssignmentManager().createReopenProcedures(getRegionsToReopen(env)));
+        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/b54b94ba/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 49b7b80..6d81067 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
@@ -1785,21 +1785,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;
   }
 
@@ -2918,11 +2924,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/b54b94ba/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/b54b94ba/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/b54b94ba/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/b54b94ba/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 a02d181..548ea41 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,22 +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
@@ -60,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;
@@ -71,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);
@@ -109,7 +118,7 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
       }
       oldConfig = peer.getPeerConfig();
       ReplicationPeerConfig newConfig =
-          replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+        replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
       // RS need to start work with the new replication config change
       if (!ReplicationUtils.isKeyConfigEqual(oldConfig, newConfig)) {
         replicationSourceManager.refreshSources(peerId);
@@ -123,4 +132,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/b54b94ba/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/b54b94ba/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 7803ac4..f355021 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("ReplicationStatisticsThread " + 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;
+  }
 }


[34/50] [abbrv] hbase git commit: HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs

Posted by zh...@apache.org.
HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs

Negative Remaining KVs and progress percent greater than 100 is because CompactionProgress#totalCompactingKVs is sometimes less than CompactionProgress#currentCompactedKVs.
Changes add a getter to CompactionProgress#totalCompactingKVs and from inside getter warning is logged. currentCompactedKVs are return when totalCompactingKVs are less than current.


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

Branch: refs/heads/HBASE-19064
Commit: 61b55166bf7fe9edc4e8105f217463ed6e693d17
Parents: 3a39942
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Wed Feb 21 13:59:59 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Feb 21 15:14:24 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegionServer.java    |  2 +-
 .../apache/hadoop/hbase/regionserver/HStore.java    |  4 ++--
 .../compactions/CompactionProgress.java             | 16 ++++++++++++----
 .../hbase/regionserver/TestMajorCompaction.java     |  2 +-
 4 files changed, 16 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/61b55166/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 49b7b80..a76dec2 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
@@ -1623,7 +1623,7 @@ public class HRegionServer extends HasThread implements
       storefileIndexSizeKB += store.getStorefilesRootLevelIndexSize() / 1024;
       CompactionProgress progress = store.getCompactionProgress();
       if (progress != null) {
-        totalCompactingKVs += progress.totalCompactingKVs;
+        totalCompactingKVs += progress.getTotalCompactingKVs();
         currentCompactedKVs += progress.currentCompactedKVs;
       }
       rootLevelIndexSizeKB += (int) (store.getStorefilesRootLevelIndexSize() / 1024);

http://git-wip-us.apache.org/repos/asf/hbase/blob/61b55166/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index bef50b4..c0ef3ef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1373,10 +1373,10 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
     writeCompactionWalRecord(filesToCompact, sfs);
     replaceStoreFiles(filesToCompact, sfs);
     if (cr.isMajor()) {
-      majorCompactedCellsCount += getCompactionProgress().totalCompactingKVs;
+      majorCompactedCellsCount += getCompactionProgress().getTotalCompactingKVs();
       majorCompactedCellsSize += getCompactionProgress().totalCompactedSize;
     } else {
-      compactedCellsCount += getCompactionProgress().totalCompactingKVs;
+      compactedCellsCount += getCompactionProgress().getTotalCompactingKVs();
       compactedCellsSize += getCompactionProgress().totalCompactedSize;
     }
     long outputBytes = getTotalSize(sfs);

http://git-wip-us.apache.org/repos/asf/hbase/blob/61b55166/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
index d40651a..577276e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java
@@ -20,6 +20,8 @@
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class holds information relevant for tracking the progress of a
@@ -32,9 +34,10 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public class CompactionProgress {
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionProgress.class);
 
   /** the total compacting key values in currently running compaction */
-  public long totalCompactingKVs;
+  private long totalCompactingKVs;
   /** the completed count of key values in currently running compaction */
   public long currentCompactedKVs = 0;
   /** the total size of data processed by the currently running compaction, in bytes */
@@ -51,7 +54,7 @@ public class CompactionProgress {
    * @return float
    */
   public float getProgressPct() {
-    return (float)currentCompactedKVs / totalCompactingKVs;
+    return (float)currentCompactedKVs / getTotalCompactingKVs();
   }
 
   /**
@@ -72,7 +75,12 @@ public class CompactionProgress {
   /**
    * @return the total compacting key values in currently running compaction
    */
-  public long getTotalCompactingKvs() {
+  public long getTotalCompactingKVs() {
+    if (totalCompactingKVs < currentCompactedKVs) {
+      LOG.warn("totalCompactingKVs={} less than currentCompactedKVs={}",
+          totalCompactingKVs, currentCompactedKVs);
+      return currentCompactedKVs;
+    }
     return totalCompactingKVs;
   }
 
@@ -92,7 +100,7 @@ public class CompactionProgress {
 
   @Override
   public String toString() {
-    return String.format("%d/%d (%.2f%%)", currentCompactedKVs, totalCompactingKVs,
+    return String.format("%d/%d (%.2f%%)", currentCompactedKVs, getTotalCompactingKVs(),
       100 * getProgressPct());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61b55166/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
index ee717f9..4fdd6b3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
@@ -221,7 +221,7 @@ public class TestMajorCompaction {
       if( progress != null ) {
         ++storeCount;
         assertTrue(progress.currentCompactedKVs > 0);
-        assertTrue(progress.totalCompactingKVs > 0);
+        assertTrue(progress.getTotalCompactingKVs() > 0);
       }
       assertTrue(storeCount > 0);
     }