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/02 08:53:08 UTC

[01/41] hbase git commit: HBASE-19868 TestCoprocessorWhitelistMasterObserver is flakey [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19064 c40ddd645 -> ab73dd502 (forced update)


HBASE-19868 TestCoprocessorWhitelistMasterObserver is flakey


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

Branch: refs/heads/HBASE-19064
Commit: 98c9d8f7a1c8155d41fafed0b9ca792cc5fc7098
Parents: 34c6c99
Author: Michael Stack <st...@apache.org>
Authored: Mon Jan 29 21:33:16 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Jan 29 21:35:28 2018 -0800

----------------------------------------------------------------------
 .../access/TestCoprocessorWhitelistMasterObserver.java       | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/98c9d8f7/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
index 1686ba3..c161b9e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
@@ -103,7 +103,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
         CoprocessorWhitelistMasterObserver.CP_COPROCESSOR_WHITELIST_PATHS_KEY,
         whitelistedPaths);
     // set retries low to raise exception quickly
-    conf.setInt("hbase.client.retries.number", 1);
+    conf.setInt("hbase.client.retries.number", 5);
     UTIL.startMiniCluster();
     UTIL.createTable(TEST_TABLE, new byte[][] { TEST_FAMILY });
     UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
@@ -137,7 +137,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
   private static void negativeTestCase(String[] whitelistedPaths,
       String coprocessorPath) throws Exception {
     Configuration conf = UTIL.getConfiguration();
-    conf.setInt("hbase.client.retries.number", 1);
+    conf.setInt("hbase.client.retries.number", 5);
     // load coprocessor under test
     conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
         CoprocessorWhitelistMasterObserver.class.getName());
@@ -269,7 +269,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
     conf.setStrings(CoprocessorWhitelistMasterObserver.CP_COPROCESSOR_WHITELIST_PATHS_KEY,
         new String[]{});
     // set retries low to raise exception quickly
-    conf.setInt("hbase.client.retries.number", 1);
+    conf.setInt("hbase.client.retries.number", 5);
     UTIL.startMiniCluster();
     HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
@@ -314,7 +314,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
     conf.setStrings(CoprocessorWhitelistMasterObserver.CP_COPROCESSOR_WHITELIST_PATHS_KEY,
         new String[]{});
     // set retries low to raise exception quickly
-    conf.setInt("hbase.client.retries.number", 1);
+    conf.setInt("hbase.client.retries.number", 5);
     UTIL.startMiniCluster();
     HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);


[30/41] hbase git commit: HBASE-19895 Add keepDeletedCells option in ScanOptions for customizing scanInfo in pre-hooks (Ankit Singhal)

Posted by zh...@apache.org.
HBASE-19895 Add keepDeletedCells option in ScanOptions for customizing scanInfo in pre-hooks (Ankit Singhal)


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

Branch: refs/heads/HBASE-19064
Commit: a11258599e7412ea4867ef850cd67bb9e7bd8e67
Parents: 99b9fff
Author: tedyu <yu...@gmail.com>
Authored: Thu Feb 1 18:27:18 2018 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Feb 1 18:27:18 2018 -0800

----------------------------------------------------------------------
 .../regionserver/CustomizedScanInfoBuilder.java | 18 +++-
 .../hadoop/hbase/regionserver/ScanInfo.java     |  9 +-
 .../hadoop/hbase/regionserver/ScanOptions.java  |  5 ++
 .../hbase/regionserver/TestCompaction.java      |  2 +-
 .../TestDefaultCompactSelection.java            |  3 +-
 .../hbase/regionserver/TestMajorCompaction.java | 88 +++++++++++++-------
 6 files changed, 87 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a1125859/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java
index c3d5e57..b791010 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomizedScanInfoBuilder.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -31,6 +32,8 @@ public class CustomizedScanInfoBuilder implements ScanOptions {
 
   private Long ttl;
 
+  private KeepDeletedCells keepDeletedCells = null;
+
   public CustomizedScanInfoBuilder(ScanInfo scanInfo) {
     this.scanInfo = scanInfo;
   }
@@ -56,14 +59,25 @@ public class CustomizedScanInfoBuilder implements ScanOptions {
   }
 
   public ScanInfo build() {
-    if (maxVersions == null && ttl == null) {
+    if (maxVersions == null && ttl == null && keepDeletedCells == null) {
       return scanInfo;
     }
-    return scanInfo.customize(getMaxVersions(), getTTL());
+    return scanInfo.customize(getMaxVersions(), getTTL(), getKeepDeletedCells());
   }
 
   @Override
   public String toString() {
     return "ScanOptions [maxVersions=" + getMaxVersions() + ", TTL=" + getTTL() + "]";
   }
+
+  @Override
+  public void setKeepDeletedCells(KeepDeletedCells keepDeletedCells) {
+    this.keepDeletedCells = keepDeletedCells;
+  }
+
+  @Override
+  public KeepDeletedCells getKeepDeletedCells() {
+    return keepDeletedCells != null ? keepDeletedCells : scanInfo.getKeepDeletedCells();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1125859/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java
index 419afff..2fde311 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java
@@ -171,10 +171,11 @@ public class ScanInfo {
   }
 
   /**
-   * Used for CP users for customizing max versions and ttl.
+   * Used for CP users for customizing max versions, ttl and keepDeletedCells.
    */
-  ScanInfo customize(int maxVersions, long ttl) {
-    return new ScanInfo(family, minVersions, maxVersions, ttl, keepDeletedCells, ttl, comparator,
-        ttl, usePread, maxVersions, parallelSeekEnabled, ttl, newVersionBehavior);
+  ScanInfo customize(int maxVersions, long ttl, KeepDeletedCells keepDeletedCells) {
+    return new ScanInfo(family, minVersions, maxVersions, ttl, keepDeletedCells, timeToPurgeDeletes,
+        comparator, tableMaxRowSize, usePread, cellsPerTimeoutCheck, parallelSeekEnabled,
+        preadMaxBytes, newVersionBehavior);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1125859/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanOptions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanOptions.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanOptions.java
index 5a35d51..aca857a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanOptions.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanOptions.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
@@ -59,4 +60,8 @@ public interface ScanOptions {
   long getTTL();
 
   void setTTL(long ttl);
+
+  void setKeepDeletedCells(KeepDeletedCells keepDeletedCells);
+
+  KeepDeletedCells getKeepDeletedCells();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1125859/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index 75110fd..6fe76d8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -218,7 +218,7 @@ public class TestCompaction {
       final int ttl = 1000;
       for (HStore store : this.r.stores.values()) {
         ScanInfo old = store.getScanInfo();
-        ScanInfo si = old.customize(old.getMaxVersions(), ttl);
+        ScanInfo si = old.customize(old.getMaxVersions(), ttl, old.getKeepDeletedCells());
         store.setScanInfo(si);
       }
       Thread.sleep(ttl);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1125859/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
index 3c3bbd8..9c45a83 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
@@ -170,7 +170,8 @@ public class TestDefaultCompactSelection extends TestCompactionPolicy {
   public void testCompactionEmptyHFile() throws IOException {
     // Set TTL
     ScanInfo oldScanInfo = store.getScanInfo();
-    ScanInfo newScanInfo = oldScanInfo.customize(oldScanInfo.getMaxVersions(), 600);
+    ScanInfo newScanInfo = oldScanInfo.customize(oldScanInfo.getMaxVersions(), 600,
+        oldScanInfo.getKeepDeletedCells());
     store.setScanInfo(newScanInfo);
     // Do not compact empty store file
     List<HStoreFile> candidates = sfCreate(0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1125859/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 e261886..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
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
@@ -134,36 +135,20 @@ public class TestMajorCompaction {
    * Test that on a major compaction, if all cells are expired or deleted, then
    * we'll end up with no product.  Make sure scanner over region returns
    * right answer in this case - and that it just basically works.
-   * @throws IOException
+   * @throws IOException exception encountered
    */
   @Test
   public void testMajorCompactingToNoOutput() throws IOException {
-    createStoreFile(r);
-    for (int i = 0; i < compactionThreshold; i++) {
-      createStoreFile(r);
-    }
-    // Now delete everything.
-    InternalScanner s = r.getScanner(new Scan());
-    do {
-      List<Cell> results = new ArrayList<>();
-      boolean result = s.next(results);
-      r.delete(new Delete(CellUtil.cloneRow(results.get(0))));
-      if (!result) break;
-    } while(true);
-    s.close();
-    // Flush
-    r.flush(true);
-    // Major compact.
-    r.compact(true);
-    s = r.getScanner(new Scan());
-    int counter = 0;
-    do {
-      List<Cell> results = new ArrayList<>();
-      boolean result = s.next(results);
-      if (!result) break;
-      counter++;
-    } while(true);
-    assertEquals(0, counter);
+    testMajorCompactingWithDeletes(KeepDeletedCells.FALSE);
+  }
+
+  /**
+   * Test that on a major compaction,Deleted cells are retained if keep deleted cells is set to true
+   * @throws IOException exception encountered
+   */
+  @Test
+  public void testMajorCompactingWithKeepDeletedCells() throws IOException {
+    testMajorCompactingWithDeletes(KeepDeletedCells.TRUE);
   }
 
   /**
@@ -298,7 +283,7 @@ public class TestMajorCompaction {
     final int ttl = 1000;
     for (HStore store : r.getStores()) {
       ScanInfo old = store.getScanInfo();
-      ScanInfo si = old.customize(old.getMaxVersions(), ttl);
+      ScanInfo si = old.customize(old.getMaxVersions(), ttl, old.getKeepDeletedCells());
       store.setScanInfo(si);
     }
     Thread.sleep(1000);
@@ -471,7 +456,9 @@ public class TestMajorCompaction {
       boolean result = s.next(results);
       assertTrue(!results.isEmpty());
       r.delete(new Delete(CellUtil.cloneRow(results.get(0))));
-      if (!result) break;
+      if (!result) {
+        break;
+      }
     } while (true);
     s.close();
     // Flush
@@ -485,10 +472,51 @@ public class TestMajorCompaction {
     do {
       List<Cell> results = new ArrayList<>();
       boolean result = s.next(results);
-      if (!result) break;
+      if (!result) {
+        break;
+      }
       counter++;
     } while (true);
     s.close();
     assertEquals(0, counter);
   }
+
+  private void testMajorCompactingWithDeletes(KeepDeletedCells keepDeletedCells)
+      throws IOException {
+    createStoreFile(r);
+    for (int i = 0; i < compactionThreshold; i++) {
+      createStoreFile(r);
+    }
+    // Now delete everything.
+    InternalScanner s = r.getScanner(new Scan());
+    int originalCount = 0;
+    do {
+      List<Cell> results = new ArrayList<>();
+      boolean result = s.next(results);
+      r.delete(new Delete(CellUtil.cloneRow(results.get(0))));
+      if (!result) break;
+      originalCount++;
+    } while (true);
+    s.close();
+    // Flush
+    r.flush(true);
+
+    for (HStore store : this.r.stores.values()) {
+      ScanInfo old = store.getScanInfo();
+      ScanInfo si = old.customize(old.getMaxVersions(), old.getTtl(), keepDeletedCells);
+      store.setScanInfo(si);
+    }
+    // Major compact.
+    r.compact(true);
+    s = r.getScanner(new Scan().setRaw(true));
+    int counter = 0;
+    do {
+      List<Cell> results = new ArrayList<>();
+      boolean result = s.next(results);
+      if (!result) break;
+      counter++;
+    } while (true);
+    assertEquals(keepDeletedCells == KeepDeletedCells.TRUE ? originalCount : 0, counter);
+
+  }
 }


[16/41] hbase git commit: HBASE-19911 Convert some tests from small to medium because they are timing out: TestNettyRpcServer, TestClientClusterStatus

Posted by zh...@apache.org.
HBASE-19911 Convert some tests from small to medium because they are timing out: TestNettyRpcServer, TestClientClusterStatus


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

Branch: refs/heads/HBASE-19064
Commit: b9cb1187b50d0ee1bdfb0e2e417d2eae6982305d
Parents: e17529b
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 31 20:21:57 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jan 31 20:23:14 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/TestClientClusterStatus.java    | 4 ++--
 .../java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java     | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b9cb1187/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 bfe28e4..bfdae8a 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
@@ -34,7 +34,7 @@ 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.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
+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.junit.AfterClass;
@@ -47,7 +47,7 @@ import org.junit.experimental.categories.Category;
 /**
  * Test the ClusterStatus.
  */
-@Category(SmallTests.class)
+@Category(MediumTests.class)
 public class TestClientClusterStatus {
 
   @ClassRule

http://git-wip-us.apache.org/repos/asf/hbase/blob/b9cb1187/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java
index f6b4613..d0ef055 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyRpcServer.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RPCTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -42,7 +42,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-@Category({ RPCTests.class, SmallTests.class })
+@Category({ RPCTests.class, MediumTests.class })
 public class TestNettyRpcServer {
 
   @ClassRule


[38/41] 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/9e94d15f/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/9e94d15f/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/9e94d15f/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 4a74646..28edd6d 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -394,6 +394,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/9e94d15f/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/9e94d15f/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/9e94d15f/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 }


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

Branch: refs/heads/HBASE-19064
Commit: 9e94d15f484d4ea4a625274ca38565eadbbc71d0
Parents: 7a3ad0f
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Jan 22 11:44:49 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 2 16:42:15 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/9e94d15f/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/9e94d15f/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/9e94d15f/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/9e94d15f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 8807884..5407c6d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -123,6 +123,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 
@@ -1724,6 +1726,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
           MasterProtos.ClearDeadServersRequest request) throws ServiceException {
         return stub.clearDeadServers(controller, request);
       }
+
+      @Override
+      public TransitReplicationPeerSyncReplicationStateResponse
+        transitReplicationPeerSyncReplicationState(RpcController controller,
+          TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+        return stub.transitReplicationPeerSyncReplicationState(controller, request);
+      }
     };
   }
 

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e94d15f/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/9e94d15f/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 ae676ea..b3af029 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/9e94d15f/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/9e94d15f/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/9e94d15f/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/9e94d15f/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/9e94d15f/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/9e94d15f/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/9e94d15f/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/9e94d15f/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 dc1763c..ed8db74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -135,6 +135,7 @@ import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
 import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
+import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure;
 import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
@@ -169,6 +170,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader;
@@ -3415,6 +3417,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/9e94d15f/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 8396145..70c8647 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/9e94d15f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 377a9c6..f5bf117 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
@@ -285,6 +286,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@@ -1958,6 +1961,20 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public TransitReplicationPeerSyncReplicationStateResponse
+    transitReplicationPeerSyncReplicationState(RpcController controller,
+      TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+    try {
+      long procId = master.transitReplicationPeerSyncReplicationState(request.getPeerId(),
+        SyncReplicationState.valueOf(request.getSyncReplicationState().getNumber()));
+      return TransitReplicationPeerSyncReplicationStateResponse.newBuilder().setProcId(procId)
+        .build();
+    } catch (ReplicationException | IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
       ListReplicationPeersRequest request) throws ServiceException {
     ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder();

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/9e94d15f/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 a8afe2d..350f4ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -601,7 +602,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/9e94d15f/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index ab9fb17..b2b5f1c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -117,6 +117,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
@@ -2938,6 +2939,21 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Test
+  public void testTransitSyncReplicationPeerState() throws Exception {
+    AccessTestAction action = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preTransitReplicationPeerSyncReplicationState(
+          ObserverContextImpl.createAndPrepare(CP_ENV), "test", SyncReplicationState.NONE);
+        return null;
+      }
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
+
+  @Test
   public void testListReplicationPeers() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override


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

Branch: refs/heads/HBASE-19064
Commit: ab73dd50279a7bbc823ec4cf9ff99e4ae1352af0
Parents: 6adf213
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 26 16:50:48 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 2 16:45:59 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/ab73dd50/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/ab73dd50/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/ab73dd50/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 659be2a..6ebcc89 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -1878,10 +1878,9 @@ public final class RequestConverter {
   }
 
   public static TransitReplicationPeerSyncReplicationStateRequest
-    buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
-      SyncReplicationState state) {
+      buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
+          SyncReplicationState state) {
     return TransitReplicationPeerSyncReplicationStateRequest.newBuilder().setPeerId(peerId)
-      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
-      .build();
+        .setSyncReplicationState(ReplicationPeerConfigUtil.toSyncReplicationState(state)).build();
   }
 }

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab73dd50/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/ab73dd50/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 d8a6442..7574d61 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
@@ -176,7 +176,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);
 


[02/41] hbase git commit: HBASE-19891 Up nightly test run timeout from 6 hours to 8

Posted by zh...@apache.org.
HBASE-19891 Up nightly test run timeout from 6 hours to 8


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

Branch: refs/heads/HBASE-19064
Commit: 9b8d7e0aefe29fea474e06ab21f9288245ac3bf5
Parents: 98c9d8f
Author: Michael Stack <st...@apache.org>
Authored: Mon Jan 29 22:14:47 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Jan 29 22:14:47 2018 -0800

----------------------------------------------------------------------
 dev-support/Jenkinsfile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9b8d7e0a/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 14092d5..c7adfa5 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -25,7 +25,7 @@ pipeline {
   }
   options {
     buildDiscarder(logRotator(numToKeepStr: '30'))
-    timeout (time: 6, unit: 'HOURS')
+    timeout (time: 8, unit: 'HOURS')
     timestamps()
     skipDefaultCheckout()
   }


[25/41] hbase git commit: HBASE-19147 TestCacheOnWrite Times Out

Posted by zh...@apache.org.
HBASE-19147 TestCacheOnWrite Times Out


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

Branch: refs/heads/HBASE-19064
Commit: 3622bb0333ea1b7c05fd47e5e0fee61b82f08109
Parents: 0db7db3
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 1 16:14:51 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 16:14:51 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/3622bb03/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index 7244e95..19d845c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.testclassification.IOTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
@@ -78,7 +78,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
  * types: data blocks, non-root index blocks, and Bloom filter blocks.
  */
 @RunWith(Parameterized.class)
-@Category({IOTests.class, MediumTests.class})
+@Category({IOTests.class, LargeTests.class})
 public class TestCacheOnWrite {
 
   @ClassRule


[15/41] hbase git commit: HBASE-19910 TestBucketCache TimesOut

Posted by zh...@apache.org.
HBASE-19910 TestBucketCache TimesOut


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

Branch: refs/heads/HBASE-19064
Commit: e17529ba738ff4669abbd549d01a55b0fab13b26
Parents: 28ee4be
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 31 18:17:01 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jan 31 18:17:01 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e17529ba/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
index 3c45958..c1824f2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.io.hfile.Cacheable;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.BucketSizeInfo;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.IndexStatistics;
 import org.apache.hadoop.hbase.testclassification.IOTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -59,7 +59,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
  * Tests will ensure that blocks' data correctness under several threads concurrency
  */
 @RunWith(Parameterized.class)
-@Category({ IOTests.class, SmallTests.class })
+@Category({ IOTests.class, MediumTests.class })
 public class TestBucketCache {
 
   @ClassRule


[11/41] hbase git commit: HBASE-19528 Major Compaction Tool; ADDENDUM

Posted by zh...@apache.org.
HBASE-19528 Major Compaction Tool; ADDENDUM

Add @ClassRule to new tests and rename those added here so they
align w/ how we name other Test classes.


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

Branch: refs/heads/HBASE-19064
Commit: 60827fc1ea93ce3c93ea9b86e618e419babed42f
Parents: 4b3b627
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 31 14:18:20 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jan 31 14:18:24 2018 -0800

----------------------------------------------------------------------
 .../compaction/MajorCompactionRequestTest.java  | 166 ------------------
 .../util/compaction/MajorCompactorTest.java     |  81 ---------
 .../compaction/TestMajorCompactionRequest.java  | 170 +++++++++++++++++++
 .../util/compaction/TestMajorCompactor.java     |  86 ++++++++++
 4 files changed, 256 insertions(+), 247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/60827fc1/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequestTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequestTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequestTest.java
deleted file mode 100644
index c5ce4e3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequestTest.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.util.compaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Optional;
-import java.util.Set;
-import java.util.stream.Collectors;
-import org.apache.commons.lang.RandomStringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-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.client.Connection;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.ArgumentMatchers.isA;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
-@Category({SmallTests.class})
-public class MajorCompactionRequestTest {
-
-  private static final HBaseTestingUtility UTILITY = new HBaseTestingUtility();
-  private static final String FAMILY = "a";
-  private Path rootRegionDir;
-  private Path regionStoreDir;
-
-  @Before public void setUp() throws Exception {
-    rootRegionDir = UTILITY.getDataTestDirOnTestFS("MajorCompactionRequestTest");
-    regionStoreDir = new Path(rootRegionDir, FAMILY);
-  }
-
-  @Test public void testStoresNeedingCompaction() throws Exception {
-    // store files older than timestamp
-    List<StoreFileInfo> storeFiles = mockStoreFiles(regionStoreDir, 5, 10);
-    MajorCompactionRequest request = makeMockRequest(100, storeFiles, false);
-    Optional<MajorCompactionRequest> result =
-        request.createRequest(mock(Configuration.class), Sets.newHashSet(FAMILY));
-    assertTrue(result.isPresent());
-
-    // store files newer than timestamp
-    storeFiles = mockStoreFiles(regionStoreDir, 5, 101);
-    request = makeMockRequest(100, storeFiles, false);
-    result = request.createRequest(mock(Configuration.class), Sets.newHashSet(FAMILY));
-    assertFalse(result.isPresent());
-  }
-
-  @Test public void testIfWeHaveNewReferenceFilesButOldStoreFiles() throws Exception {
-    // this tests that reference files that are new, but have older timestamps for the files
-    // they reference still will get compacted.
-    TableName table = TableName.valueOf("MajorCompactorTest");
-    TableDescriptor htd = UTILITY.createTableDescriptor(table, Bytes.toBytes(FAMILY));
-    RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
-    HRegion region =
-        HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, UTILITY.getConfiguration(), htd);
-
-    Configuration configuration = mock(Configuration.class);
-    // the reference file timestamp is newer
-    List<StoreFileInfo> storeFiles = mockStoreFiles(regionStoreDir, 4, 101);
-    List<Path> paths = storeFiles.stream().map(StoreFileInfo::getPath).collect(Collectors.toList());
-    // the files that are referenced are older, thus we still compact.
-    HRegionFileSystem fileSystem =
-        mockFileSystem(region.getRegionInfo(), true, storeFiles, 50);
-    MajorCompactionRequest majorCompactionRequest = spy(new MajorCompactionRequest(configuration,
-        region.getRegionInfo(), Sets.newHashSet(FAMILY), 100));
-    doReturn(mock(Connection.class)).when(majorCompactionRequest).getConnection(eq(configuration));
-    doReturn(paths).when(majorCompactionRequest).getReferenceFilePaths(any(FileSystem.class),
-        any(Path.class));
-    doReturn(fileSystem).when(majorCompactionRequest).getFileSystem(any(Connection.class));
-    Set<String> result = majorCompactionRequest.getStoresRequiringCompaction(Sets.newHashSet("a"));
-    assertEquals(FAMILY, Iterables.getOnlyElement(result));
-  }
-
-  private HRegionFileSystem mockFileSystem(RegionInfo info, boolean hasReferenceFiles,
-      List<StoreFileInfo> storeFiles) throws IOException {
-    long timestamp = storeFiles.stream().findFirst().get().getModificationTime();
-    return mockFileSystem(info, hasReferenceFiles, storeFiles, timestamp);
-  }
-
-  private HRegionFileSystem mockFileSystem(RegionInfo info, boolean hasReferenceFiles,
-      List<StoreFileInfo> storeFiles, long referenceFileTimestamp) throws IOException {
-    FileSystem fileSystem = mock(FileSystem.class);
-    if (hasReferenceFiles) {
-      FileStatus fileStatus = mock(FileStatus.class);
-      doReturn(referenceFileTimestamp).when(fileStatus).getModificationTime();
-      doReturn(fileStatus).when(fileSystem).getFileLinkStatus(isA(Path.class));
-    }
-    HRegionFileSystem mockSystem = mock(HRegionFileSystem.class);
-    doReturn(info).when(mockSystem).getRegionInfo();
-    doReturn(regionStoreDir).when(mockSystem).getStoreDir(FAMILY);
-    doReturn(hasReferenceFiles).when(mockSystem).hasReferences(anyString());
-    doReturn(storeFiles).when(mockSystem).getStoreFiles(anyString());
-    doReturn(fileSystem).when(mockSystem).getFileSystem();
-    return mockSystem;
-  }
-
-  private List<StoreFileInfo> mockStoreFiles(Path regionStoreDir, int howMany, long timestamp)
-      throws IOException {
-    List<StoreFileInfo> infos = Lists.newArrayList();
-    int i = 0;
-    while (i < howMany) {
-      StoreFileInfo storeFileInfo = mock(StoreFileInfo.class);
-      doReturn(timestamp).doReturn(timestamp).when(storeFileInfo).getModificationTime();
-      doReturn(new Path(regionStoreDir, RandomStringUtils.randomAlphabetic(10))).when(storeFileInfo)
-          .getPath();
-      infos.add(storeFileInfo);
-      i++;
-    }
-    return infos;
-  }
-
-  private MajorCompactionRequest makeMockRequest(long timestamp, List<StoreFileInfo> storeFiles,
-      boolean references) throws IOException {
-    Configuration configuration = mock(Configuration.class);
-    RegionInfo regionInfo = mock(RegionInfo.class);
-    when(regionInfo.getEncodedName()).thenReturn("HBase");
-    when(regionInfo.getTable()).thenReturn(TableName.valueOf("foo"));
-    MajorCompactionRequest request =
-        new MajorCompactionRequest(configuration, regionInfo, Sets.newHashSet("a"), timestamp);
-    MajorCompactionRequest spy = spy(request);
-    HRegionFileSystem fileSystem = mockFileSystem(regionInfo, references, storeFiles);
-    doReturn(fileSystem).when(spy).getFileSystem(isA(Connection.class));
-    doReturn(mock(Connection.class)).when(spy).getConnection(eq(configuration));
-    return spy;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/60827fc1/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTest.java
deleted file mode 100644
index 3fb37ec..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTest.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.util.compaction;
-
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-import org.junit.After;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-
-@Category({ MiscTests.class, MediumTests.class })
-public class MajorCompactorTest {
-
-  public static final byte[] FAMILY = Bytes.toBytes("a");
-  private HBaseTestingUtility utility;
-
-  @Before public void setUp() throws Exception {
-    utility = new HBaseTestingUtility();
-    utility.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 10);
-    utility.startMiniCluster();
-  }
-
-  @After public void tearDown() throws Exception {
-    utility.shutdownMiniCluster();
-  }
-
-  @Test public void testCompactingATable() throws Exception {
-    TableName tableName = TableName.valueOf("MajorCompactorTest");
-    utility.createMultiRegionTable(tableName, FAMILY, 5);
-    utility.waitTableAvailable(tableName);
-    Connection connection = utility.getConnection();
-    Table table = connection.getTable(tableName);
-    // write data and flush multiple store files:
-    for (int i = 0; i < 5; i++) {
-      utility.loadRandomRows(table, FAMILY, 50, 100);
-      utility.flush(tableName);
-    }
-    table.close();
-    int numberOfRegions = utility.getAdmin().getRegions(tableName).size();
-    int numHFiles = utility.getNumHFiles(tableName, FAMILY);
-    // we should have a table with more store files than we would before we major compacted.
-    assertTrue(numberOfRegions < numHFiles);
-
-    MajorCompactor compactor =
-        new MajorCompactor(utility.getConfiguration(), tableName,
-            Sets.newHashSet(Bytes.toString(FAMILY)), 1, System.currentTimeMillis(), 200);
-    compactor.initializeWorkQueues();
-    compactor.compactAllRegions();
-    compactor.shutdown();
-
-    // verify that the store has been completely major compacted.
-    numberOfRegions = utility.getAdmin().getRegions(tableName).size();
-    numHFiles = utility.getNumHFiles(tableName, FAMILY);
-    assertEquals(numHFiles, numberOfRegions);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/60827fc1/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/TestMajorCompactionRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/TestMajorCompactionRequest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/TestMajorCompactionRequest.java
new file mode 100644
index 0000000..b626481
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/TestMajorCompactionRequest.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.compaction;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+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.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isA;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+@Category({SmallTests.class})
+public class TestMajorCompactionRequest {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMajorCompactionRequest.class);
+
+  private static final HBaseTestingUtility UTILITY = new HBaseTestingUtility();
+  private static final String FAMILY = "a";
+  private Path rootRegionDir;
+  private Path regionStoreDir;
+
+  @Before public void setUp() throws Exception {
+    rootRegionDir = UTILITY.getDataTestDirOnTestFS("TestMajorCompactionRequest");
+    regionStoreDir = new Path(rootRegionDir, FAMILY);
+  }
+
+  @Test public void testStoresNeedingCompaction() throws Exception {
+    // store files older than timestamp
+    List<StoreFileInfo> storeFiles = mockStoreFiles(regionStoreDir, 5, 10);
+    MajorCompactionRequest request = makeMockRequest(100, storeFiles, false);
+    Optional<MajorCompactionRequest> result =
+        request.createRequest(mock(Configuration.class), Sets.newHashSet(FAMILY));
+    assertTrue(result.isPresent());
+
+    // store files newer than timestamp
+    storeFiles = mockStoreFiles(regionStoreDir, 5, 101);
+    request = makeMockRequest(100, storeFiles, false);
+    result = request.createRequest(mock(Configuration.class), Sets.newHashSet(FAMILY));
+    assertFalse(result.isPresent());
+  }
+
+  @Test public void testIfWeHaveNewReferenceFilesButOldStoreFiles() throws Exception {
+    // this tests that reference files that are new, but have older timestamps for the files
+    // they reference still will get compacted.
+    TableName table = TableName.valueOf("TestMajorCompactor");
+    TableDescriptor htd = UTILITY.createTableDescriptor(table, Bytes.toBytes(FAMILY));
+    RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    HRegion region =
+        HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, UTILITY.getConfiguration(), htd);
+
+    Configuration configuration = mock(Configuration.class);
+    // the reference file timestamp is newer
+    List<StoreFileInfo> storeFiles = mockStoreFiles(regionStoreDir, 4, 101);
+    List<Path> paths = storeFiles.stream().map(StoreFileInfo::getPath).collect(Collectors.toList());
+    // the files that are referenced are older, thus we still compact.
+    HRegionFileSystem fileSystem =
+        mockFileSystem(region.getRegionInfo(), true, storeFiles, 50);
+    MajorCompactionRequest majorCompactionRequest = spy(new MajorCompactionRequest(configuration,
+        region.getRegionInfo(), Sets.newHashSet(FAMILY), 100));
+    doReturn(mock(Connection.class)).when(majorCompactionRequest).getConnection(eq(configuration));
+    doReturn(paths).when(majorCompactionRequest).getReferenceFilePaths(any(FileSystem.class),
+        any(Path.class));
+    doReturn(fileSystem).when(majorCompactionRequest).getFileSystem(any(Connection.class));
+    Set<String> result = majorCompactionRequest.getStoresRequiringCompaction(Sets.newHashSet("a"));
+    assertEquals(FAMILY, Iterables.getOnlyElement(result));
+  }
+
+  private HRegionFileSystem mockFileSystem(RegionInfo info, boolean hasReferenceFiles,
+      List<StoreFileInfo> storeFiles) throws IOException {
+    long timestamp = storeFiles.stream().findFirst().get().getModificationTime();
+    return mockFileSystem(info, hasReferenceFiles, storeFiles, timestamp);
+  }
+
+  private HRegionFileSystem mockFileSystem(RegionInfo info, boolean hasReferenceFiles,
+      List<StoreFileInfo> storeFiles, long referenceFileTimestamp) throws IOException {
+    FileSystem fileSystem = mock(FileSystem.class);
+    if (hasReferenceFiles) {
+      FileStatus fileStatus = mock(FileStatus.class);
+      doReturn(referenceFileTimestamp).when(fileStatus).getModificationTime();
+      doReturn(fileStatus).when(fileSystem).getFileLinkStatus(isA(Path.class));
+    }
+    HRegionFileSystem mockSystem = mock(HRegionFileSystem.class);
+    doReturn(info).when(mockSystem).getRegionInfo();
+    doReturn(regionStoreDir).when(mockSystem).getStoreDir(FAMILY);
+    doReturn(hasReferenceFiles).when(mockSystem).hasReferences(anyString());
+    doReturn(storeFiles).when(mockSystem).getStoreFiles(anyString());
+    doReturn(fileSystem).when(mockSystem).getFileSystem();
+    return mockSystem;
+  }
+
+  private List<StoreFileInfo> mockStoreFiles(Path regionStoreDir, int howMany, long timestamp)
+      throws IOException {
+    List<StoreFileInfo> infos = Lists.newArrayList();
+    int i = 0;
+    while (i < howMany) {
+      StoreFileInfo storeFileInfo = mock(StoreFileInfo.class);
+      doReturn(timestamp).doReturn(timestamp).when(storeFileInfo).getModificationTime();
+      doReturn(new Path(regionStoreDir, RandomStringUtils.randomAlphabetic(10))).when(storeFileInfo)
+          .getPath();
+      infos.add(storeFileInfo);
+      i++;
+    }
+    return infos;
+  }
+
+  private MajorCompactionRequest makeMockRequest(long timestamp, List<StoreFileInfo> storeFiles,
+      boolean references) throws IOException {
+    Configuration configuration = mock(Configuration.class);
+    RegionInfo regionInfo = mock(RegionInfo.class);
+    when(regionInfo.getEncodedName()).thenReturn("HBase");
+    when(regionInfo.getTable()).thenReturn(TableName.valueOf("foo"));
+    MajorCompactionRequest request =
+        new MajorCompactionRequest(configuration, regionInfo, Sets.newHashSet("a"), timestamp);
+    MajorCompactionRequest spy = spy(request);
+    HRegionFileSystem fileSystem = mockFileSystem(regionInfo, references, storeFiles);
+    doReturn(fileSystem).when(spy).getFileSystem(isA(Connection.class));
+    doReturn(mock(Connection.class)).when(spy).getConnection(eq(configuration));
+    return spy;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/60827fc1/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/TestMajorCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/TestMajorCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/TestMajorCompactor.java
new file mode 100644
index 0000000..ccf0146
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/TestMajorCompactor.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.compaction;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.junit.After;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+
+@Category({ MiscTests.class, MediumTests.class })
+public class TestMajorCompactor {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMajorCompactor.class);
+
+  public static final byte[] FAMILY = Bytes.toBytes("a");
+  private HBaseTestingUtility utility;
+
+  @Before public void setUp() throws Exception {
+    utility = new HBaseTestingUtility();
+    utility.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 10);
+    utility.startMiniCluster();
+  }
+
+  @After public void tearDown() throws Exception {
+    utility.shutdownMiniCluster();
+  }
+
+  @Test public void testCompactingATable() throws Exception {
+    TableName tableName = TableName.valueOf("TestMajorCompactor");
+    utility.createMultiRegionTable(tableName, FAMILY, 5);
+    utility.waitTableAvailable(tableName);
+    Connection connection = utility.getConnection();
+    Table table = connection.getTable(tableName);
+    // write data and flush multiple store files:
+    for (int i = 0; i < 5; i++) {
+      utility.loadRandomRows(table, FAMILY, 50, 100);
+      utility.flush(tableName);
+    }
+    table.close();
+    int numberOfRegions = utility.getAdmin().getRegions(tableName).size();
+    int numHFiles = utility.getNumHFiles(tableName, FAMILY);
+    // we should have a table with more store files than we would before we major compacted.
+    assertTrue(numberOfRegions < numHFiles);
+
+    MajorCompactor compactor =
+        new MajorCompactor(utility.getConfiguration(), tableName,
+            Sets.newHashSet(Bytes.toString(FAMILY)), 1, System.currentTimeMillis(), 200);
+    compactor.initializeWorkQueues();
+    compactor.compactAllRegions();
+    compactor.shutdown();
+
+    // verify that the store has been completely major compacted.
+    numberOfRegions = utility.getAdmin().getRegions(tableName).size();
+    numHFiles = utility.getNumHFiles(tableName, FAMILY);
+    assertEquals(numHFiles, numberOfRegions);
+  }
+}
\ No newline at end of file


[07/41] hbase git commit: HBASE-19728 Add lock to filesCompacting in all place.

Posted by zh...@apache.org.
HBASE-19728 Add lock to filesCompacting in all place.


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

Branch: refs/heads/HBASE-19064
Commit: f7faad0477d66de1e9d89a00fe9c4d8b0b80f198
Parents: 08bcd64
Author: binlijin <bi...@gmail.com>
Authored: Wed Jan 31 14:05:30 2018 +0800
Committer: binlijin <bi...@gmail.com>
Committed: Wed Jan 31 14:05:30 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/f7faad04/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 f228d44..b00758f 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
@@ -1451,7 +1451,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
     this.lock.writeLock().lock();
     try {
       this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
-      filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock();
+      synchronized (filesCompacting) {
+        filesCompacting.removeAll(compactedFiles);
+      }
     } finally {
       this.lock.writeLock().unlock();
     }
@@ -2306,7 +2308,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
 
   @Override
   public boolean needsCompaction() {
-    return this.storeEngine.needsCompaction(this.filesCompacting);
+    List<HStoreFile> filesCompactingClone = null;
+    synchronized (filesCompacting) {
+      filesCompactingClone = Lists.newArrayList(filesCompacting);
+    }
+    return this.storeEngine.needsCompaction(filesCompactingClone);
   }
 
   /**


[31/41] hbase git commit: HBASE-19901 HBASE-19901 Up yetus proclimit on nightlies; REAPPLY TO TEST

Posted by zh...@apache.org.
HBASE-19901 HBASE-19901 Up yetus proclimit on nightlies; REAPPLY TO TEST


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

Branch: refs/heads/HBASE-19064
Commit: cb7bfc21da4e2658602a9a2b2080c4d883287dd1
Parents: a112585
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 1 20:26:46 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 20:27:28 2018 -0800

----------------------------------------------------------------------
 dev-support/hbase-personality.sh   | 8 ++++++++
 dev-support/hbase_nightly_yetus.sh | 2 ++
 2 files changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cb7bfc21/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index e24930d..9064f0b 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -64,6 +64,14 @@ function personality_globals
 
   # Override the maven options
   MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
+
+  # Yetus 0.7.0 enforces limits. Default proclimit is 1000.
+  # Up it. See HBASE-19902 for how we arrived at this number.
+  PROCLIMIT=10000
+
+  # Set docker container to run with 20g. Default is 4g in yetus.
+  # See HBASE-19902 for how we arrived at 20g.
+  DOCKERMEMLIMIT=20g
 }
 
 ## @description  Parse extra arguments required by personalities, if any.

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb7bfc21/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index 4e67354..651a2e2 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -67,6 +67,8 @@ YETUS_ARGS=("--whitespace-tabs-ignore-list=${WHITESPACE_IGNORE_LIST}" "${YETUS_A
 YETUS_ARGS=("--sentinel" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--proclimit=${PROCLIMIT}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--dockermemlimit=${DOCKERMEMLIMIT}" "${YETUS_ARGS[@]}")
 
 # Currently, flaky list is calculated only for master branch.
 UNDERSCORED_BRANCH_NAME=$(echo ${BRANCH_NAME} | tr '.-' '_')


[29/41] hbase git commit: HBASE-19841 Every HTU should be local until DFS starts

Posted by zh...@apache.org.
HBASE-19841 Every HTU should be local until DFS starts


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

Branch: refs/heads/HBASE-19064
Commit: 99b9fff07bb2669792f9c1c8a796605971d02592
Parents: 18eec8c
Author: Mike Drob <md...@apache.org>
Authored: Mon Jan 29 14:32:39 2018 -0600
Committer: Mike Drob <md...@apache.org>
Committed: Thu Feb 1 20:24:11 2018 -0600

----------------------------------------------------------------------
 .../apache/hadoop/hbase/util/CommonFSUtils.java |   8 +-
 .../hadoop/hbase/HBaseCommonTestingUtility.java |   4 +-
 .../org/apache/hadoop/hbase/HBaseTestCase.java  |   2 +-
 .../hadoop/hbase/HBaseTestingUtility.java       | 118 +++++++++++--------
 .../TestRegionObserverScannerOpenHook.java      |   7 +-
 .../coprocessor/TestRegionObserverStacking.java |   4 +-
 .../hbase/io/asyncfs/TestLocalAsyncOutput.java  |   4 +-
 .../hadoop/hbase/regionserver/TestHStore.java   |   4 +-
 .../hbase/wal/WALPerformanceEvaluation.java     |   2 -
 hbase-server/src/test/resources/hbase-site.xml  |   9 --
 10 files changed, 86 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
index 9efec07..5b46de9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
@@ -394,7 +394,13 @@ public abstract class CommonFSUtils {
 
   public static FileSystem getWALFileSystem(final Configuration c) throws IOException {
     Path p = getWALRootDir(c);
-    return p.getFileSystem(c);
+    FileSystem fs = p.getFileSystem(c);
+    // hadoop-core does fs caching, so need to propogate this if set
+    String enforceStreamCapability = c.get(UNSAFE_STREAM_CAPABILITY_ENFORCE);
+    if (enforceStreamCapability != null) {
+      fs.getConf().set(UNSAFE_STREAM_CAPABILITY_ENFORCE, enforceStreamCapability);
+    }
+    return fs;
   }
 
   private static boolean isValidWALRootDir(Path walDir, final Configuration c) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
index 693f9b2..74f653c 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
@@ -63,11 +63,11 @@ public class HBaseCommonTestingUtility {
   protected Configuration conf;
 
   public HBaseCommonTestingUtility() {
-    this(HBaseConfiguration.create());
+    this(null);
   }
 
   public HBaseCommonTestingUtility(Configuration conf) {
-    this.conf = conf;
+    this.conf = (conf == null ? HBaseConfiguration.create() : conf);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
index 5f2ffb2..96f3d55 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
@@ -72,7 +72,7 @@ public abstract class HBaseTestCase extends TestCase {
 
   protected final HBaseTestingUtility testUtil = new HBaseTestingUtility();
 
-  public volatile Configuration conf = HBaseConfiguration.create();
+  public volatile Configuration conf = testUtil.getConfiguration();
   public final FSTableDescriptors fsTableDescriptors;
   {
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 75abd5e..09e6935 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -119,6 +119,7 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.VisibilityLabelsCache;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
@@ -213,8 +214,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    */
   private volatile Connection connection;
 
-  private boolean localMode = false;
-
   /** Filesystem URI used for map-reduce mini-cluster setup */
   private static String FS_URI;
 
@@ -298,57 +297,82 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   public static final Collection<Object[]> BLOOM_AND_COMPRESSION_COMBINATIONS =
       bloomAndCompressionCombinations();
 
+
+  /**
+   * <p>Create an HBaseTestingUtility using a default configuration.
+   *
+   * <p>Initially, all tmp files are written to a local test data directory.
+   * Once {@link #startMiniDFSCluster} is called, either directly or via
+   * {@link #startMiniCluster()}, tmp data will be written to the DFS directory instead.
+   *
+   * <p>Previously, there was a distinction between the type of utility returned by
+   * {@link #createLocalHTU()} and this constructor; this is no longer the case. All
+   * HBaseTestingUtility objects will behave as local until a DFS cluster is started,
+   * at which point they will switch to using mini DFS for storage.
+   */
   public HBaseTestingUtility() {
     this(HBaseConfiguration.create());
   }
 
-  public HBaseTestingUtility(Configuration conf) {
+  /**
+   * <p>Create an HBaseTestingUtility using a given configuration.
+   *
+   * <p>Initially, all tmp files are written to a local test data directory.
+   * Once {@link #startMiniDFSCluster} is called, either directly or via
+   * {@link #startMiniCluster()}, tmp data will be written to the DFS directory instead.
+   *
+   * <p>Previously, there was a distinction between the type of utility returned by
+   * {@link #createLocalHTU()} and this constructor; this is no longer the case. All
+   * HBaseTestingUtility objects will behave as local until a DFS cluster is started,
+   * at which point they will switch to using mini DFS for storage.
+   *
+   * @param conf The configuration to use for further operations
+   */
+  public HBaseTestingUtility(@Nullable Configuration conf) {
     super(conf);
 
     // a hbase checksum verification failure will cause unit tests to fail
     ChecksumUtil.generateExceptionForChecksumFailureForTest(true);
 
-    // prevent contention for ports if other hbase thread(s) already running
+    // if conf is provided, prevent contention for ports if other hbase thread(s) are running
     if (conf != null) {
       if (conf.getInt(HConstants.MASTER_INFO_PORT, HConstants.DEFAULT_MASTER_INFOPORT)
               == HConstants.DEFAULT_MASTER_INFOPORT) {
         conf.setInt(HConstants.MASTER_INFO_PORT, -1);
-        LOG.debug("Config property " + HConstants.MASTER_INFO_PORT + " changed to -1");
+        LOG.debug("Config property {} changed to -1", HConstants.MASTER_INFO_PORT);
       }
       if (conf.getInt(HConstants.REGIONSERVER_PORT, HConstants.DEFAULT_REGIONSERVER_PORT)
               == HConstants.DEFAULT_REGIONSERVER_PORT) {
         conf.setInt(HConstants.REGIONSERVER_PORT, -1);
-        LOG.debug("Config property " + HConstants.REGIONSERVER_PORT + " changed to -1");
+        LOG.debug("Config property {} changed to -1", HConstants.REGIONSERVER_PORT);
       }
     }
+
+    // Every cluster is a local cluster until we start DFS
+    // Note that conf could be null, but this.conf will not be
+    String dataTestDir = getDataTestDir().toString();
+    this.conf.set("fs.defaultFS","file:///");
+    this.conf.set(HConstants.HBASE_DIR, "file://" + dataTestDir);
+    LOG.debug("Setting {} to {}", HConstants.HBASE_DIR, dataTestDir);
+    this.conf.setBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE,false);
   }
 
   /**
-   * Create an HBaseTestingUtility where all tmp files are written to the local test data dir.
-   * It is needed to properly base FSUtil.getRootDirs so that they drop temp files in the proper
-   * test dir.  Use this when you aren't using an Mini HDFS cluster.
-   * @return HBaseTestingUtility that use local fs for temp files.
+   * @deprecated use {@link HBaseTestingUtility#HBaseTestingUtility()} instead
+   * @return a normal HBaseTestingUtility
    */
+  @Deprecated
   public static HBaseTestingUtility createLocalHTU() {
-    Configuration c = HBaseConfiguration.create();
-    return createLocalHTU(c);
+    return new HBaseTestingUtility();
   }
 
   /**
-   * Create an HBaseTestingUtility where all tmp files are written to the local test data dir.
-   * It is needed to properly base FSUtil.getRootDirs so that they drop temp files in the proper
-   * test dir.  Use this when you aren't using an Mini HDFS cluster.
-   * @param c Configuration (will be modified)
-   * @return HBaseTestingUtility that use local fs for temp files.
+   * @deprecated use {@link HBaseTestingUtility#HBaseTestingUtility(Configuration)} instead
+   * @return a normal HBaseTestingUtility
    */
+  @Deprecated
   public static HBaseTestingUtility createLocalHTU(Configuration c) {
-    HBaseTestingUtility htu = new HBaseTestingUtility(c);
-    String dataTestDir = htu.getDataTestDir().toString();
-    htu.getConfiguration().set("fs.defaultFS","file:///");
-    htu.getConfiguration().set(HConstants.HBASE_DIR, "file://" + dataTestDir);
-    LOG.debug("Setting " + HConstants.HBASE_DIR + " to " + dataTestDir);
-    htu.localMode = true;
-    return htu;
+    return new HBaseTestingUtility(c);
   }
 
   /**
@@ -612,30 +636,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    */
   public MiniDFSCluster startMiniDFSCluster(int servers, final String hosts[])
   throws Exception {
-    createDirsAndSetProperties();
-    EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
-
-    // Error level to skip some warnings specific to the minicluster. See HBASE-4709
-    org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.util.MBeans.class).
-        setLevel(org.apache.log4j.Level.ERROR);
-    org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class).
-        setLevel(org.apache.log4j.Level.ERROR);
-
-    TraceUtil.initTracer(conf);
-
-    this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
-      true, null, null, hosts, null);
-
-    // Set this just-started cluster as our filesystem.
-    setFs();
-
-    // Wait for the cluster to be totally up
-    this.dfsCluster.waitClusterUp();
-
-    //reset the test directory for test file system
-    dataTestDirOnTestFS = null;
-
-    return this.dfsCluster;
+    return startMiniDFSCluster(servers, null, hosts);
   }
 
   private void setFs() throws IOException {
@@ -645,23 +646,38 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     }
     FileSystem fs = this.dfsCluster.getFileSystem();
     FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
+
+    // re-enable this check with dfs
+    conf.unset(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE);
   }
 
   public MiniDFSCluster startMiniDFSCluster(int servers, final  String racks[], String hosts[])
       throws Exception {
     createDirsAndSetProperties();
+    EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
+
+    // Error level to skip some warnings specific to the minicluster. See HBASE-4709
+    org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.util.MBeans.class).
+        setLevel(org.apache.log4j.Level.ERROR);
+    org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class).
+        setLevel(org.apache.log4j.Level.ERROR);
+
+    TraceUtil.initTracer(conf);
+
     this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
         true, null, racks, hosts, null);
 
     // Set this just-started cluster as our filesystem.
-    FileSystem fs = this.dfsCluster.getFileSystem();
-    FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
+    setFs();
 
     // Wait for the cluster to be totally up
     this.dfsCluster.waitClusterUp();
 
     //reset the test directory for test file system
     dataTestDirOnTestFS = null;
+    String dataTestDir = getDataTestDir().toString();
+    conf.set(HConstants.HBASE_DIR, dataTestDir);
+    LOG.debug("Setting {} to {}", HConstants.HBASE_DIR, dataTestDir);
 
     return this.dfsCluster;
   }
@@ -957,7 +973,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
     // Bring up mini dfs cluster. This spews a bunch of warnings about missing
     // scheme. Complaints are 'Scheme is undefined for build/test/data/dfs/name1'.
-    if(this.dfsCluster == null && !localMode) {
+    if(this.dfsCluster == null) {
       LOG.info("STARTING DFS");
       dfsCluster = startMiniDFSCluster(numDataNodes, dataNodeHosts);
     } else LOG.info("NOT STARTING DFS");

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index e7a9f61..bdea5fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -209,7 +208,8 @@ public class TestRegionObserverScannerOpenHook {
     byte[] A = Bytes.toBytes("A");
     byte[][] FAMILIES = new byte[][] { A };
 
-    Configuration conf = HBaseConfiguration.create();
+    // Use new HTU to not overlap with the DFS cluster started in #CompactionStacking
+    Configuration conf = new HBaseTestingUtility().getConfiguration();
     HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
     RegionCoprocessorHost h = region.getCoprocessorHost();
     h.load(NoDataFromScan.class, Coprocessor.PRIORITY_HIGHEST, conf);
@@ -234,7 +234,8 @@ public class TestRegionObserverScannerOpenHook {
     byte[] A = Bytes.toBytes("A");
     byte[][] FAMILIES = new byte[][] { A };
 
-    Configuration conf = HBaseConfiguration.create();
+    // Use new HTU to not overlap with the DFS cluster started in #CompactionStacking
+    Configuration conf = new HBaseTestingUtility().getConfiguration();
     HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
     RegionCoprocessorHost h = region.getCoprocessorHost();
     h.load(NoDataFromFlush.class, Coprocessor.PRIORITY_HIGHEST, conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
index cbc00d3..ab5da8f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
@@ -24,12 +24,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MockRegionServerServices;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
@@ -146,7 +144,7 @@ public class TestRegionObserverStacking extends TestCase {
     byte[] A = Bytes.toBytes("A");
     byte[][] FAMILIES = new byte[][] { A } ;
 
-    Configuration conf = HBaseConfiguration.create();
+    Configuration conf = TEST_UTIL.getConfiguration();
     HRegion region = initHRegion(TABLE, getClass().getName(),
       conf, FAMILIES);
     RegionCoprocessorHost h = region.getCoprocessorHost();

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
index f2f9142..406af17 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
@@ -22,7 +22,7 @@ import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -47,7 +47,7 @@ public class TestLocalAsyncOutput {
 
   private static Class<? extends Channel> CHANNEL_CLASS = NioSocketChannel.class;
 
-  private static final HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility();
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   @AfterClass
   public static void tearDownAfterClass() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/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 9d6aedb..8988d39 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
@@ -241,7 +241,7 @@ public class TestHStore {
   public void testFlushSizeSizing() throws Exception {
     LOG.info("Setting up a faulty file system that cannot write in " +
       this.name.getMethodName());
-    final Configuration conf = HBaseConfiguration.create();
+    final Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     // Only retry once.
     conf.setInt("hbase.hstore.flush.retries.number", 1);
     User user = User.createUserForTesting(conf, this.name.getMethodName(),
@@ -668,7 +668,7 @@ public class TestHStore {
   public void testHandleErrorsInFlush() throws Exception {
     LOG.info("Setting up a faulty file system that cannot write");
 
-    final Configuration conf = HBaseConfiguration.create();
+    final Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     User user = User.createUserForTesting(conf,
         "testhandleerrorsinflush", new String[]{"foo"});
     // Inject our faulty LocalFileSystem

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index a9bad00..b735069 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -197,8 +197,6 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
             latencyHistogram.update(System.nanoTime() - now);
           }
         }
-        long totalTime = (System.currentTimeMillis() - startTime);
-        logBenchmarkResult(Thread.currentThread().getName(), numIterations, totalTime);
       } catch (Exception e) {
         LOG.error(getClass().getSimpleName() + " Thread failed", e);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/99b9fff0/hbase-server/src/test/resources/hbase-site.xml
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/hbase-site.xml b/hbase-server/src/test/resources/hbase-site.xml
index dbdf776..64a1964 100644
--- a/hbase-server/src/test/resources/hbase-site.xml
+++ b/hbase-server/src/test/resources/hbase-site.xml
@@ -158,13 +158,4 @@
     <name>hbase.hconnection.threads.keepalivetime</name>
     <value>3</value>
   </property>
-  <property>
-    <name>hbase.unsafe.stream.capability.enforce</name>
-    <value>false</value>
-    <description>
-      Controls whether HBase will check for stream capabilities (hflush/hsync).
-      Disable this if you intend to run on LocalFileSystem.
-      WARNING: Doing so may expose you to additional risk of data loss!
-    </description>
-  </property>
 </configuration>


[27/41] hbase git commit: HBASE-19901 Up yetus proclimit on nightlies; ADDENDUM -- remove docker mem sizing for the moment...

Posted by zh...@apache.org.
HBASE-19901 Up yetus proclimit on nightlies; ADDENDUM -- remove docker mem sizing for the moment...


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

Branch: refs/heads/HBASE-19064
Commit: 5f5ddf55416a09c9675a0026b75dfc2aaf12632d
Parents: 94dad84
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 1 16:38:50 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 16:39:27 2018 -0800

----------------------------------------------------------------------
 dev-support/hbase-personality.sh   | 2 +-
 dev-support/hbase_nightly_yetus.sh | 1 -
 2 files changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5f5ddf55/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 5526e09..ddd0df4 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -71,7 +71,7 @@ function personality_globals
 
   # Set docker container to run with 20g. Default is 4g in yetus.
   # See HBASE-19902 for how we arrived at 20g.
-  DOCKERMEMLIMIT=20g
+  # UNUSED AT MOMENT DOCKERMEMLIMIT=20g
 }
 
 ## @description  Parse extra arguments required by personalities, if any.

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f5ddf55/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index 651a2e2..3e6fc7c 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -68,7 +68,6 @@ YETUS_ARGS=("--sentinel" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--proclimit=${PROCLIMIT}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--dockermemlimit=${DOCKERMEMLIMIT}" "${YETUS_ARGS[@]}")
 
 # Currently, flaky list is calculated only for master branch.
 UNDERSCORED_BRANCH_NAME=$(echo ${BRANCH_NAME} | tr '.-' '_')


[10/41] hbase git commit: HBASE-19528 - Major Compaction Tool

Posted by zh...@apache.org.
HBASE-19528 - Major Compaction Tool


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

Branch: refs/heads/HBASE-19064
Commit: 4b3b627abe34f8426fddd914e941d12f79da0752
Parents: 7c318ce
Author: Rahul Gidwani <ch...@apache.org>
Authored: Thu Jan 25 12:47:50 2018 -0800
Committer: Rahul Gidwani <ch...@apache.org>
Committed: Wed Jan 31 10:18:03 2018 -0800

----------------------------------------------------------------------
 .../compaction/ClusterCompactionQueues.java     | 137 +++++++
 .../util/compaction/MajorCompactionRequest.java | 171 +++++++++
 .../hbase/util/compaction/MajorCompactor.java   | 379 +++++++++++++++++++
 .../compaction/MajorCompactionRequestTest.java  | 166 ++++++++
 .../util/compaction/MajorCompactorTest.java     |  81 ++++
 5 files changed, 934 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4b3b627a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java
new file mode 100644
index 0000000..c0d34d9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.compaction;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+
+@InterfaceAudience.Private
+class ClusterCompactionQueues {
+
+  private final Map<ServerName, List<MajorCompactionRequest>> compactionQueues;
+  private final Set<ServerName> compactingServers;
+  private final ReadWriteLock lock;
+  private final int concurrentServers;
+
+  ClusterCompactionQueues(int concurrentServers) {
+    this.concurrentServers = concurrentServers;
+
+    this.compactionQueues = Maps.newHashMap();
+    this.lock = new ReentrantReadWriteLock();
+    this.compactingServers = Sets.newHashSet();
+  }
+
+  void addToCompactionQueue(ServerName serverName, MajorCompactionRequest info) {
+    this.lock.writeLock().lock();
+    try {
+      List<MajorCompactionRequest> result = this.compactionQueues.get(serverName);
+      if (result == null) {
+        result = Lists.newArrayList();
+        compactionQueues.put(serverName, result);
+      }
+      result.add(info);
+    } finally {
+      this.lock.writeLock().unlock();
+    }
+  }
+
+  boolean hasWorkItems() {
+    lock.readLock().lock();
+    try {
+      return !this.compactionQueues.values().stream().allMatch(List::isEmpty);
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  int getCompactionRequestsLeftToFinish() {
+    lock.readLock().lock();
+    try {
+      int size = 0;
+      for (List<MajorCompactionRequest> queue : compactionQueues.values()) {
+        size += queue.size();
+      }
+      return size;
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  @VisibleForTesting List<MajorCompactionRequest> getQueue(ServerName serverName) {
+    lock.readLock().lock();
+    try {
+      return compactionQueues.get(serverName);
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  MajorCompactionRequest reserveForCompaction(ServerName serverName) {
+    lock.writeLock().lock();
+    try {
+      if (!compactionQueues.get(serverName).isEmpty()) {
+        compactingServers.add(serverName);
+        return compactionQueues.get(serverName).remove(0);
+      }
+      return null;
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  void releaseCompaction(ServerName serverName) {
+    lock.writeLock().lock();
+    try {
+      compactingServers.remove(serverName);
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  boolean atCapacity() {
+    lock.readLock().lock();
+    try {
+      return compactingServers.size() >= concurrentServers;
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  Optional<ServerName> getLargestQueueFromServersNotCompacting() {
+    lock.readLock().lock();
+    try {
+      return compactionQueues.entrySet().stream()
+          .filter(entry -> !compactingServers.contains(entry.getKey()))
+          .max(Map.Entry.comparingByValue(
+            (o1, o2) -> Integer.compare(o1.size(), o2.size()))).map(Map.Entry::getKey);
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b3b627a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
new file mode 100644
index 0000000..51b2b9d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.compaction;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+
+@InterfaceAudience.Private
+class MajorCompactionRequest {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MajorCompactionRequest.class);
+
+  private final Configuration configuration;
+  private final RegionInfo region;
+  private Set<String> stores;
+  private final long timestamp;
+
+  @VisibleForTesting
+  MajorCompactionRequest(Configuration configuration, RegionInfo region,
+      Set<String> stores, long timestamp) {
+    this.configuration = configuration;
+    this.region = region;
+    this.stores = stores;
+    this.timestamp = timestamp;
+  }
+
+  static Optional<MajorCompactionRequest> newRequest(Configuration configuration, RegionInfo info,
+      Set<String> stores, long timestamp) throws IOException {
+    MajorCompactionRequest request =
+        new MajorCompactionRequest(configuration, info, stores, timestamp);
+    return request.createRequest(configuration, stores);
+  }
+
+  RegionInfo getRegion() {
+    return region;
+  }
+
+  Set<String> getStores() {
+    return stores;
+  }
+
+  void setStores(Set<String> stores) {
+    this.stores = stores;
+  }
+
+  @VisibleForTesting
+  Optional<MajorCompactionRequest> createRequest(Configuration configuration,
+      Set<String> stores) throws IOException {
+    Set<String> familiesToCompact = getStoresRequiringCompaction(stores);
+    MajorCompactionRequest request = null;
+    if (!familiesToCompact.isEmpty()) {
+      request = new MajorCompactionRequest(configuration, region, familiesToCompact, timestamp);
+    }
+    return Optional.ofNullable(request);
+  }
+
+  Set<String> getStoresRequiringCompaction(Set<String> requestedStores) throws IOException {
+    try(Connection connection = getConnection(configuration)) {
+      HRegionFileSystem fileSystem = getFileSystem(connection);
+      Set<String> familiesToCompact = Sets.newHashSet();
+      for (String family : requestedStores) {
+        // do we have any store files?
+        Collection<StoreFileInfo> storeFiles = fileSystem.getStoreFiles(family);
+        if (storeFiles == null) {
+          LOG.info("Excluding store: " + family + " for compaction for region:  " + fileSystem
+              .getRegionInfo().getEncodedName(), " has no store files");
+          continue;
+        }
+        // check for reference files
+        if (fileSystem.hasReferences(family) && familyHasReferenceFile(fileSystem, family)) {
+          familiesToCompact.add(family);
+          LOG.info("Including store: " + family + " with: " + storeFiles.size()
+              + " files for compaction for region: " + fileSystem.getRegionInfo().getEncodedName());
+          continue;
+        }
+        // check store file timestamps
+        boolean includeStore = false;
+        for (StoreFileInfo storeFile : storeFiles) {
+          if (storeFile.getModificationTime() < timestamp) {
+            LOG.info("Including store: " + family + " with: " + storeFiles.size()
+                + " files for compaction for region: "
+                + fileSystem.getRegionInfo().getEncodedName());
+            familiesToCompact.add(family);
+            includeStore = true;
+            break;
+          }
+        }
+        if (!includeStore) {
+          LOG.info("Excluding store: " + family + " for compaction for region:  " + fileSystem
+              .getRegionInfo().getEncodedName(), " already compacted");
+        }
+      }
+      return familiesToCompact;
+    }
+  }
+
+  @VisibleForTesting
+  Connection getConnection(Configuration configuration) throws IOException {
+    return ConnectionFactory.createConnection(configuration);
+  }
+
+  private boolean familyHasReferenceFile(HRegionFileSystem fileSystem, String family)
+      throws IOException {
+    List<Path> referenceFiles =
+        getReferenceFilePaths(fileSystem.getFileSystem(), fileSystem.getStoreDir(family));
+    for (Path referenceFile : referenceFiles) {
+      FileStatus status = fileSystem.getFileSystem().getFileLinkStatus(referenceFile);
+      if (status.getModificationTime() < timestamp) {
+        LOG.info("Including store: " + family + " for compaction for region:  " + fileSystem
+            .getRegionInfo().getEncodedName() + " (reference store files)");
+        return true;
+      }
+    }
+    return false;
+
+  }
+
+  @VisibleForTesting
+  List<Path> getReferenceFilePaths(FileSystem fileSystem, Path familyDir)
+      throws IOException {
+    return FSUtils.getReferenceFilePaths(fileSystem, familyDir);
+  }
+
+  @VisibleForTesting
+  HRegionFileSystem getFileSystem(Connection connection) throws IOException {
+    Admin admin = connection.getAdmin();
+    return HRegionFileSystem.openRegionFromFileSystem(admin.getConfiguration(),
+        FSUtils.getCurrentFileSystem(admin.getConfiguration()),
+        FSUtils.getTableDir(FSUtils.getRootDir(admin.getConfiguration()), region.getTable()),
+        region, true);
+  }
+
+  @Override
+  public String toString() {
+    return "region: " + region.getEncodedName() + " store(s): " + stores;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b3b627a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactor.java
new file mode 100644
index 0000000..c3372bb
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactor.java
@@ -0,0 +1,379 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.util.compaction;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.CompactionState;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
+import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+public class MajorCompactor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MajorCompactor.class);
+  private static final Set<MajorCompactionRequest> ERRORS = ConcurrentHashMap.newKeySet();
+
+  private final ClusterCompactionQueues clusterCompactionQueues;
+  private final long timestamp;
+  private final Set<String> storesToCompact;
+  private final ExecutorService executor;
+  private final long sleepForMs;
+  private final Connection connection;
+  private final TableName tableName;
+
+  public MajorCompactor(Configuration conf, TableName tableName, Set<String> storesToCompact,
+      int concurrency, long timestamp, long sleepForMs) throws IOException {
+    this.connection = ConnectionFactory.createConnection(conf);
+    this.tableName = tableName;
+    this.timestamp = timestamp;
+    this.storesToCompact = storesToCompact;
+    this.executor = Executors.newFixedThreadPool(concurrency);
+    this.clusterCompactionQueues = new ClusterCompactionQueues(concurrency);
+    this.sleepForMs = sleepForMs;
+  }
+
+  public void compactAllRegions() throws Exception {
+    List<Future<?>> futures = Lists.newArrayList();
+    while (clusterCompactionQueues.hasWorkItems() || !futuresComplete(futures)) {
+      while (clusterCompactionQueues.atCapacity()) {
+        LOG.debug("Waiting for servers to complete Compactions");
+        Thread.sleep(sleepForMs);
+      }
+      Optional<ServerName> serverToProcess =
+          clusterCompactionQueues.getLargestQueueFromServersNotCompacting();
+      if (serverToProcess.isPresent() && clusterCompactionQueues.hasWorkItems()) {
+        ServerName serverName = serverToProcess.get();
+        // check to see if the region has moved... if so we have to enqueue it again with
+        // the proper serverName
+        MajorCompactionRequest request = clusterCompactionQueues.reserveForCompaction(serverName);
+
+        ServerName currentServer = connection.getRegionLocator(tableName)
+            .getRegionLocation(request.getRegion().getStartKey()).getServerName();
+
+        if (!currentServer.equals(serverName)) {
+          // add it back to the queue with the correct server it should be picked up in the future.
+          LOG.info("Server changed for region: " + request.getRegion().getEncodedName() + " from: "
+              + serverName + " to: " + currentServer + " re-queuing request");
+          clusterCompactionQueues.addToCompactionQueue(currentServer, request);
+          clusterCompactionQueues.releaseCompaction(serverName);
+        } else {
+          LOG.info("Firing off compaction request for server: " + serverName + ", " + request
+              + " total queue size left: " + clusterCompactionQueues
+              .getCompactionRequestsLeftToFinish());
+          futures.add(executor.submit(new Compact(serverName, request)));
+        }
+      } else {
+        // haven't assigned anything so we sleep.
+        Thread.sleep(sleepForMs);
+      }
+    }
+    LOG.info("All compactions have completed");
+  }
+
+  private boolean futuresComplete(List<Future<?>> futures) {
+    futures.removeIf(Future::isDone);
+    return futures.isEmpty();
+  }
+
+  public void shutdown() throws Exception {
+    executor.shutdown();
+    executor.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+    if (!ERRORS.isEmpty()) {
+      StringBuilder builder =
+          new StringBuilder().append("Major compaction failed, there were: ").append(ERRORS.size())
+              .append(" regions / stores that failed compacting\n")
+              .append("Failed compaction requests\n").append("--------------------------\n")
+              .append(Joiner.on("\n").join(ERRORS));
+      LOG.error(builder.toString());
+    }
+    if (connection != null) {
+      connection.close();
+    }
+    LOG.info("All regions major compacted successfully");
+  }
+
+  @VisibleForTesting void initializeWorkQueues() throws IOException {
+    if (storesToCompact.isEmpty()) {
+      connection.getTable(tableName).getDescriptor().getColumnFamilyNames()
+          .forEach(a -> storesToCompact.add(Bytes.toString(a)));
+      LOG.info("No family specified, will execute for all families");
+    }
+    LOG.info(
+        "Initializing compaction queues for table:  " + tableName + " with cf: " + storesToCompact);
+    List<HRegionLocation> regionLocations =
+        connection.getRegionLocator(tableName).getAllRegionLocations();
+    for (HRegionLocation location : regionLocations) {
+      Optional<MajorCompactionRequest> request = MajorCompactionRequest
+          .newRequest(connection.getConfiguration(), location.getRegion(), storesToCompact,
+              timestamp);
+      request.ifPresent(majorCompactionRequest -> clusterCompactionQueues
+          .addToCompactionQueue(location.getServerName(), majorCompactionRequest));
+    }
+  }
+
+  class Compact implements Runnable {
+
+    private final ServerName serverName;
+    private final MajorCompactionRequest request;
+
+    Compact(ServerName serverName, MajorCompactionRequest request) {
+      this.serverName = serverName;
+      this.request = request;
+    }
+
+    @Override public void run() {
+      try {
+        compactAndWait(request);
+      } catch (NotServingRegionException e) {
+        // this region has split or merged
+        LOG.warn("Region is invalid, requesting updated regions", e);
+        // lets updated the cluster compaction queues with these newly created regions.
+        addNewRegions();
+      } catch (Exception e) {
+        LOG.warn("Error compacting:", e);
+      } finally {
+        clusterCompactionQueues.releaseCompaction(serverName);
+      }
+    }
+
+    void compactAndWait(MajorCompactionRequest request) throws Exception {
+      Admin admin = connection.getAdmin();
+      try {
+        // only make the request if the region is not already major compacting
+        if (!isCompacting(request)) {
+          Set<String> stores = request.getStoresRequiringCompaction(storesToCompact);
+          if (!stores.isEmpty()) {
+            request.setStores(stores);
+            for (String store : request.getStores()) {
+              admin.majorCompactRegion(request.getRegion().getEncodedNameAsBytes(),
+                  Bytes.toBytes(store));
+            }
+          }
+        }
+        while (isCompacting(request)) {
+          Thread.sleep(sleepForMs);
+          LOG.debug("Waiting for compaction to complete for region: " + request.getRegion()
+              .getEncodedName());
+        }
+      } finally {
+        // Make sure to wait for the CompactedFileDischarger chore to do its work
+        int waitForArchive = connection.getConfiguration()
+            .getInt("hbase.hfile.compaction.discharger.interval", 2 * 60 * 1000);
+        Thread.sleep(waitForArchive);
+        // check if compaction completed successfully, otherwise put that request back in the
+        // proper queue
+        Set<String> storesRequiringCompaction =
+            request.getStoresRequiringCompaction(storesToCompact);
+        if (!storesRequiringCompaction.isEmpty()) {
+          // this happens, when a region server is marked as dead, flushes a store file and
+          // the new regionserver doesn't pick it up because its accounted for in the WAL replay,
+          // thus you have more store files on the filesystem than the regionserver knows about.
+          boolean regionHasNotMoved = connection.getRegionLocator(tableName)
+              .getRegionLocation(request.getRegion().getStartKey()).getServerName()
+              .equals(serverName);
+          if (regionHasNotMoved) {
+            LOG.error("Not all store files were compacted, this may be due to the regionserver not "
+                + "being aware of all store files.  Will not reattempt compacting, " + request);
+            ERRORS.add(request);
+          } else {
+            request.setStores(storesRequiringCompaction);
+            clusterCompactionQueues.addToCompactionQueue(serverName, request);
+            LOG.info("Compaction failed for the following stores: " + storesRequiringCompaction
+                + " region: " + request.getRegion().getEncodedName());
+          }
+        } else {
+          LOG.info("Compaction complete for region: " + request.getRegion().getEncodedName()
+              + " -> cf(s): " + request.getStores());
+        }
+      }
+    }
+  }
+
+  private boolean isCompacting(MajorCompactionRequest request) throws Exception {
+    CompactionState compactionState = connection.getAdmin()
+        .getCompactionStateForRegion(request.getRegion().getEncodedNameAsBytes());
+    return compactionState.equals(CompactionState.MAJOR) || compactionState
+        .equals(CompactionState.MAJOR_AND_MINOR);
+  }
+
+  private void addNewRegions() {
+    try {
+      List<HRegionLocation> locations =
+          connection.getRegionLocator(tableName).getAllRegionLocations();
+      for (HRegionLocation location : locations) {
+        if (location.getRegion().getRegionId() > timestamp) {
+          Optional<MajorCompactionRequest> compactionRequest = MajorCompactionRequest
+              .newRequest(connection.getConfiguration(), location.getRegion(), storesToCompact,
+                  timestamp);
+          compactionRequest.ifPresent(request -> clusterCompactionQueues
+              .addToCompactionQueue(location.getServerName(), request));
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    Options options = new Options();
+    options.addOption(
+        Option.builder("table")
+            .required()
+            .desc("table name")
+            .hasArg()
+            .build()
+    );
+    options.addOption(
+        Option.builder("cf")
+            .optionalArg(true)
+            .desc("column families: comma separated eg: a,b,c")
+            .hasArg()
+            .build()
+    );
+    options.addOption(
+        Option.builder("servers")
+            .required()
+            .desc("Concurrent servers compacting")
+            .hasArg()
+            .build()
+    );
+    options.addOption(
+        Option.builder("minModTime").
+            desc("Compact if store files have modification time < minModTime")
+            .hasArg()
+            .build()
+    );
+    options.addOption(
+        Option.builder("zk")
+            .optionalArg(true)
+            .desc("zk quorum")
+            .hasArg()
+            .build()
+    );
+    options.addOption(
+        Option.builder("rootDir")
+            .optionalArg(true)
+            .desc("hbase.rootDir")
+            .hasArg()
+            .build()
+    );
+    options.addOption(
+        Option.builder("sleep")
+            .desc("Time to sleepForMs (ms) for checking compaction status per region and available "
+                + "work queues: default 30s")
+            .hasArg()
+            .build()
+    );
+    options.addOption(
+        Option.builder("retries")
+        .desc("Max # of retries for a compaction request," + " defaults to 3")
+            .hasArg()
+            .build()
+    );
+    options.addOption(
+        Option.builder("dryRun")
+            .desc("Dry run, will just output a list of regions that require compaction based on "
+            + "parameters passed")
+            .hasArg(false)
+            .build()
+    );
+
+    final CommandLineParser cmdLineParser = new DefaultParser();
+    CommandLine commandLine = null;
+    try {
+      commandLine = cmdLineParser.parse(options, args);
+    } catch (ParseException parseException) {
+      System.out.println(
+          "ERROR: Unable to parse command-line arguments " + Arrays.toString(args) + " due to: "
+              + parseException);
+      printUsage(options);
+
+    }
+    String tableName = commandLine.getOptionValue("table");
+    String cf = commandLine.getOptionValue("cf", null);
+    Set<String> families = Sets.newHashSet();
+    if (cf != null) {
+      Iterables.addAll(families, Splitter.on(",").split(cf));
+    }
+
+
+    Configuration configuration = HBaseConfiguration.create();
+    int concurrency = Integer.parseInt(commandLine.getOptionValue("servers"));
+    long minModTime = Long.parseLong(
+        commandLine.getOptionValue("minModTime", String.valueOf(System.currentTimeMillis())));
+    String quorum =
+        commandLine.getOptionValue("zk", configuration.get(HConstants.ZOOKEEPER_QUORUM));
+    String rootDir = commandLine.getOptionValue("rootDir", configuration.get(HConstants.HBASE_DIR));
+    long sleep = Long.valueOf(commandLine.getOptionValue("sleep", Long.toString(30000)));
+
+    configuration.set(HConstants.HBASE_DIR, rootDir);
+    configuration.set(HConstants.ZOOKEEPER_QUORUM, quorum);
+
+    MajorCompactor compactor =
+        new MajorCompactor(configuration, TableName.valueOf(tableName), families, concurrency,
+            minModTime, sleep);
+
+    compactor.initializeWorkQueues();
+    if (!commandLine.hasOption("dryRun")) {
+      compactor.compactAllRegions();
+    }
+    compactor.shutdown();
+  }
+
+  private static void printUsage(final Options options) {
+    String header = "\nUsage instructions\n\n";
+    String footer = "\n";
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp(MajorCompactor.class.getSimpleName(), header, options, footer, true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b3b627a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequestTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequestTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequestTest.java
new file mode 100644
index 0000000..c5ce4e3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequestTest.java
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.util.compaction;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+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.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isA;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+@Category({SmallTests.class})
+public class MajorCompactionRequestTest {
+
+  private static final HBaseTestingUtility UTILITY = new HBaseTestingUtility();
+  private static final String FAMILY = "a";
+  private Path rootRegionDir;
+  private Path regionStoreDir;
+
+  @Before public void setUp() throws Exception {
+    rootRegionDir = UTILITY.getDataTestDirOnTestFS("MajorCompactionRequestTest");
+    regionStoreDir = new Path(rootRegionDir, FAMILY);
+  }
+
+  @Test public void testStoresNeedingCompaction() throws Exception {
+    // store files older than timestamp
+    List<StoreFileInfo> storeFiles = mockStoreFiles(regionStoreDir, 5, 10);
+    MajorCompactionRequest request = makeMockRequest(100, storeFiles, false);
+    Optional<MajorCompactionRequest> result =
+        request.createRequest(mock(Configuration.class), Sets.newHashSet(FAMILY));
+    assertTrue(result.isPresent());
+
+    // store files newer than timestamp
+    storeFiles = mockStoreFiles(regionStoreDir, 5, 101);
+    request = makeMockRequest(100, storeFiles, false);
+    result = request.createRequest(mock(Configuration.class), Sets.newHashSet(FAMILY));
+    assertFalse(result.isPresent());
+  }
+
+  @Test public void testIfWeHaveNewReferenceFilesButOldStoreFiles() throws Exception {
+    // this tests that reference files that are new, but have older timestamps for the files
+    // they reference still will get compacted.
+    TableName table = TableName.valueOf("MajorCompactorTest");
+    TableDescriptor htd = UTILITY.createTableDescriptor(table, Bytes.toBytes(FAMILY));
+    RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    HRegion region =
+        HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, UTILITY.getConfiguration(), htd);
+
+    Configuration configuration = mock(Configuration.class);
+    // the reference file timestamp is newer
+    List<StoreFileInfo> storeFiles = mockStoreFiles(regionStoreDir, 4, 101);
+    List<Path> paths = storeFiles.stream().map(StoreFileInfo::getPath).collect(Collectors.toList());
+    // the files that are referenced are older, thus we still compact.
+    HRegionFileSystem fileSystem =
+        mockFileSystem(region.getRegionInfo(), true, storeFiles, 50);
+    MajorCompactionRequest majorCompactionRequest = spy(new MajorCompactionRequest(configuration,
+        region.getRegionInfo(), Sets.newHashSet(FAMILY), 100));
+    doReturn(mock(Connection.class)).when(majorCompactionRequest).getConnection(eq(configuration));
+    doReturn(paths).when(majorCompactionRequest).getReferenceFilePaths(any(FileSystem.class),
+        any(Path.class));
+    doReturn(fileSystem).when(majorCompactionRequest).getFileSystem(any(Connection.class));
+    Set<String> result = majorCompactionRequest.getStoresRequiringCompaction(Sets.newHashSet("a"));
+    assertEquals(FAMILY, Iterables.getOnlyElement(result));
+  }
+
+  private HRegionFileSystem mockFileSystem(RegionInfo info, boolean hasReferenceFiles,
+      List<StoreFileInfo> storeFiles) throws IOException {
+    long timestamp = storeFiles.stream().findFirst().get().getModificationTime();
+    return mockFileSystem(info, hasReferenceFiles, storeFiles, timestamp);
+  }
+
+  private HRegionFileSystem mockFileSystem(RegionInfo info, boolean hasReferenceFiles,
+      List<StoreFileInfo> storeFiles, long referenceFileTimestamp) throws IOException {
+    FileSystem fileSystem = mock(FileSystem.class);
+    if (hasReferenceFiles) {
+      FileStatus fileStatus = mock(FileStatus.class);
+      doReturn(referenceFileTimestamp).when(fileStatus).getModificationTime();
+      doReturn(fileStatus).when(fileSystem).getFileLinkStatus(isA(Path.class));
+    }
+    HRegionFileSystem mockSystem = mock(HRegionFileSystem.class);
+    doReturn(info).when(mockSystem).getRegionInfo();
+    doReturn(regionStoreDir).when(mockSystem).getStoreDir(FAMILY);
+    doReturn(hasReferenceFiles).when(mockSystem).hasReferences(anyString());
+    doReturn(storeFiles).when(mockSystem).getStoreFiles(anyString());
+    doReturn(fileSystem).when(mockSystem).getFileSystem();
+    return mockSystem;
+  }
+
+  private List<StoreFileInfo> mockStoreFiles(Path regionStoreDir, int howMany, long timestamp)
+      throws IOException {
+    List<StoreFileInfo> infos = Lists.newArrayList();
+    int i = 0;
+    while (i < howMany) {
+      StoreFileInfo storeFileInfo = mock(StoreFileInfo.class);
+      doReturn(timestamp).doReturn(timestamp).when(storeFileInfo).getModificationTime();
+      doReturn(new Path(regionStoreDir, RandomStringUtils.randomAlphabetic(10))).when(storeFileInfo)
+          .getPath();
+      infos.add(storeFileInfo);
+      i++;
+    }
+    return infos;
+  }
+
+  private MajorCompactionRequest makeMockRequest(long timestamp, List<StoreFileInfo> storeFiles,
+      boolean references) throws IOException {
+    Configuration configuration = mock(Configuration.class);
+    RegionInfo regionInfo = mock(RegionInfo.class);
+    when(regionInfo.getEncodedName()).thenReturn("HBase");
+    when(regionInfo.getTable()).thenReturn(TableName.valueOf("foo"));
+    MajorCompactionRequest request =
+        new MajorCompactionRequest(configuration, regionInfo, Sets.newHashSet("a"), timestamp);
+    MajorCompactionRequest spy = spy(request);
+    HRegionFileSystem fileSystem = mockFileSystem(regionInfo, references, storeFiles);
+    doReturn(fileSystem).when(spy).getFileSystem(isA(Connection.class));
+    doReturn(mock(Connection.class)).when(spy).getConnection(eq(configuration));
+    return spy;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/4b3b627a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTest.java
new file mode 100644
index 0000000..3fb37ec
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTest.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.compaction;
+
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.junit.After;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+
+@Category({ MiscTests.class, MediumTests.class })
+public class MajorCompactorTest {
+
+  public static final byte[] FAMILY = Bytes.toBytes("a");
+  private HBaseTestingUtility utility;
+
+  @Before public void setUp() throws Exception {
+    utility = new HBaseTestingUtility();
+    utility.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 10);
+    utility.startMiniCluster();
+  }
+
+  @After public void tearDown() throws Exception {
+    utility.shutdownMiniCluster();
+  }
+
+  @Test public void testCompactingATable() throws Exception {
+    TableName tableName = TableName.valueOf("MajorCompactorTest");
+    utility.createMultiRegionTable(tableName, FAMILY, 5);
+    utility.waitTableAvailable(tableName);
+    Connection connection = utility.getConnection();
+    Table table = connection.getTable(tableName);
+    // write data and flush multiple store files:
+    for (int i = 0; i < 5; i++) {
+      utility.loadRandomRows(table, FAMILY, 50, 100);
+      utility.flush(tableName);
+    }
+    table.close();
+    int numberOfRegions = utility.getAdmin().getRegions(tableName).size();
+    int numHFiles = utility.getNumHFiles(tableName, FAMILY);
+    // we should have a table with more store files than we would before we major compacted.
+    assertTrue(numberOfRegions < numHFiles);
+
+    MajorCompactor compactor =
+        new MajorCompactor(utility.getConfiguration(), tableName,
+            Sets.newHashSet(Bytes.toString(FAMILY)), 1, System.currentTimeMillis(), 200);
+    compactor.initializeWorkQueues();
+    compactor.compactAllRegions();
+    compactor.shutdown();
+
+    // verify that the store has been completely major compacted.
+    numberOfRegions = utility.getAdmin().getRegions(tableName).size();
+    numHFiles = utility.getNumHFiles(tableName, FAMILY);
+    assertEquals(numHFiles, numberOfRegions);
+  }
+}
\ No newline at end of file


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

Branch: refs/heads/HBASE-19064
Commit: 7a3ad0f6326794a29e3801e84aba0dbd13ef6947
Parents: 42e4b89
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 19 18:38:39 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 2 16:39:40 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       |  11 +-
 .../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, 663 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7a3ad0f6/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/7a3ad0f6/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/7a3ad0f6/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 7a7eb5e..8d84d14 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -215,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);
@@ -222,7 +222,6 @@ public class TestWALLockup {
     // There is no 'stop' once a logRoller is running.. it just dies.
     logRoller.start();
     // Now get a region and start adding in edits.
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
     final HRegion region = initHRegion(tableName, null, null, dodgyWAL);
     byte [] bytes = Bytes.toBytes(getName());
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(
@@ -236,7 +235,7 @@ public class TestWALLockup {
       Put put = new Put(bytes);
       put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
       WALKeyImpl key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
-          htd.getTableName(), System.currentTimeMillis(), mvcc, scopes);
+          TableName.META_TABLE_NAME, System.currentTimeMillis(), mvcc, scopes);
       WALEdit edit = new WALEdit();
       CellScanner CellScanner = put.cellScanner();
       assertTrue(CellScanner.advance());
@@ -389,10 +388,11 @@ public class TestWALLockup {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + getName());
     DodgyFSLog dodgyWAL1 = new DodgyFSLog(fs, rootDir, getName(), CONF);
-
+    dodgyWAL1.init();
     Path rootDir2 = new Path(dir + getName() + "2");
     final DodgyFSLog dodgyWAL2 = new DodgyFSLog(fs, rootDir2, getName() + "2",
         CONF);
+    dodgyWAL2.init();
     // Add a listener to force ringbuffer event handler sleep for a while
     dodgyWAL2.registerWALActionsListener(new DummyWALActionsListener());
 
@@ -403,7 +403,6 @@ public class TestWALLockup {
     // There is no 'stop' once a logRoller is running.. it just dies.
     logRoller.start();
     // Now get a region and start adding in edits.
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
     final HRegion region = initHRegion(tableName, null, null, dodgyWAL1);
     byte[] bytes = Bytes.toBytes(getName());
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(
@@ -414,7 +413,7 @@ public class TestWALLockup {
       Put put = new Put(bytes);
       put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
       WALKeyImpl key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
-          htd.getTableName(), System.currentTimeMillis(), mvcc, scopes);
+          TableName.META_TABLE_NAME, System.currentTimeMillis(), mvcc, scopes);
       WALEdit edit = new WALEdit();
       CellScanner CellScanner = put.cellScanner();
       assertTrue(CellScanner.advance());

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

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


[09/41] hbase git commit: HBASE-19887 Do not overwrite the surefire junit listener property in the pom of sub modules

Posted by zh...@apache.org.
HBASE-19887 Do not overwrite the surefire junit listener property in the pom of sub modules


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

Branch: refs/heads/HBASE-19064
Commit: 7c318cead902d8aa7c6defe67a6e76509c854d52
Parents: 2e6bc12
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 30 22:29:13 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Jan 31 20:49:18 2018 +0800

----------------------------------------------------------------------
 hbase-archetypes/hbase-client-project/pom.xml   |  13 -
 .../hbase-shaded-client-project/pom.xml         |  13 -
 .../hadoop/hbase/util/PoolMapTestBase.java      |  61 +++++
 .../hadoop/hbase/util/TestReusablePoolMap.java  |  90 +++++++
 .../hbase/util/TestRoundRobinPoolMap.java       | 102 ++++++++
 .../hbase/util/TestThreadLocalPoolMap.java      |  84 +++++++
 .../hbase/ResourceCheckerJUnitListener.java     |  23 +-
 hbase-external-blockcache/pom.xml               |  11 -
 hbase-http/pom.xml                              |   6 -
 hbase-mapreduce/pom.xml                         |  12 -
 hbase-replication/pom.xml                       |  17 --
 hbase-rest/pom.xml                              |   6 -
 hbase-server/pom.xml                            |   6 -
 .../ServerResourceCheckerJUnitListener.java     |  27 ---
 .../apache/hadoop/hbase/util/TestPoolMap.java   | 238 -------------------
 hbase-shell/pom.xml                             |  12 -
 hbase-zookeeper/pom.xml                         |  12 -
 17 files changed, 347 insertions(+), 386 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-archetypes/hbase-client-project/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-client-project/pom.xml b/hbase-archetypes/hbase-client-project/pom.xml
index 489446a..17aed9b 100644
--- a/hbase-archetypes/hbase-client-project/pom.xml
+++ b/hbase-archetypes/hbase-client-project/pom.xml
@@ -40,19 +40,6 @@
     <surefire.version>2.19</surefire.version>
     <junit.version>4.12</junit.version>
   </properties>
-
-  <build>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-surefire-plugin</artifactId>
-          <version>${surefire.version}</version>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-  </build>
-
   <dependencies>
     <!-- Dependency for hbase-testing-util must precede compile-scoped dependencies. -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-archetypes/hbase-shaded-client-project/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/pom.xml b/hbase-archetypes/hbase-shaded-client-project/pom.xml
index 59acf94..37f5e22 100644
--- a/hbase-archetypes/hbase-shaded-client-project/pom.xml
+++ b/hbase-archetypes/hbase-shaded-client-project/pom.xml
@@ -40,19 +40,6 @@
     <surefire.version>2.19</surefire.version>
     <junit.version>4.12</junit.version>
   </properties>
-
-  <build>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-surefire-plugin</artifactId>
-          <version>${surefire.version}</version>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-  </build>
-
   <dependencies>
     <!-- Dependency for hbase-testing-util must precede compile-scoped dependencies. -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-client/src/test/java/org/apache/hadoop/hbase/util/PoolMapTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/util/PoolMapTestBase.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/PoolMapTestBase.java
new file mode 100644
index 0000000..1b24252
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/PoolMapTestBase.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.hadoop.hbase.util.PoolMap.PoolType;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class PoolMapTestBase {
+
+  protected PoolMap<String, String> poolMap;
+
+  protected static final int POOL_SIZE = 3;
+
+  @Before
+  public void setUp() throws Exception {
+    this.poolMap = new PoolMap<>(getPoolType(), POOL_SIZE);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    this.poolMap.clear();
+  }
+
+  protected abstract PoolType getPoolType();
+
+  protected void runThread(final String randomKey, final String randomValue,
+      final String expectedValue) throws InterruptedException {
+    final AtomicBoolean matchFound = new AtomicBoolean(false);
+    Thread thread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        poolMap.put(randomKey, randomValue);
+        String actualValue = poolMap.get(randomKey);
+        matchFound
+            .set(expectedValue == null ? actualValue == null : expectedValue.equals(actualValue));
+      }
+    });
+    thread.start();
+    thread.join();
+    assertTrue(matchFound.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestReusablePoolMap.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestReusablePoolMap.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestReusablePoolMap.java
new file mode 100644
index 0000000..3fcaebb
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestReusablePoolMap.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.PoolMap.PoolType;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, SmallTests.class })
+public class TestReusablePoolMap extends PoolMapTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestReusablePoolMap.class);
+
+  @Override
+  protected PoolType getPoolType() {
+    return PoolType.Reusable;
+  }
+
+  @Test
+  public void testSingleThreadedClient() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    String randomKey = String.valueOf(rand.nextInt());
+    String randomValue = String.valueOf(rand.nextInt());
+    // As long as we poll values we put, the pool size should remain zero
+    runThread(randomKey, randomValue, randomValue);
+    assertEquals(0, poolMap.size(randomKey));
+  }
+
+  @Test
+  public void testMultiThreadedClients() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    // As long as we poll values we put, the pool size should remain zero
+    for (int i = 0; i < POOL_SIZE; i++) {
+      String randomKey = String.valueOf(rand.nextInt());
+      String randomValue = String.valueOf(rand.nextInt());
+      runThread(randomKey, randomValue, randomValue);
+      assertEquals(0, poolMap.size(randomKey));
+    }
+    poolMap.clear();
+    String randomKey = String.valueOf(rand.nextInt());
+    for (int i = 0; i < POOL_SIZE - 1; i++) {
+      String randomValue = String.valueOf(rand.nextInt());
+      runThread(randomKey, randomValue, randomValue);
+      assertEquals(0, poolMap.size(randomKey));
+    }
+    assertEquals(0, poolMap.size(randomKey));
+  }
+
+  @Test
+  public void testPoolCap() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    // As long as we poll values we put, the pool size should remain zero
+    String randomKey = String.valueOf(rand.nextInt());
+    List<String> randomValues = new ArrayList<>();
+    for (int i = 0; i < POOL_SIZE * 2; i++) {
+      String randomValue = String.valueOf(rand.nextInt());
+      randomValues.add(randomValue);
+      runThread(randomKey, randomValue, randomValue);
+    }
+    assertEquals(0, poolMap.size(randomKey));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java
new file mode 100644
index 0000000..a71cf29
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.PoolMap.PoolType;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, SmallTests.class })
+public class TestRoundRobinPoolMap extends PoolMapTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestRoundRobinPoolMap.class);
+
+  @Override
+  protected PoolType getPoolType() {
+    return PoolType.RoundRobin;
+  }
+
+  @Test
+  public void testSingleThreadedClient() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    String randomKey = String.valueOf(rand.nextInt());
+    String randomValue = String.valueOf(rand.nextInt());
+    // As long as the pool is not full, we'll get null back.
+    // This forces the user to create new values that can be used to populate
+    // the pool.
+    runThread(randomKey, randomValue, null);
+    assertEquals(1, poolMap.size(randomKey));
+  }
+
+  @Test
+  public void testMultiThreadedClients() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    for (int i = 0; i < POOL_SIZE; i++) {
+      String randomKey = String.valueOf(rand.nextInt());
+      String randomValue = String.valueOf(rand.nextInt());
+      // As long as the pool is not full, we'll get null back
+      runThread(randomKey, randomValue, null);
+      // As long as we use distinct keys, each pool will have one value
+      assertEquals(1, poolMap.size(randomKey));
+    }
+    poolMap.clear();
+    String randomKey = String.valueOf(rand.nextInt());
+    for (int i = 0; i < POOL_SIZE - 1; i++) {
+      String randomValue = String.valueOf(rand.nextInt());
+      // As long as the pool is not full, we'll get null back
+      runThread(randomKey, randomValue, null);
+      // since we use the same key, the pool size should grow
+      assertEquals(i + 1, poolMap.size(randomKey));
+    }
+    // at the end of the day, there should be as many values as we put
+    assertEquals(POOL_SIZE - 1, poolMap.size(randomKey));
+  }
+
+  @Test
+  public void testPoolCap() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    String randomKey = String.valueOf(rand.nextInt());
+    List<String> randomValues = new ArrayList<>();
+    for (int i = 0; i < POOL_SIZE * 2; i++) {
+      String randomValue = String.valueOf(rand.nextInt());
+      randomValues.add(randomValue);
+      if (i < POOL_SIZE - 1) {
+        // As long as the pool is not full, we'll get null back
+        runThread(randomKey, randomValue, null);
+      } else {
+        // when the pool becomes full, we expect the value we get back to be
+        // what we put earlier, in round-robin order
+        runThread(randomKey, randomValue, randomValues.get((i - POOL_SIZE + 1) % POOL_SIZE));
+      }
+    }
+    assertEquals(POOL_SIZE, poolMap.size(randomKey));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java
new file mode 100644
index 0000000..5f047c4
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.PoolMap.PoolType;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, SmallTests.class })
+public class TestThreadLocalPoolMap extends PoolMapTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestThreadLocalPoolMap.class);
+
+  @Override
+  protected PoolType getPoolType() {
+    return PoolType.ThreadLocal;
+  }
+
+  @Test
+  public void testSingleThreadedClient() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    String randomKey = String.valueOf(rand.nextInt());
+    String randomValue = String.valueOf(rand.nextInt());
+    // As long as the pool is not full, we should get back what we put
+    runThread(randomKey, randomValue, randomValue);
+    assertEquals(1, poolMap.size(randomKey));
+  }
+
+  @Test
+  public void testMultiThreadedClients() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    // As long as the pool is not full, we should get back what we put
+    for (int i = 0; i < POOL_SIZE; i++) {
+      String randomKey = String.valueOf(rand.nextInt());
+      String randomValue = String.valueOf(rand.nextInt());
+      runThread(randomKey, randomValue, randomValue);
+      assertEquals(1, poolMap.size(randomKey));
+    }
+    String randomKey = String.valueOf(rand.nextInt());
+    for (int i = 0; i < POOL_SIZE; i++) {
+      String randomValue = String.valueOf(rand.nextInt());
+      runThread(randomKey, randomValue, randomValue);
+      assertEquals(i + 1, poolMap.size(randomKey));
+    }
+  }
+
+  @Test
+  public void testPoolCap() throws InterruptedException, ExecutionException {
+    Random rand = ThreadLocalRandom.current();
+    String randomKey = String.valueOf(rand.nextInt());
+    for (int i = 0; i < POOL_SIZE * 2; i++) {
+      String randomValue = String.valueOf(rand.nextInt());
+      // as of HBASE-4150, pool limit is no longer used with ThreadLocalPool
+      runThread(randomKey, randomValue, randomValue);
+    }
+    assertEquals(POOL_SIZE * 2, poolMap.size(randomKey));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java
index 709646b..225d94f 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceCheckerJUnitListener.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
@@ -19,25 +18,24 @@
 
 package org.apache.hadoop.hbase;
 
-
-
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.hadoop.hbase.ResourceChecker.Phase;
 import org.apache.hadoop.hbase.util.JVM;
 import org.junit.runner.notification.RunListener;
 
 /**
  * Listen to the test progress and check the usage of:
- * - threads
- * - open file descriptor
- * - max open file descriptor
- * <p/>
+ * <ul>
+ * <li>threads</li>
+ * <li>open file descriptor</li>
+ * <li>max open file descriptor</li>
+ * </ul>
+ * <p>
  * When surefire forkMode=once/always/perthread, this code is executed on the forked process.
  */
 public class ResourceCheckerJUnitListener extends RunListener {
@@ -91,7 +89,7 @@ public class ResourceCheckerJUnitListener extends RunListener {
         return 0;
       }
       JVM jvm = new JVM();
-      return (int)jvm.getOpenFileDescriptorCount();
+      return (int) jvm.getOpenFileDescriptorCount();
     }
 
     @Override
@@ -107,7 +105,7 @@ public class ResourceCheckerJUnitListener extends RunListener {
         return 0;
       }
       JVM jvm = new JVM();
-      return (int)jvm.getMaxFileDescriptorCount();
+      return (int) jvm.getMaxFileDescriptorCount();
     }
   }
 
@@ -117,7 +115,7 @@ public class ResourceCheckerJUnitListener extends RunListener {
       if (!JVM.isUnix()) {
         return 0;
       }
-      return (int)(new JVM().getSystemLoadAverage()*100);
+      return (int) (new JVM().getSystemLoadAverage() * 100);
     }
   }
 
@@ -193,4 +191,3 @@ public class ResourceCheckerJUnitListener extends RunListener {
     end(descriptionToShortTestName(description));
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-external-blockcache/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-external-blockcache/pom.xml b/hbase-external-blockcache/pom.xml
index ce79e1a..630650f 100644
--- a/hbase-external-blockcache/pom.xml
+++ b/hbase-external-blockcache/pom.xml
@@ -79,17 +79,6 @@
           </execution>
         </executions>
       </plugin>
-      <plugin>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.hbase.ResourceCheckerJUnitListener</value>
-            </property>
-          </properties>
-        </configuration>
-      </plugin>
       <!-- Make a jar and put the sources in the jar -->
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-http/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-http/pom.xml b/hbase-http/pom.xml
index f1951e8..84f098a 100644
--- a/hbase-http/pom.xml
+++ b/hbase-http/pom.xml
@@ -122,12 +122,6 @@
       <plugin>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.hbase.ResourceCheckerJUnitListener</value>
-            </property>
-          </properties>
           <systemPropertyVariables>
             <test.build.webapps>target/test-classes/webapps</test.build.webapps>
           </systemPropertyVariables>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/pom.xml b/hbase-mapreduce/pom.xml
index 0b3724b..4a416dc 100644
--- a/hbase-mapreduce/pom.xml
+++ b/hbase-mapreduce/pom.xml
@@ -66,18 +66,6 @@
           </archive>
         </configuration>
       </plugin>
-      <!-- Testing plugins -->
-      <plugin>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.hbase.ServerResourceCheckerJUnitListener</value>
-            </property>
-          </properties>
-        </configuration>
-      </plugin>
       <!-- Make a jar and put the sources in the jar -->
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index 6bcbf8d..86a844a 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -47,23 +47,6 @@
           <skipAssembly>true</skipAssembly>
         </configuration>
       </plugin>
-      <plugin>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <!-- Always skip the second part executions, since we only run
-          simple unit tests in this module -->
-        <executions>
-          <execution>
-            <id>secondPartTestsExecution</id>
-            <phase>test</phase>
-            <goals>
-              <goal>test</goal>
-            </goals>
-            <configuration>
-              <skip>true</skip>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
       <!-- Make a jar and put the sources in the jar -->
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index d1036f2..d4ab0b4 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -125,12 +125,6 @@
       <plugin>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.hbase.ServerResourceCheckerJUnitListener</value>
-            </property>
-          </properties>
           <systemPropertyVariables>
             <test.build.webapps>target/test-classes/webapps</test.build.webapps>
           </systemPropertyVariables>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 49052ef..20186a7 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -237,12 +237,6 @@
       <plugin>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.hbase.ServerResourceCheckerJUnitListener</value>
-            </property>
-          </properties>
           <systemPropertyVariables>
             <test.build.webapps>target/test-classes/webapps</test.build.webapps>
           </systemPropertyVariables>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-server/src/test/java/org/apache/hadoop/hbase/ServerResourceCheckerJUnitListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ServerResourceCheckerJUnitListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ServerResourceCheckerJUnitListener.java
deleted file mode 100644
index 4b750e4..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ServerResourceCheckerJUnitListener.java
+++ /dev/null
@@ -1,27 +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;
-
-/**
- * Monitor the resources. use by the tests All resources in {@link ResourceCheckerJUnitListener}
- *  plus the number of connection.
- */
-public class ServerResourceCheckerJUnitListener extends ResourceCheckerJUnitListener {
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java
deleted file mode 100644
index dcca330..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.util;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import junit.framework.TestCase;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.PoolMap.PoolType;
-import org.junit.ClassRule;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
-@RunWith(Suite.class)
-@Suite.SuiteClasses({TestPoolMap.TestRoundRobinPoolType.class, TestPoolMap.TestThreadLocalPoolType.class,
-        TestPoolMap.TestReusablePoolType.class})
-@Category({MiscTests.class, SmallTests.class})
-public class TestPoolMap {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestPoolMap.class);
-
-  public abstract static class TestPoolType extends TestCase {
-    protected PoolMap<String, String> poolMap;
-    protected Random random = new Random();
-
-    protected static final int POOL_SIZE = 3;
-
-    @Override
-    protected void setUp() throws Exception {
-      this.poolMap = new PoolMap<>(getPoolType(), POOL_SIZE);
-    }
-
-    protected abstract PoolType getPoolType();
-
-    @Override
-    protected void tearDown() throws Exception {
-      this.poolMap.clear();
-    }
-
-    protected void runThread(final String randomKey, final String randomValue,
-        final String expectedValue) throws InterruptedException {
-      final AtomicBoolean matchFound = new AtomicBoolean(false);
-      Thread thread = new Thread(new Runnable() {
-        @Override
-        public void run() {
-          poolMap.put(randomKey, randomValue);
-          String actualValue = poolMap.get(randomKey);
-          matchFound.set(expectedValue == null ? actualValue == null
-              : expectedValue.equals(actualValue));
-        }
-      });
-      thread.start();
-      thread.join();
-      assertTrue(matchFound.get());
-    }
-  }
-
-  @Category({MiscTests.class, SmallTests.class})
-  public static class TestRoundRobinPoolType extends TestPoolType {
-    @Override
-    protected PoolType getPoolType() {
-      return PoolType.RoundRobin;
-    }
-
-    public void testSingleThreadedClient() throws InterruptedException,
-        ExecutionException {
-      String randomKey = String.valueOf(random.nextInt());
-      String randomValue = String.valueOf(random.nextInt());
-      // As long as the pool is not full, we'll get null back.
-      // This forces the user to create new values that can be used to populate
-      // the pool.
-      runThread(randomKey, randomValue, null);
-      assertEquals(1, poolMap.size(randomKey));
-    }
-
-    public void testMultiThreadedClients() throws InterruptedException,
-        ExecutionException {
-      for (int i = 0; i < POOL_SIZE; i++) {
-        String randomKey = String.valueOf(random.nextInt());
-        String randomValue = String.valueOf(random.nextInt());
-        // As long as the pool is not full, we'll get null back
-        runThread(randomKey, randomValue, null);
-        // As long as we use distinct keys, each pool will have one value
-        assertEquals(1, poolMap.size(randomKey));
-      }
-      poolMap.clear();
-      String randomKey = String.valueOf(random.nextInt());
-      for (int i = 0; i < POOL_SIZE - 1; i++) {
-        String randomValue = String.valueOf(random.nextInt());
-        // As long as the pool is not full, we'll get null back
-        runThread(randomKey, randomValue, null);
-        // since we use the same key, the pool size should grow
-        assertEquals(i + 1, poolMap.size(randomKey));
-      }
-      // at the end of the day, there should be as many values as we put
-      assertEquals(POOL_SIZE - 1, poolMap.size(randomKey));
-    }
-
-    public void testPoolCap() throws InterruptedException, ExecutionException {
-      String randomKey = String.valueOf(random.nextInt());
-      List<String> randomValues = new ArrayList<>();
-      for (int i = 0; i < POOL_SIZE * 2; i++) {
-        String randomValue = String.valueOf(random.nextInt());
-        randomValues.add(randomValue);
-        if (i < POOL_SIZE - 1) {
-          // As long as the pool is not full, we'll get null back
-          runThread(randomKey, randomValue, null);
-        } else {
-          // when the pool becomes full, we expect the value we get back to be
-          // what we put earlier, in round-robin order
-          runThread(randomKey, randomValue,
-              randomValues.get((i - POOL_SIZE + 1) % POOL_SIZE));
-        }
-      }
-      assertEquals(POOL_SIZE, poolMap.size(randomKey));
-    }
-
-  }
-
-  @Category({MiscTests.class, SmallTests.class})
-  public static class TestThreadLocalPoolType extends TestPoolType {
-    @Override
-    protected PoolType getPoolType() {
-      return PoolType.ThreadLocal;
-    }
-
-    public void testSingleThreadedClient() throws InterruptedException,
-        ExecutionException {
-      String randomKey = String.valueOf(random.nextInt());
-      String randomValue = String.valueOf(random.nextInt());
-      // As long as the pool is not full, we should get back what we put
-      runThread(randomKey, randomValue, randomValue);
-      assertEquals(1, poolMap.size(randomKey));
-    }
-
-    public void testMultiThreadedClients() throws InterruptedException,
-        ExecutionException {
-      // As long as the pool is not full, we should get back what we put
-      for (int i = 0; i < POOL_SIZE; i++) {
-        String randomKey = String.valueOf(random.nextInt());
-        String randomValue = String.valueOf(random.nextInt());
-        runThread(randomKey, randomValue, randomValue);
-        assertEquals(1, poolMap.size(randomKey));
-      }
-      String randomKey = String.valueOf(random.nextInt());
-      for (int i = 0; i < POOL_SIZE; i++) {
-        String randomValue = String.valueOf(random.nextInt());
-        runThread(randomKey, randomValue, randomValue);
-        assertEquals(i + 1, poolMap.size(randomKey));
-      }
-    }
-
-    public void testPoolCap() throws InterruptedException, ExecutionException {
-      String randomKey = String.valueOf(random.nextInt());
-      for (int i = 0; i < POOL_SIZE * 2; i++) {
-        String randomValue = String.valueOf(random.nextInt());
-        // as of HBASE-4150, pool limit is no longer used with ThreadLocalPool
-          runThread(randomKey, randomValue, randomValue);
-      }
-      assertEquals(POOL_SIZE * 2, poolMap.size(randomKey));
-    }
-
-  }
-
-  @Category({MiscTests.class, SmallTests.class})
-  public static class TestReusablePoolType extends TestPoolType {
-    @Override
-    protected PoolType getPoolType() {
-      return PoolType.Reusable;
-    }
-
-    public void testSingleThreadedClient() throws InterruptedException,
-        ExecutionException {
-      String randomKey = String.valueOf(random.nextInt());
-      String randomValue = String.valueOf(random.nextInt());
-      // As long as we poll values we put, the pool size should remain zero
-      runThread(randomKey, randomValue, randomValue);
-      assertEquals(0, poolMap.size(randomKey));
-    }
-
-    public void testMultiThreadedClients() throws InterruptedException,
-        ExecutionException {
-      // As long as we poll values we put, the pool size should remain zero
-      for (int i = 0; i < POOL_SIZE; i++) {
-        String randomKey = String.valueOf(random.nextInt());
-        String randomValue = String.valueOf(random.nextInt());
-        runThread(randomKey, randomValue, randomValue);
-        assertEquals(0, poolMap.size(randomKey));
-      }
-      poolMap.clear();
-      String randomKey = String.valueOf(random.nextInt());
-      for (int i = 0; i < POOL_SIZE - 1; i++) {
-        String randomValue = String.valueOf(random.nextInt());
-        runThread(randomKey, randomValue, randomValue);
-        assertEquals(0, poolMap.size(randomKey));
-      }
-      assertEquals(0, poolMap.size(randomKey));
-    }
-
-    public void testPoolCap() throws InterruptedException, ExecutionException {
-      // As long as we poll values we put, the pool size should remain zero
-      String randomKey = String.valueOf(random.nextInt());
-      List<String> randomValues = new ArrayList<>();
-      for (int i = 0; i < POOL_SIZE * 2; i++) {
-        String randomValue = String.valueOf(random.nextInt());
-        randomValues.add(randomValue);
-        runThread(randomKey, randomValue, randomValue);
-      }
-      assertEquals(0, poolMap.size(randomKey));
-    }
-
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-shell/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shell/pom.xml b/hbase-shell/pom.xml
index 0f623d6..deb2fe4 100644
--- a/hbase-shell/pom.xml
+++ b/hbase-shell/pom.xml
@@ -123,18 +123,6 @@
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
       </plugin>
-      <!-- Testing plugins -->
-      <plugin>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.hbase.ServerResourceCheckerJUnitListener</value>
-            </property>
-          </properties>
-        </configuration>
-      </plugin>
     </plugins>
     <!-- General Resources -->
     <pluginManagement>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c318cea/hbase-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/pom.xml b/hbase-zookeeper/pom.xml
index 24364f7..8fb5b15 100644
--- a/hbase-zookeeper/pom.xml
+++ b/hbase-zookeeper/pom.xml
@@ -98,18 +98,6 @@
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
       </plugin>
-      <!-- Testing plugins -->
-      <plugin>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.hbase.ResourceCheckerJUnitListener</value>
-            </property>
-          </properties>
-        </configuration>
-      </plugin>
     </plugins>
     <!-- General Resources -->
     <pluginManagement>


[19/41] hbase git commit: HBASE-19906 TestZooKeeper Timeout Includes move of TestQoSFunction from smalltests to mediumtests; it timedout in a run.

Posted by zh...@apache.org.
HBASE-19906 TestZooKeeper Timeout Includes move of TestQoSFunction from smalltests to mediumtests; it timedout in a run.


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

Branch: refs/heads/HBASE-19064
Commit: b21b8bfb91662bc65327f924504975997e0946da
Parents: 52cb9aa
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 31 14:02:11 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 10:53:18 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/master/HMaster.java |  5 +++++
 .../hadoop/hbase/master/assignment/AssignmentManager.java | 10 +++++++++-
 .../apache/hadoop/hbase/regionserver/TestQosFunction.java |  4 ++--
 3 files changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b21b8bfb/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 22f3bb9..d422960 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
@@ -869,6 +869,11 @@ public class HMaster extends HRegionServer implements MasterServices {
     // Make sure meta assigned before proceeding.
     status.setStatus("Recovering  Meta Region");
 
+    // Check if master is shutting down because issue initializing regionservers or balancer.
+    if (isStopped()) {
+      return;
+    }
+
     // we recover hbase:meta region servers inside master initialization and
     // handle other failed servers in SSH in order to start up master node ASAP
     MasterMetaBootstrap metaBootstrap = createMetaBootstrap(this, status);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b21b8bfb/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 d6b73f4..4bcd771 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
@@ -1895,9 +1895,17 @@ public class AssignmentManager implements ServerListener {
         .getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO,
             RegionInfo.DEFAULT_REPLICA_ID);
     RegionState regionStateNode = getRegionStates().getRegionState(hri);
-    if (!regionStateNode.getServerName().equals(serverName)) {
+    if (regionStateNode == null) {
+      LOG.warn("RegionStateNode is null for " + hri);
       return;
     }
+    ServerName rsnServerName = regionStateNode.getServerName();
+    if (rsnServerName != null && !rsnServerName.equals(serverName)) {
+      return;
+    } else if (rsnServerName == null) {
+      LOG.warn("Empty ServerName in RegionStateNode; proceeding anyways in case latched " +
+          "RecoverMetaProcedure so meta latch gets cleaned up.");
+    }
     // meta has been assigned to crashed server.
     LOG.info("Meta assigned to crashed " + serverName + "; reassigning...");
     // Handle failure and wake event

http://git-wip-us.apache.org/repos/asf/hbase/blob/b21b8bfb/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
index de415c7..99d140f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
@@ -22,8 +22,8 @@ import static org.mockito.Mockito.when;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.*;
 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.testclassification.SmallTests;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiReque
  * Basic test that qos function is sort of working; i.e. a change in method naming style
  * over in pb doesn't break it.
  */
-@Category({RegionServerTests.class, SmallTests.class})
+@Category({RegionServerTests.class, MediumTests.class})
 public class TestQosFunction extends QosTestHelper {
 
   @ClassRule


[04/41] hbase git commit: HBASE-19868 TestCoprocessorWhitelistMasterObserver is flakey

Posted by zh...@apache.org.
HBASE-19868 TestCoprocessorWhitelistMasterObserver is flakey

Recategorize TestCoprocessorWhitelistMasterObserver as Large test


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

Branch: refs/heads/HBASE-19064
Commit: 194890b69492dcddb9ebdd4e398aeeffc7435fac
Parents: 67f91f1
Author: Peter Somogyi <ps...@cloudera.com>
Authored: Tue Jan 30 11:49:27 2018 +0100
Committer: Michael Stack <st...@apache.org>
Committed: Tue Jan 30 06:01:24 2018 -0800

----------------------------------------------------------------------
 .../access/TestCoprocessorWhitelistMasterObserver.java         | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/194890b6/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
index c161b9e..f9b610a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
@@ -50,9 +50,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Performs coprocessor loads for variuos paths and malformed strings
+ * Performs coprocessor loads for various paths and malformed strings
  */
-@Category({SecurityTests.class, MediumTests.class})
+@Category({SecurityTests.class, LargeTests.class})
 public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
 
   @ClassRule


[33/41] hbase git commit: HBASE-19904 Break dependency of WAL constructor on Replication

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 0d9db31..88e1aa2 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
@@ -47,7 +47,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -139,9 +138,7 @@ import org.apache.hadoop.hbase.regionserver.Region.RowLock;
 import org.apache.hadoop.hbase.regionserver.TestHStore.FaultyFileSystem;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
-import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
@@ -381,8 +378,8 @@ public class TestHRegion {
     final Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, logDir);
-    return new WALFactory(walConf, Collections.<WALActionsListener> singletonList(new MetricsWAL()),
-        callingMethod).getWAL(RegionInfoBuilder.newBuilder(tableName).build());
+    return new WALFactory(walConf, callingMethod)
+        .getWAL(RegionInfoBuilder.newBuilder(tableName).build());
   }
 
   @Test
@@ -642,7 +639,7 @@ public class TestHRegion {
   public void testSkipRecoveredEditsReplay() throws Exception {
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, CONF, family);
-    final WALFactory wals = new WALFactory(CONF, null, method);
+    final WALFactory wals = new WALFactory(CONF, method);
     try {
       Path regiondir = region.getRegionFileSystem().getRegionDir();
       FileSystem fs = region.getRegionFileSystem().getFileSystem();
@@ -693,7 +690,7 @@ public class TestHRegion {
   public void testSkipRecoveredEditsReplaySomeIgnored() throws Exception {
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, CONF, family);
-    final WALFactory wals = new WALFactory(CONF, null, method);
+    final WALFactory wals = new WALFactory(CONF, method);
     try {
       Path regiondir = region.getRegionFileSystem().getRegionDir();
       FileSystem fs = region.getRegionFileSystem().getFileSystem();
@@ -781,7 +778,7 @@ public class TestHRegion {
   public void testSkipRecoveredEditsReplayTheLastFileIgnored() throws Exception {
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, CONF, family);
-    final WALFactory wals = new WALFactory(CONF, null, method);
+    final WALFactory wals = new WALFactory(CONF, method);
     try {
       Path regiondir = region.getRegionFileSystem().getRegionDir();
       FileSystem fs = region.getRegionFileSystem().getFileSystem();
@@ -850,7 +847,7 @@ public class TestHRegion {
     CONF.setClass(HConstants.REGION_IMPL, HRegionForTesting.class, Region.class);
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, CONF, family);
-    final WALFactory wals = new WALFactory(CONF, null, method);
+    final WALFactory wals = new WALFactory(CONF, method);
     try {
       Path regiondir = region.getRegionFileSystem().getRegionDir();
       FileSystem fs = region.getRegionFileSystem().getFileSystem();
@@ -961,7 +958,7 @@ public class TestHRegion {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log");
     final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
     FSUtils.setRootDir(walConf, logDir);
-    final WALFactory wals = new WALFactory(walConf, null, method);
+    final WALFactory wals = new WALFactory(walConf, method);
     final WAL wal = wals.getWAL(RegionInfoBuilder.newBuilder(tableName).build());
 
     this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
@@ -4694,7 +4691,7 @@ public class TestHRegion {
     // XXX: The spied AsyncFSWAL can not work properly because of a Mockito defect that can not
     // deal with classes which have a field of an inner class. See discussions in HBASE-15536.
     walConf.set(WALFactory.WAL_PROVIDER, "filesystem");
-    final WALFactory wals = new WALFactory(walConf, null, UUID.randomUUID().toString());
+    final WALFactory wals = new WALFactory(walConf, UUID.randomUUID().toString());
     final WAL wal = spy(wals.getWAL(RegionInfoBuilder.newBuilder(tableName).build()));
     this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
         HConstants.EMPTY_END_ROW, false, tableDurability, wal,
@@ -4844,9 +4841,7 @@ public class TestHRegion {
   static WALFactory createWALFactory(Configuration conf, Path rootDir) throws IOException {
     Configuration confForWAL = new Configuration(conf);
     confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
-    return new WALFactory(confForWAL,
-        Collections.<WALActionsListener>singletonList(new MetricsWAL()),
-        "hregion-" + RandomStringUtils.randomNumeric(8));
+    return new WALFactory(confForWAL, "hregion-" + RandomStringUtils.randomNumeric(8));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 8988d39..c2bd83a 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
@@ -216,7 +216,7 @@ public class TestHStore {
     RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
     Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
-    WALFactory wals = new WALFactory(walConf, null, methodName);
+    WALFactory wals = new WALFactory(walConf, methodName);
     region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info), wals.getWAL(info), conf,
         htd, null);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 1e3bdcd..5b0a60f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -117,7 +117,7 @@ public class TestStoreFileRefresherChore {
         new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
-    final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId);
+    final WALFactory wals = new WALFactory(walConf, "log_" + replicaId);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region =
         new HRegion(fs, wals.getWAL(info),

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index cdc1572..c7a2a7c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -121,7 +121,7 @@ public class TestWALMonotonicallyIncreasingSeqId {
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     this.walConf = walConf;
-    wals = new WALFactory(walConf, null, "log_" + replicaId);
+    wals = new WALFactory(walConf, "log_" + replicaId);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region = HRegion.createHRegion(info, TEST_UTIL.getDefaultRootDirPath(), conf, htd,
       wals.getWAL(info));

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
index 37c3b37..c6059b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
@@ -159,11 +159,11 @@ public abstract class AbstractTestLogRolling  {
   /**
    * Tests that log rolling doesn't hang when no data is written.
    */
-  @Test(timeout = 120000)
+  @Test
   public void testLogRollOnNothingWritten() throws Exception {
     final Configuration conf = TEST_UTIL.getConfiguration();
     final WALFactory wals =
-        new WALFactory(conf, null, ServerName.valueOf("test.com", 8080, 1).toString());
+      new WALFactory(conf, ServerName.valueOf("test.com", 8080, 1).toString());
     final WAL newLog = wals.getWAL(null);
     try {
       // Now roll the log before we write anything.
@@ -183,8 +183,6 @@ public abstract class AbstractTestLogRolling  {
 
   /**
    * Tests that logs are deleted
-   * @throws IOException
-   * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
    */
   @Test
   public void testLogRolling() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 c0510d3..e49cda0 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
@@ -26,7 +26,6 @@ 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;
@@ -71,7 +70,7 @@ public abstract class AbstractTestProtobufLog<W extends Closeable> {
   public void setUp() throws Exception {
     fs = TEST_UTIL.getDFSCluster().getFileSystem();
     dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
-    wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
+    wals = new WALFactory(TEST_UTIL.getConfiguration(), currentTest.getMethodName());
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 d18b75c..b1e304e 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
@@ -22,9 +22,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
@@ -43,7 +43,6 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -155,14 +154,14 @@ public abstract class AbstractTestWALReplay {
     this.hbaseRootDir = FSUtils.getRootDir(this.conf);
     this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
     String serverName =
-        ServerName.valueOf(currentTest.getMethodName() + "-manual", 16010,
-            System.currentTimeMillis()).toString();
+      ServerName.valueOf(currentTest.getMethodName() + "-manual", 16010, System.currentTimeMillis())
+          .toString();
     this.logName = AbstractFSWALProvider.getWALDirectoryName(serverName);
     this.logDir = new Path(this.hbaseRootDir, logName);
     if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
       TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
     }
-    this.wals = new WALFactory(conf, null, currentTest.getMethodName());
+    this.wals = new WALFactory(conf, currentTest.getMethodName());
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index b7aa0e3..f5fabbc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -123,7 +123,7 @@ public class TestDurability {
 
   @Test
   public void testDurability() throws Exception {
-    WALFactory wals = new WALFactory(CONF, null,
+    WALFactory wals = new WALFactory(CONF,
         ServerName.valueOf("TestDurability", 16010, System.currentTimeMillis()).toString());
     HRegion region = createHRegion(wals, Durability.USE_DEFAULT);
     WAL wal = region.getWAL();
@@ -187,7 +187,7 @@ public class TestDurability {
     byte[] col3 = Bytes.toBytes("col3");
 
     // Setting up region
-    WALFactory wals = new WALFactory(CONF, null,
+    WALFactory wals = new WALFactory(CONF,
         ServerName.valueOf("TestIncrement", 16010, System.currentTimeMillis()).toString());
     HRegion region = createHRegion(wals, Durability.USE_DEFAULT);
     WAL wal = region.getWAL();
@@ -253,7 +253,7 @@ public class TestDurability {
     byte[] col1 = Bytes.toBytes("col1");
 
     // Setting up region
-    WALFactory wals = new WALFactory(CONF, null,
+    WALFactory wals = new WALFactory(CONF,
         ServerName
             .valueOf("testIncrementWithReturnResultsSetToFalse", 16010, System.currentTimeMillis())
             .toString());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
index e27fb97..3476aaf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
@@ -191,7 +191,7 @@ public class TestLogRollAbort {
     String logName = ServerName.valueOf("testLogRollAfterSplitStart",
         16010, System.currentTimeMillis()).toString();
     Path thisTestsDir = new Path(HBASELOGDIR, AbstractFSWALProvider.getWALDirectoryName(logName));
-    final WALFactory wals = new WALFactory(conf, null, logName);
+    final WALFactory wals = new WALFactory(conf, logName);
 
     try {
       // put some entries in an WAL

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index c83e4e7..5ee0dfa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -91,7 +91,7 @@ public class TestLogRollingNoCluster {
     conf.set(WALFactory.WAL_PROVIDER, "filesystem");
     FSUtils.setRootDir(conf, dir);
     conf.set("hbase.regionserver.hlog.writer.impl", HighLatencySyncWriter.class.getName());
-    final WALFactory wals = new WALFactory(conf, null, TestLogRollingNoCluster.class.getName());
+    final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName());
     final WAL wal = wals.getWAL(null);
 
     Appender [] appenders = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index d7d3166..0967a75 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertEquals;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import org.apache.hadoop.conf.Configuration;
@@ -99,9 +97,8 @@ public class TestWALActionsListener {
   @Test
   public void testActionListener() throws Exception {
     DummyWALActionsListener observer = new DummyWALActionsListener();
-    List<WALActionsListener> list = new ArrayList<>(1);
-    list.add(observer);
-    final WALFactory wals = new WALFactory(conf, list, "testActionListener");
+    final WALFactory wals = new WALFactory(conf, "testActionListener");
+    wals.getWALProvider().addWALActionsListener(observer);
     DummyWALActionsListener laterobserver = new DummyWALActionsListener();
     RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(SOME_BYTES))
         .setStartKey(SOME_BYTES).setEndKey(SOME_BYTES).build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
index fe773cb..4effe41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
@@ -104,10 +104,10 @@ public class TestReplicationEmptyWALRecovery extends TestReplicationBase {
     for (int i = 0; i < numRs; i++) {
       HRegionServer hrs = utility1.getHBaseCluster().getRegionServer(i);
       Replication replicationService = (Replication) hrs.getReplicationSourceService();
-      replicationService.preLogRoll(null, emptyWalPaths.get(i));
-      replicationService.postLogRoll(null, emptyWalPaths.get(i));
+      replicationService.getReplicationManager().preLogRoll(emptyWalPaths.get(i));
+      replicationService.getReplicationManager().postLogRoll(emptyWalPaths.get(i));
       RegionInfo regionInfo =
-          utility1.getHBaseCluster().getRegions(htable1.getName()).get(0).getRegionInfo();
+        utility1.getHBaseCluster().getRegions(htable1.getName()).get(0).getRegionInfo();
       WAL wal = hrs.getWAL(regionInfo);
       wal.rollWriter(true);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index f4d4d71..f5d2a2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -60,8 +58,6 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-
 @Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationSmallTests extends TestReplicationBase {
 
@@ -333,20 +329,6 @@ public class TestReplicationSmallTests extends TestReplicationBase {
   }
 
   /**
-   * Test for HBASE-9038, Replication.scopeWALEdits would NPE if it wasn't filtering out the
-   * compaction WALEdit.
-   */
-  @Test
-  public void testCompactionWALEdits() throws Exception {
-    WALProtos.CompactionDescriptor compactionDescriptor =
-        WALProtos.CompactionDescriptor.getDefaultInstance();
-    RegionInfo hri = RegionInfoBuilder.newBuilder(htable1.getName())
-        .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).build();
-    WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
-    Replication.scopeWALEdits(new WALKeyImpl(), edit, htable1.getConfiguration(), null);
-  }
-
-  /**
    * Test for HBASE-8663
    * <p>
    * Create two new Tables with colfamilies enabled for replication then run

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 6bdbbd2..a8afe2d 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
@@ -59,6 +59,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
@@ -80,6 +82,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
@@ -100,6 +103,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 
 /**
@@ -179,8 +183,8 @@ public abstract class TestReplicationSourceManager {
         HConstants.HREGION_OLDLOGDIR_NAME);
     logDir = new Path(utility.getDataTestDir(),
         HConstants.HREGION_LOGDIR_NAME);
-    replication = new Replication(new DummyServer(), fs, logDir, oldLogDir);
-
+    replication = new Replication();
+    replication.initialize(new DummyServer(), fs, logDir, oldLogDir, null);
     managerOfCluster = getManagerFromCluster();
     if (managerOfCluster != null) {
       // After replication procedure, we need to add peer by hand (other than by receiving
@@ -269,10 +273,26 @@ public abstract class TestReplicationSourceManager {
     WALEdit edit = new WALEdit();
     edit.add(kv);
 
-    List<WALActionsListener> listeners = new ArrayList<>(1);
-    listeners.add(replication);
-    final WALFactory wals = new WALFactory(utility.getConfiguration(), listeners,
-        URLEncoder.encode("regionserver:60020", "UTF8"));
+    WALFactory wals =
+      new WALFactory(utility.getConfiguration(), URLEncoder.encode("regionserver:60020", "UTF8"));
+    ReplicationSourceManager replicationManager = replication.getReplicationManager();
+    wals.getWALProvider().addWALActionsListener(new WALActionsListener() {
+
+      @Override
+      public void preLogRoll(Path oldPath, Path newPath) throws IOException {
+        replicationManager.preLogRoll(newPath);
+      }
+
+      @Override
+      public void postLogRoll(Path oldPath, Path newPath) throws IOException {
+        replicationManager.postLogRoll(newPath);
+      }
+
+      @Override
+      public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
+        replicationManager.scopeWALEdits(logKey, logEdit);
+      }
+    });
     final WAL wal = wals.getWAL(hri);
     manager.init();
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame"));
@@ -418,6 +438,21 @@ public abstract class TestReplicationSourceManager {
     }
   }
 
+  /**
+   * Test for HBASE-9038, Replication.scopeWALEdits would NPE if it wasn't filtering out the
+   * compaction WALEdit.
+   */
+  @Test
+  public void testCompactionWALEdits() throws Exception {
+    TableName tableName = TableName.valueOf("testCompactionWALEdits");
+    WALProtos.CompactionDescriptor compactionDescriptor =
+      WALProtos.CompactionDescriptor.getDefaultInstance();
+    RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW).build();
+    WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
+    ReplicationSourceManager.scopeWALEdits(new WALKeyImpl(), edit, conf);
+  }
+
   @Test
   public void testBulkLoadWALEditsWithoutBulkLoadReplicationEnabled() throws Exception {
     NavigableMap<byte[], Integer> scope = new TreeMap<>(Bytes.BYTES_COMPARATOR);
@@ -427,7 +462,7 @@ public abstract class TestReplicationSourceManager {
     WALKeyImpl logKey = new WALKeyImpl(scope);
 
     // 3. Get the scopes for the key
-    Replication.scopeWALEdits(logKey, logEdit, conf, manager);
+    ReplicationSourceManager.scopeWALEdits(logKey, logEdit, conf);
 
     // 4. Assert that no bulk load entry scopes are added if bulk load hfile replication is disabled
     assertNull("No bulk load entries scope should be added if bulk load replication is disabled.",
@@ -446,7 +481,7 @@ public abstract class TestReplicationSourceManager {
     bulkLoadConf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
 
     // 4. Get the scopes for the key
-    Replication.scopeWALEdits(logKey, logEdit, bulkLoadConf, manager);
+    ReplicationSourceManager.scopeWALEdits(logKey, logEdit, bulkLoadConf);
 
     NavigableMap<byte[], Integer> scopes = logKey.getReplicationScopes();
     // Assert family with replication scope global is present in the key scopes

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
index d89c9c2..2146e47 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
@@ -26,8 +26,6 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.NavigableMap;
 import java.util.NoSuchElementException;
 import java.util.OptionalLong;
@@ -119,10 +117,9 @@ public class TestWALEntryStream {
   @Before
   public void setUp() throws Exception {
     walQueue = new PriorityBlockingQueue<>();
-    List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
     pathWatcher = new PathWatcher();
-    listeners.add(pathWatcher);
-    final WALFactory wals = new WALFactory(conf, listeners, tn.getMethodName());
+    final WALFactory wals = new WALFactory(conf, tn.getMethodName());
+    wals.getWALProvider().addWALActionsListener(pathWatcher);
     log = wals.getWAL(info);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 485e5b8..3928d9c 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
@@ -25,6 +25,7 @@ import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DE
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -79,8 +80,15 @@ public class IOTestProvider implements WALProvider {
     none
   }
 
-  private FSHLog log = null;
+  private WALFactory factory;
 
+  private Configuration conf;
+
+  private volatile FSHLog log;
+
+  private String providerId;
+
+  private List<WALActionsListener> listeners = new ArrayList<>();
   /**
    * @param factory factory that made us, identity used for FS layout. may not be null
    * @param conf may not be null
@@ -89,41 +97,60 @@ public class IOTestProvider implements WALProvider {
    *                   null
    */
   @Override
-  public void init(final WALFactory factory, final Configuration conf,
-      final List<WALActionsListener> listeners, String providerId) throws IOException {
-    if (null != log) {
+  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
+    if (factory != null) {
       throw new IllegalStateException("WALProvider.init should only be called once.");
     }
-    if (null == providerId) {
-      providerId = DEFAULT_PROVIDER_ID;
-    }
-    final String logPrefix = factory.factoryId + WAL_FILE_NAME_DELIMITER + providerId;
-    log = new IOTestWAL(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf),
-        AbstractFSWALProvider.getWALDirectoryName(factory.factoryId),
-        HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners, true, logPrefix,
-        META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
+    this.factory = factory;
+    this.conf = conf;
+    this.providerId = providerId != null ? providerId : DEFAULT_PROVIDER_ID;
+
+
   }
 
   @Override
   public List<WAL> getWALs() {
-    List<WAL> wals = new ArrayList<>(1);
-    wals.add(log);
-    return wals;
+    return Collections.singletonList(log);
+  }
+
+  private FSHLog createWAL() throws IOException {
+    String logPrefix = factory.factoryId + WAL_FILE_NAME_DELIMITER + providerId;
+    return new IOTestWAL(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf),
+        AbstractFSWALProvider.getWALDirectoryName(factory.factoryId),
+        HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners, true, logPrefix,
+        META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
   }
 
   @Override
   public WAL getWAL(RegionInfo region) throws IOException {
-   return log;
+    FSHLog log = this.log;
+    if (log != null) {
+      return log;
+    }
+    synchronized (this) {
+      log = this.log;
+      if (log == null) {
+        log = createWAL();
+        this.log = log;
+      }
+    }
+    return log;
   }
 
   @Override
   public void close() throws IOException {
-    log.close();
+    FSHLog log = this.log;
+    if (log != null) {
+      log.close();
+    }
   }
 
   @Override
   public void shutdown() throws IOException {
-    log.shutdown();
+    FSHLog log = this.log;
+    if (log != null) {
+      log.shutdown();
+    }
   }
 
   private static class IOTestWAL extends FSHLog {
@@ -255,4 +282,10 @@ public class IOTestProvider implements WALProvider {
   public long getLogFileSize() {
     return this.log.getLogFileSize();
   }
+
+  @Override
+  public void addWALActionsListener(WALActionsListener listener) {
+    // TODO Implement WALProvider.addWALActionLister
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java
index 3a39ee9..b24daa1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java
@@ -185,7 +185,7 @@ public class TestBoundedRegionGroupingStrategy {
       // Set HDFS root directory for storing WAL
       FSUtils.setRootDir(CONF, TEST_UTIL.getDataTestDirOnTestFS());
 
-      wals = new WALFactory(CONF, null, "setMembershipDedups");
+      wals = new WALFactory(CONF, "setMembershipDedups");
       Set<WAL> seen = new HashSet<>(temp * 4);
       int count = 0;
       // we know that this should see one of the wals more than once

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
index 5aea0cf..c3615a2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
@@ -200,7 +200,7 @@ public class TestFSHLogProvider {
     }
     Configuration localConf = new Configuration(conf);
     localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName());
-    WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
+    WALFactory wals = new WALFactory(localConf, currentTest.getMethodName());
     try {
       RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
       RegionInfo hri2 = RegionInfoBuilder.newBuilder(htd2.getTableName()).build();
@@ -280,7 +280,7 @@ public class TestFSHLogProvider {
     }
     Configuration localConf = new Configuration(conf);
     localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName());
-    WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
+    WALFactory wals = new WALFactory(localConf, currentTest.getMethodName());
     try {
       WAL wal = wals.getWAL(null);
       assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(wal));
@@ -354,7 +354,7 @@ public class TestFSHLogProvider {
   public void setMembershipDedups() throws IOException {
     Configuration localConf = new Configuration(conf);
     localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName());
-    WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
+    WALFactory wals = new WALFactory(localConf, currentTest.getMethodName());
     try {
       final Set<WAL> seen = new HashSet<>(1);
       assertTrue("first attempt to add WAL from default provider should work.",

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
index 814320a..8193806 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
@@ -119,7 +119,7 @@ public class TestSecureWAL {
     final byte[] value = Bytes.toBytes("Test value");
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
     final WALFactory wals =
-        new WALFactory(TEST_UTIL.getConfiguration(), null, tableName.getNameAsString());
+        new WALFactory(TEST_UTIL.getConfiguration(), tableName.getNameAsString());
 
     // Write the WAL
     final WAL wal = wals.getWAL(regionInfo);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
index 5679d96..a65d97c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
@@ -105,7 +105,7 @@ public class TestWALFactory {
     fs = cluster.getFileSystem();
     dir = new Path(hbaseDir, currentTest.getMethodName());
     this.currentServername = ServerName.valueOf(currentTest.getMethodName(), 16010, 1);
-    wals = new WALFactory(conf, null, this.currentServername.toString());
+    wals = new WALFactory(conf, this.currentServername.toString());
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
index 9e88f6e..b20b3a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
@@ -106,7 +106,7 @@ public class TestWALMethods {
 
     final Configuration walConf = new Configuration(util.getConfiguration());
     FSUtils.setRootDir(walConf, regiondir);
-    (new WALFactory(walConf, null, "dummyLogName")).getWAL(null);
+    (new WALFactory(walConf, "dummyLogName")).getWAL(null);
 
     NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
     assertEquals(7, files.size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
index 3cbd37e..bc21a65 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
@@ -152,7 +152,7 @@ public class TestWALReaderOnSecureWAL {
       WALProvider.AsyncWriter.class);
     conf.setBoolean(WAL_ENCRYPTION, true);
     FileSystem fs = TEST_UTIL.getTestFileSystem();
-    final WALFactory wals = new WALFactory(conf, null, currentTest.getMethodName());
+    final WALFactory wals = new WALFactory(conf, currentTest.getMethodName());
     Path walPath = writeWAL(wals, currentTest.getMethodName(), offheap);
 
     // Insure edits are not plaintext
@@ -195,9 +195,8 @@ public class TestWALReaderOnSecureWAL {
       WALProvider.Writer.class);
     conf.setBoolean(WAL_ENCRYPTION, false);
     FileSystem fs = TEST_UTIL.getTestFileSystem();
-    final WALFactory wals = new WALFactory(conf, null,
-        ServerName.valueOf(currentTest.getMethodName(), 16010,
-            System.currentTimeMillis()).toString());
+    final WALFactory wals = new WALFactory(conf, ServerName
+        .valueOf(currentTest.getMethodName(), 16010, System.currentTimeMillis()).toString());
     Path walPath = writeWAL(wals, currentTest.getMethodName(), false);
 
     // Ensure edits are plaintext

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
index 412acb6..40fad6a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
@@ -90,7 +90,7 @@ public class TestWALRootDir {
   @Test
   public void testWALRootDir() throws Exception {
     RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build();
-    wals = new WALFactory(conf, null, "testWALRootDir");
+    wals = new WALFactory(conf, "testWALRootDir");
     WAL log = wals.getWAL(regionInfo);
 
     assertEquals(1, getWALFiles(walFs, walRootDir).size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 9b98859..011c9ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -194,7 +194,7 @@ public class TestWALSplit {
     REGIONS.clear();
     Collections.addAll(REGIONS, "bbb", "ccc");
     InstrumentedLogWriter.activateFailure = false;
-    wals = new WALFactory(conf, null, name.getMethodName());
+    wals = new WALFactory(conf, name.getMethodName());
     WALDIR = new Path(HBASELOGDIR,
         AbstractFSWALProvider.getWALDirectoryName(ServerName.valueOf(name.getMethodName(),
             16010, System.currentTimeMillis()).toString()));
@@ -629,7 +629,7 @@ public class TestWALSplit {
         LOG.debug("no previous CORRUPTDIR to clean.");
       }
       // change to the faulty reader
-      wals = new WALFactory(conf, null, name.getMethodName());
+      wals = new WALFactory(conf, name.getMethodName());
       generateWALs(-1);
       // Our reader will render all of these files corrupt.
       final Set<String> walDirContents = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index b735069..7afbb0b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -319,7 +319,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
       rootRegionDir = rootRegionDir.makeQualified(fs.getUri(), fs.getWorkingDirectory());
       cleanRegionRootDir(fs, rootRegionDir);
       FSUtils.setRootDir(getConf(), rootRegionDir);
-      final WALFactory wals = new WALFactory(getConf(), null, "wals");
+      final WALFactory wals = new WALFactory(getConf(), "wals");
       final HRegion[] regions = new HRegion[numRegions];
       final Runnable[] benchmarks = new Runnable[numRegions];
       final MockRegionServerServices mockServices = new MockRegionServerServices(getConf());


[06/41] hbase git commit: Revert "HBASE-19899 Dump ulimit -a, fd count, and free output at end of build into system dir"

Posted by zh...@apache.org.
Revert "HBASE-19899 Dump ulimit -a, fd count, and free output at end of build into system dir"

Doesn't work. Reverting.

This reverts commit 1ca06d9eea680c7b60af6836cb9887b2814f42e5.


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

Branch: refs/heads/HBASE-19064
Commit: 08bcd64c258703a923aa61912d445efbc2d4e937
Parents: 1ca06d9
Author: Michael Stack <st...@apache.org>
Authored: Tue Jan 30 12:01:52 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Jan 30 12:01:52 2018 -0800

----------------------------------------------------------------------
 dev-support/gather_machine_environment.sh | 5 +----
 1 file changed, 1 insertion(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/08bcd64c/dev-support/gather_machine_environment.sh
----------------------------------------------------------------------
diff --git a/dev-support/gather_machine_environment.sh b/dev-support/gather_machine_environment.sh
index db1e74b..589dffe 100755
--- a/dev-support/gather_machine_environment.sh
+++ b/dev-support/gather_machine_environment.sh
@@ -47,8 +47,5 @@ ps -Aww >"${output}/ps-Aww" 2>&1 || true
 ifconfig -a >"${output}/ifconfig-a" 2>&1 || true
 lsblk -ta >"${output}/lsblk-ta" 2>&1 || true
 lsblk -fa >"${output}/lsblk-fa" 2>&1 || true
-ulimit -a >"${output}/ulimit-a" 2>&1 || true
+ulimit -l >"${output}/ulimit-l" 2>&1 || true
 uptime >"${output}/uptime" 2>&1 || true
-free -h >"${output}/free -h" 2>&1 || true
-lsof|wc -l >"${output}/fd-count" 2>&1 || true
-


[08/41] hbase git commit: HBASE-19901 HBASE-19901 Up yetus proclimit on nightlies

Posted by zh...@apache.org.
HBASE-19901 HBASE-19901 Up yetus proclimit on nightlies


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

Branch: refs/heads/HBASE-19064
Commit: 2e6bc12441f50a4de9ee87e96132cb067556a528
Parents: f7faad0
Author: Michael Stack <st...@apache.org>
Authored: Tue Jan 30 21:50:35 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Jan 30 22:25:24 2018 -0800

----------------------------------------------------------------------
 dev-support/hbase-personality.sh   | 5 +++++
 dev-support/hbase_nightly_yetus.sh | 1 +
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2e6bc124/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index e24930d..80d2586 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -64,6 +64,11 @@ function personality_globals
 
   # Override the maven options
   MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
+
+  # Yetus 0.7.0 enforces limits.
+  # Default proclimit is 1000. Up it.
+  # We seem to use close to 3k.
+  PROCLIMIT=5000
 }
 
 ## @description  Parse extra arguments required by personalities, if any.

http://git-wip-us.apache.org/repos/asf/hbase/blob/2e6bc124/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index 4e67354..3e6fc7c 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -67,6 +67,7 @@ YETUS_ARGS=("--whitespace-tabs-ignore-list=${WHITESPACE_IGNORE_LIST}" "${YETUS_A
 YETUS_ARGS=("--sentinel" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--proclimit=${PROCLIMIT}" "${YETUS_ARGS[@]}")
 
 # Currently, flaky list is calculated only for master branch.
 UNDERSCORED_BRANCH_NAME=$(echo ${BRANCH_NAME} | tr '.-' '_')


[32/41] hbase git commit: HBASE-19901 HBASE-19901 Up yetus proclimit on nightlies; AMENDMENT hardcode proclimit and docker memlimit in nightly script...

Posted by zh...@apache.org.
HBASE-19901 HBASE-19901 Up yetus proclimit on nightlies; AMENDMENT hardcode proclimit and docker memlimit in nightly script...


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

Branch: refs/heads/HBASE-19064
Commit: a2bc19aa112b8cd0697845d0825d277ff3a8bcfc
Parents: cb7bfc2
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 1 20:54:13 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 20:54:13 2018 -0800

----------------------------------------------------------------------
 dev-support/hbase_nightly_yetus.sh | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a2bc19aa/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index 651a2e2..27d11c8 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -67,8 +67,9 @@ YETUS_ARGS=("--whitespace-tabs-ignore-list=${WHITESPACE_IGNORE_LIST}" "${YETUS_A
 YETUS_ARGS=("--sentinel" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--proclimit=${PROCLIMIT}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--dockermemlimit=${DOCKERMEMLIMIT}" "${YETUS_ARGS[@]}")
+# Why are these not being picked up from hbase-personality?
+YETUS_ARGS=("--proclimit=10000" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--dockermemlimit=20g" "${YETUS_ARGS[@]}")
 
 # Currently, flaky list is calculated only for master branch.
 UNDERSCORED_BRANCH_NAME=$(echo ${BRANCH_NAME} | tr '.-' '_')


[14/41] hbase git commit: HBASE-19909 TestRegionLocationFinder Timeout

Posted by zh...@apache.org.
HBASE-19909 TestRegionLocationFinder Timeout


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

Branch: refs/heads/HBASE-19064
Commit: 28ee4be0fed60d92b6ba65b135a985644f5ec478
Parents: bc0e9ce
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 31 17:53:48 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jan 31 17:55:16 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/balancer/TestRegionLocationFinder.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/28ee4be0/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
index 59454f5..e4613ea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -42,7 +42,7 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-@Category({MasterTests.class, SmallTests.class})
+@Category({MasterTests.class, MediumTests.class})
 public class TestRegionLocationFinder {
 
   @ClassRule


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

Branch: refs/heads/HBASE-19064
Commit: 42e4b892ac3f97bf60b697f0ef579ab40c149de2
Parents: a7eeb2b
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Jan 13 18:55:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 2 16:05:36 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/42e4b892/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/42e4b892/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/42e4b892/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/42e4b892/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/42e4b892/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/42e4b892/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/42e4b892/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/42e4b892/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/42e4b892/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/42e4b892/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/42e4b892/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"
 


[18/41] hbase git commit: HBASE-19911 Convert some tests from small to medium because they are timing out: TestNettyRpcServer, TestClientClusterStatus; ADDENDUM2 Up timeout for smalltests from 30 seconds to 60 seconds -- a pause on jenkins can mess up sm

Posted by zh...@apache.org.
HBASE-19911 Convert some tests from small to medium because they are timing out: TestNettyRpcServer, TestClientClusterStatus; ADDENDUM2 Up timeout for smalltests from 30 seconds to 60 seconds -- a pause on jenkins can mess up smalltests


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

Branch: refs/heads/HBASE-19064
Commit: 52cb9aaf6e98e89b4891ad50ac6192f54229c021
Parents: 9272f40
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 1 08:24:54 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 08:25:44 2018 -0800

----------------------------------------------------------------------
 .../test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/52cb9aaf/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 74bd70e..734ce3f 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
@@ -63,7 +63,9 @@ public final class HBaseClassTestRule implements TestRule {
     for (Class<?> c : categories[0].value()) {
       if (c == SmallTests.class) {
         // See SmallTests. Supposed to run 15 seconds.
-        return 30;
+        // Lots of these timeout on Jenkins... a stall of ten or twenty seconds mess up what looks
+        // fine when run local.
+        return 60;
       } else if (c == MediumTests.class) {
         // See MediumTests. Supposed to run 50 seconds.
         return 180;


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

Branch: refs/heads/HBASE-19064
Commit: a7eeb2b8c9a450eb15436a71d4fadf704c333b20
Parents: fc6d140
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 11 21:08:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 2 16:05:36 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  26 ++--
 .../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, 536 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a7eeb2b8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index faf3b77..24094e0 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,8 +606,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  @Override
-  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+  protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException {
     try {
       return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup,
         channelClass);
@@ -623,6 +622,11 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    return createAsyncWriter(fs, path);
+  }
+
   private void waitForSafePoint() {
     consumeLock.lock();
     try {
@@ -665,21 +669,21 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     } finally {
       consumeLock.unlock();
     }
-    return executeClose(closeExecutor, oldWriter);
+    return executeClose(oldWriter);
   }
 
   @Override
   protected void doShutdown() throws IOException {
     waitForSafePoint();
-    executeClose(closeExecutor, writer);
+    executeClose(writer);
     closeExecutor.shutdown();
     try {
       if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) {
-        LOG.error("We have waited " + waitOnShutdownInSeconds + " seconds but"
-          + " the close of async writer doesn't complete."
-          + "Please check the status of underlying filesystem"
-          + " or increase the wait time by the config \""
-          + ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS + "\"");
+        LOG.error("We have waited " + waitOnShutdownInSeconds + " seconds but" +
+          " the close of async writer doesn't complete." +
+          "Please check the status of underlying filesystem" +
+          " or increase the wait time by the config \"" + ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS +
+          "\"");
       }
     } catch (InterruptedException e) {
       LOG.error("The wait for close of async writer is interrupted");
@@ -692,7 +696,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  private static long executeClose(ExecutorService closeExecutor, AsyncWriter writer) {
+  protected final long executeClose(AsyncWriter writer) {
     long fileLength;
     if (writer != null) {
       fileLength = writer.getLength();
@@ -700,7 +704,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         try {
           writer.close();
         } catch (IOException e) {
-          LOG.warn("close old writer failed", e);
+          LOG.warn("close writer failed", e);
         }
       });
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7eeb2b8/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/a7eeb2b8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
new file mode 100644
index 0000000..6bf9e02
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
+/**
+ * An AsyncFSWAL which writes data to two filesystems.
+ */
+@InterfaceAudience.Private
+public class DualAsyncFSWAL extends AsyncFSWAL {
+
+  private final FileSystem remoteFs;
+
+  private final Path remoteWalDir;
+
+  public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteRootDir,
+      String logDir, String archiveDir, Configuration conf, List<WALActionsListener> listeners,
+      boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup,
+      Class<? extends Channel> channelClass) throws FailedLogCloseException, IOException {
+    super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix,
+        eventLoopGroup, channelClass);
+    this.remoteFs = remoteFs;
+    this.remoteWalDir = new Path(remoteRootDir, logDir);
+  }
+
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    AsyncWriter localWriter = super.createWriterInstance(path);
+    AsyncWriter remoteWriter;
+    boolean succ = false;
+    try {
+      remoteWriter = createAsyncWriter(remoteFs, new Path(remoteWalDir, path.getName()));
+      succ = true;
+    } finally {
+      if (!succ) {
+        executeClose(localWriter);
+      }
+    }
+    return CombinedAsyncWriter.create(CombinedAsyncWriter.Mode.SEQUENTIAL, remoteWriter,
+      localWriter);
+  }
+}

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


[12/41] hbase git commit: HBASE-19907 TestMetaWithReplicas still flakey

Posted by zh...@apache.org.
HBASE-19907 TestMetaWithReplicas still flakey


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

Branch: refs/heads/HBASE-19064
Commit: 414b2d0889f511089ad575a9c96814a95b5e8797
Parents: 60827fc
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 31 15:17:38 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jan 31 15:17:38 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/TestMetaWithReplicas.java     | 14 ++++++++++++--
 1 file changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/414b2d08/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 1111bb1..736bee0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -113,11 +114,20 @@ public class TestMetaWithReplicas {
     // Fun. All meta region replicas have ended up on the one server. This will cause this test
     // to fail ... sometimes.
     if (sns.size() == 1) {
+      int count = TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size();
+      assertTrue("count=" + count, count == REGIONSERVERS_COUNT);
       LOG.warn("All hbase:meta replicas are on the one server; moving hbase:meta");
       int metaServerIndex = TEST_UTIL.getHBaseCluster().getServerWithMeta();
-      int newServerIndex = (metaServerIndex + 1) % REGIONSERVERS_COUNT;
+      int newServerIndex = metaServerIndex;
+      while (newServerIndex == metaServerIndex) {
+        newServerIndex = (newServerIndex + 1) % REGIONSERVERS_COUNT;
+      }
+      assertNotEquals(metaServerIndex, newServerIndex);
       ServerName destinationServerName =
           TEST_UTIL.getHBaseCluster().getRegionServer(newServerIndex).getServerName();
+      ServerName metaServerName =
+          TEST_UTIL.getHBaseCluster().getRegionServer(metaServerIndex).getServerName();
+      assertNotEquals(destinationServerName, metaServerName);
       TEST_UTIL.getAdmin().move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
           Bytes.toBytes(destinationServerName.toString()));
     }


[24/41] hbase git commit: HBASE-19901 Up yetus proclimit on nightlies; ADDENDUM -- up proclimit from 6k to 10k and set Docker memory limit of 20G (default is 4G)

Posted by zh...@apache.org.
HBASE-19901 Up yetus proclimit on nightlies; ADDENDUM -- up proclimit from 6k to 10k and set Docker memory limit of 20G (default is 4G)


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

Branch: refs/heads/HBASE-19064
Commit: 0db7db3cd1022f794c1cf6d83858e7c83964cd73
Parents: adccbb7
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 1 14:06:02 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 14:08:04 2018 -0800

----------------------------------------------------------------------
 dev-support/hbase-personality.sh   | 11 +++++++----
 dev-support/hbase_nightly_yetus.sh |  1 +
 2 files changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0db7db3c/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 80d2586..5526e09 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -65,10 +65,13 @@ function personality_globals
   # Override the maven options
   MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
 
-  # Yetus 0.7.0 enforces limits.
-  # Default proclimit is 1000. Up it.
-  # We seem to use close to 3k.
-  PROCLIMIT=5000
+  # Yetus 0.7.0 enforces limits. Default proclimit is 1000.
+  # Up it. See HBASE-19902 for how we arrived at this number. 
+  PROCLIMIT=10000
+
+  # Set docker container to run with 20g. Default is 4g in yetus.
+  # See HBASE-19902 for how we arrived at 20g.
+  DOCKERMEMLIMIT=20g
 }
 
 ## @description  Parse extra arguments required by personalities, if any.

http://git-wip-us.apache.org/repos/asf/hbase/blob/0db7db3c/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index 3e6fc7c..651a2e2 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -68,6 +68,7 @@ YETUS_ARGS=("--sentinel" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--proclimit=${PROCLIMIT}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--dockermemlimit=${DOCKERMEMLIMIT}" "${YETUS_ARGS[@]}")
 
 # Currently, flaky list is calculated only for master branch.
 UNDERSCORED_BRANCH_NAME=$(echo ${BRANCH_NAME} | tr '.-' '_')


[05/41] hbase git commit: HBASE-19899 Dump ulimit -a, fd count, and free output at end of build into system dir

Posted by zh...@apache.org.
HBASE-19899 Dump ulimit -a, fd count, and free output at end of build into system dir


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

Branch: refs/heads/HBASE-19064
Commit: 1ca06d9eea680c7b60af6836cb9887b2814f42e5
Parents: 194890b
Author: Michael Stack <st...@apache.org>
Authored: Tue Jan 30 11:53:09 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Jan 30 11:53:09 2018 -0800

----------------------------------------------------------------------
 dev-support/gather_machine_environment.sh | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1ca06d9e/dev-support/gather_machine_environment.sh
----------------------------------------------------------------------
diff --git a/dev-support/gather_machine_environment.sh b/dev-support/gather_machine_environment.sh
index 589dffe..db1e74b 100755
--- a/dev-support/gather_machine_environment.sh
+++ b/dev-support/gather_machine_environment.sh
@@ -47,5 +47,8 @@ ps -Aww >"${output}/ps-Aww" 2>&1 || true
 ifconfig -a >"${output}/ifconfig-a" 2>&1 || true
 lsblk -ta >"${output}/lsblk-ta" 2>&1 || true
 lsblk -fa >"${output}/lsblk-fa" 2>&1 || true
-ulimit -l >"${output}/ulimit-l" 2>&1 || true
+ulimit -a >"${output}/ulimit-a" 2>&1 || true
 uptime >"${output}/uptime" 2>&1 || true
+free -h >"${output}/free -h" 2>&1 || true
+lsof|wc -l >"${output}/fd-count" 2>&1 || true
+


[21/41] hbase git commit: HBASE-19912 Remove useless 'writeToWAL' flag of Region#checkAndRowMutate

Posted by zh...@apache.org.
HBASE-19912 Remove useless 'writeToWAL' flag of Region#checkAndRowMutate

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

Branch: refs/heads/HBASE-19064
Commit: 38c8144a065bc6d330330f611ec8beaa1477a884
Parents: 57911d0
Author: kewang <cp...@gmail.com>
Authored: Thu Feb 1 18:22:05 2018 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 12:33:48 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/HRegion.java     | 10 ++++------
 .../apache/hadoop/hbase/regionserver/RSRpcServices.java   |  2 +-
 .../java/org/apache/hadoop/hbase/regionserver/Region.java |  3 +--
 .../org/apache/hadoop/hbase/regionserver/TestHRegion.java |  2 +-
 4 files changed, 7 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/38c8144a/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 ae0f3d1..ecc9a74 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
@@ -3919,16 +3919,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   throws IOException{
     checkMutationType(mutation, row);
     return doCheckAndRowMutate(row, family, qualifier, op, comparator, null,
-      mutation, writeToWAL);
+      mutation);
   }
 
   @Override
   public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
-                                   CompareOperator op, ByteArrayComparable comparator, RowMutations rm,
-                                   boolean writeToWAL)
+    CompareOperator op, ByteArrayComparable comparator, RowMutations rm)
   throws IOException {
-    return doCheckAndRowMutate(row, family, qualifier, op, comparator, rm, null,
-      writeToWAL);
+    return doCheckAndRowMutate(row, family, qualifier, op, comparator, rm, null);
   }
 
   /**
@@ -3937,7 +3935,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   private boolean doCheckAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
                                       CompareOperator op, ByteArrayComparable comparator, RowMutations rowMutations,
-                                      Mutation mutation, boolean writeToWAL)
+                                      Mutation mutation)
   throws IOException {
     // Could do the below checks but seems wacky with two callers only. Just comment out for now.
     // One caller passes a Mutation, the other passes RowMutation. Presume all good so we don't

http://git-wip-us.apache.org/repos/asf/hbase/blob/38c8144a/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 e540464..48eac79 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
@@ -619,7 +619,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           resultOrExceptionOrBuilder.build());
     }
     return region.checkAndRowMutate(row, family, qualifier, op,
-        comparator, rm, Boolean.TRUE);
+        comparator, rm);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/38c8144a/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 75f02a3..52d01fe 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
@@ -309,12 +309,11 @@ public interface Region extends ConfigurationObserver {
    * @param op the comparison operator
    * @param comparator
    * @param mutations
-   * @param writeToWAL
    * @return true if mutations were applied, false otherwise
    * @throws IOException
    */
   boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier, CompareOperator op,
-      ByteArrayComparable comparator, RowMutations mutations, boolean writeToWAL)
+      ByteArrayComparable comparator, RowMutations mutations)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/38c8144a/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 04dcb92..0d9db31 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
@@ -6387,7 +6387,7 @@ public class TestHRegion {
     RowMutations rm = new RowMutations(row);
     rm.add(p);
     assertTrue(region.checkAndRowMutate(row, fam1, qual1, CompareOperator.EQUAL,
-        new BinaryComparator(qual1), rm, false));
+        new BinaryComparator(qual1), rm));
     result = region.get(new Get(row));
     c = result.getColumnLatestCell(fam1, qual1);
     assertEquals(10L, c.getTimestamp());


[13/41] hbase git commit: HBASE-19908 TestCoprocessorShortCircuitRPC Timeout....

Posted by zh...@apache.org.
HBASE-19908 TestCoprocessorShortCircuitRPC Timeout....


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

Branch: refs/heads/HBASE-19064
Commit: bc0e9ce8f462d8b3f4ca67ad80d455edb39605bf
Parents: 414b2d0
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 31 17:32:39 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jan 31 17:32:39 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bc0e9ce8/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java
index c3f7e80..6cff379 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorShortCircuitRPC.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.SharedConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -41,7 +41,7 @@ import org.junit.rules.TestName;
  * Ensure Coprocessors get ShortCircuit Connections when they get a Connection from their
  * CoprocessorEnvironment.
  */
-@Category({CoprocessorTests.class, SmallTests.class})
+@Category({CoprocessorTests.class, MediumTests.class})
 public class TestCoprocessorShortCircuitRPC {
 
   @ClassRule


[23/41] hbase git commit: HBASE-19897 RowMutations should follow the fluent pattern

Posted by zh...@apache.org.
HBASE-19897 RowMutations should follow the fluent pattern


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

Branch: refs/heads/HBASE-19064
Commit: adccbb7edf3986701d54b9ca93fcf5a8e99548fb
Parents: d472422
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Feb 1 13:58:54 2018 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Feb 2 05:33:08 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/RowMutations.java       | 66 +++++++++++++++-----
 .../hbase/client/TestFromClientSide3.java       | 24 +++----
 2 files changed, 61 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/adccbb7e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
index 4ff9eb1..1eb3151 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
@@ -22,9 +22,9 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Performs multiple mutations atomically on a single row.
@@ -38,6 +38,21 @@ import org.apache.hadoop.hbase.util.Bytes;
  */
 @InterfaceAudience.Public
 public class RowMutations implements Row {
+
+  /**
+   * Create a {@link RowMutations} with the specified mutations.
+   * @param mutations the mutations to send
+   * @return RowMutations
+   * @throws IOException if any row in mutations is different to another
+   */
+  public static RowMutations of(List<? extends Mutation> mutations) throws IOException {
+    if (CollectionUtils.isEmpty(mutations)) {
+      throw new IllegalArgumentException("Can't instantiate a RowMutations by empty list");
+    }
+    return new RowMutations(mutations.get(0).getRow(), mutations.size())
+        .add(mutations);
+  }
+
   private final List<Mutation> mutations;
   private final byte [] row;
 
@@ -50,8 +65,7 @@ public class RowMutations implements Row {
    * @param initialCapacity the initial capacity of the RowMutations
    */
   public RowMutations(byte [] row, int initialCapacity) {
-    Mutation.checkRow(row);
-    this.row = Bytes.copy(row);
+    this.row = Bytes.copy(Mutation.checkRow(row));
     if (initialCapacity <= 0) {
       this.mutations = new ArrayList<>();
     } else {
@@ -62,29 +76,53 @@ public class RowMutations implements Row {
   /**
    * Add a {@link Put} operation to the list of mutations
    * @param p The {@link Put} to add
-   * @throws IOException
+   * @throws IOException if the row of added mutation doesn't match the original row
+   * @deprecated since 2.0 version and will be removed in 3.0 version.
+   *             use {@link #add(Mutation)}
    */
+  @Deprecated
   public void add(Put p) throws IOException {
-    internalAdd(p);
+    add((Mutation) p);
   }
 
   /**
    * Add a {@link Delete} operation to the list of mutations
    * @param d The {@link Delete} to add
-   * @throws IOException
+   * @throws IOException if the row of added mutation doesn't match the original row
+   * @deprecated since 2.0 version and will be removed in 3.0 version.
+   *             use {@link #add(Mutation)}
    */
+  @Deprecated
   public void add(Delete d) throws IOException {
-    internalAdd(d);
+    add((Mutation) d);
   }
 
-  private void internalAdd(Mutation m) throws IOException {
-    int res = Bytes.compareTo(this.row, m.getRow());
-    if (res != 0) {
-      throw new WrongRowIOException("The row in the recently added Put/Delete <" +
-          Bytes.toStringBinary(m.getRow()) + "> doesn't match the original one <" +
+  /**
+   * Currently only supports {@link Put} and {@link Delete} mutations.
+   *
+   * @param mutation The data to send.
+   * @throws IOException if the row of added mutation doesn't match the original row
+   */
+  public RowMutations add(Mutation mutation) throws IOException {
+    return add(Collections.singletonList(mutation));
+  }
+
+  /**
+   * Currently only supports {@link Put} and {@link Delete} mutations.
+   *
+   * @param mutations The data to send.
+   * @throws IOException if the row of added mutation doesn't match the original row
+   */
+  public RowMutations add(List<? extends Mutation> mutations) throws IOException {
+    for (Mutation mutation : mutations) {
+      if (!Bytes.equals(row, mutation.getRow())) {
+        throw new WrongRowIOException("The row in the recently added Put/Delete <" +
+          Bytes.toStringBinary(mutation.getRow()) + "> doesn't match the original one <" +
           Bytes.toStringBinary(this.row) + ">");
+      }
     }
-    mutations.add(m);
+    this.mutations.addAll(mutations);
+    return this;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/adccbb7e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 36b3b90..60c124a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.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.Collections;
 import java.util.List;
 import java.util.Optional;
 import java.util.Random;
@@ -452,10 +453,9 @@ public class TestFromClientSide3 {
       byte [][] QUALIFIERS = new byte [][] {
         Bytes.toBytes("a"), Bytes.toBytes("b")
       };
-      RowMutations arm = new RowMutations(ROW);
-      Put p = new Put(ROW);
-      p.addColumn(FAMILY, QUALIFIERS[0], VALUE);
-      arm.add(p);
+
+      RowMutations arm = RowMutations.of(Collections.singletonList(
+        new Put(ROW).addColumn(FAMILY, QUALIFIERS[0], VALUE)));
       Object[] batchResult = new Object[1];
       t.batch(Arrays.asList(arm), batchResult);
 
@@ -463,13 +463,9 @@ public class TestFromClientSide3 {
       Result r = t.get(g);
       assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[0])));
 
-      arm = new RowMutations(ROW);
-      p = new Put(ROW);
-      p.addColumn(FAMILY, QUALIFIERS[1], VALUE);
-      arm.add(p);
-      Delete d = new Delete(ROW);
-      d.addColumns(FAMILY, QUALIFIERS[0]);
-      arm.add(d);
+      arm = RowMutations.of(Arrays.asList(
+        new Put(ROW).addColumn(FAMILY, QUALIFIERS[1], VALUE),
+        new Delete(ROW).addColumns(FAMILY, QUALIFIERS[0])));
       t.batch(Arrays.asList(arm), batchResult);
       r = t.get(g);
       assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1])));
@@ -477,10 +473,8 @@ public class TestFromClientSide3 {
 
       // Test that we get the correct remote exception for RowMutations from batch()
       try {
-        arm = new RowMutations(ROW);
-        p = new Put(ROW);
-        p.addColumn(new byte[]{'b', 'o', 'g', 'u', 's'}, QUALIFIERS[0], VALUE);
-        arm.add(p);
+        arm = RowMutations.of(Collections.singletonList(
+          new Put(ROW).addColumn(new byte[]{'b', 'o', 'g', 'u', 's'}, QUALIFIERS[0], VALUE)));
         t.batch(Arrays.asList(arm), batchResult);
         fail("Expected RetriesExhaustedWithDetailsException with NoSuchColumnFamilyException");
       } catch(RetriesExhaustedWithDetailsException e) {


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

Branch: refs/heads/HBASE-19064
Commit: 6adf213af31f7759b8ad3d9e41cb2b737fd2ff76
Parents: 9e94d15
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 25 20:09:00 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 2 16:45:59 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/6adf213a/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/6adf213a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
index 604e0bb..5ec14cd 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
@@ -36,6 +36,8 @@ public class ReplicationPeerImpl implements ReplicationPeer {
 
   private volatile PeerState peerState;
 
+  private volatile SyncReplicationState syncReplicationState;
+
   private final List<ReplicationPeerConfigListener> peerConfigListeners;
 
   /**
@@ -45,12 +47,13 @@ public class ReplicationPeerImpl implements ReplicationPeer {
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerImpl(Configuration conf, String id, boolean peerState,
-      ReplicationPeerConfig peerConfig) {
+  public ReplicationPeerImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig,
+      boolean peerState, SyncReplicationState syncReplicationState) {
     this.conf = conf;
     this.id = id;
     this.peerState = peerState ? PeerState.ENABLED : PeerState.DISABLED;
     this.peerConfig = peerConfig;
+    this.syncReplicationState = syncReplicationState;
     this.peerConfigListeners = new ArrayList<>();
   }
 
@@ -77,37 +80,26 @@ public class ReplicationPeerImpl implements ReplicationPeer {
     return peerState;
   }
 
-  /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
+  @Override
+  public SyncReplicationState getSyncReplicationState() {
+    return syncReplicationState;
+  }
+
   @Override
   public ReplicationPeerConfig getPeerConfig() {
     return peerConfig;
   }
 
-  /**
-   * Get the configuration object required to communicate with this peer
-   * @return configuration object
-   */
   @Override
   public Configuration getConfiguration() {
     return conf;
   }
 
-  /**
-   * Get replicable (table, cf-list) map of this peer
-   * @return the replicable (table, cf-list) map
-   */
   @Override
   public Map<TableName, List<String>> getTableCFs() {
     return this.peerConfig.getTableCFsMap();
   }
 
-  /**
-   * Get replicable namespace set of this peer
-   * @return the replicable namespaces set
-   */
   @Override
   public Set<String> getNamespaces() {
     return this.peerConfig.getNamespaces();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6adf213a/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/6adf213a/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/6adf213a/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/6adf213a/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/6adf213a/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/6adf213a/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/6adf213a/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/6adf213a/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 350f4ec..d8a6442 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,6 +174,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/6adf213a/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/6adf213a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
deleted file mode 100644
index e6031c6..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.wal;
-
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-import java.io.IOException;
-import java.util.Optional;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
-import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
-import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogTestHelper;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ RegionServerTests.class, MediumTests.class })
-public class TestSynchronousReplicationWALProvider {
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static String PEER_ID = "1";
-
-  private static String REMOTE_WAL_DIR = "/RemoteWAL";
-
-  private static TableName TABLE = TableName.valueOf("table");
-
-  private static TableName TABLE_NO_REP = TableName.valueOf("table-no-rep");
-
-  private static RegionInfo REGION = RegionInfoBuilder.newBuilder(TABLE).build();
-
-  private static RegionInfo REGION_NO_REP = RegionInfoBuilder.newBuilder(TABLE_NO_REP).build();
-
-  private static WALFactory FACTORY;
-
-  private static Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
-    if (info.getTable().equals(TABLE)) {
-      return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR));
-    } else {
-      return Optional.empty();
-    }
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    UTIL.startMiniDFSCluster(3);
-    FACTORY = new WALFactory(UTIL.getConfiguration(), "test",
-        TestSynchronousReplicationWALProvider::getPeerIdAndRemoteWALDir);
-    UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID));
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws IOException {
-    FACTORY.close();
-    UTIL.shutdownMiniDFSCluster();
-  }
-
-  private void testReadWrite(DualAsyncFSWAL wal) throws Exception {
-    int recordCount = 100;
-    int columnCount = 10;
-    byte[] row = Bytes.toBytes("testRow");
-    long timestamp = System.currentTimeMillis();
-    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
-    ProtobufLogTestHelper.doWrite(wal, REGION, TABLE, columnCount, recordCount, row, timestamp,
-      mvcc);
-    Path localFile = wal.getCurrentFileName();
-    Path remoteFile = new Path(REMOTE_WAL_DIR + "/" + PEER_ID, localFile.getName());
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
-      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
-      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    wal.rollWriter();
-    DistributedFileSystem dfs = (DistributedFileSystem) UTIL.getDFSCluster().getFileSystem();
-    UTIL.waitFor(5000, new ExplainingPredicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return dfs.isFileClosed(localFile) && dfs.isFileClosed(remoteFile);
-      }
-
-      @Override
-      public String explainFailure() throws Exception {
-        StringBuilder sb = new StringBuilder();
-        if (!dfs.isFileClosed(localFile)) {
-          sb.append(localFile + " has not been closed yet.");
-        }
-        if (!dfs.isFileClosed(remoteFile)) {
-          sb.append(remoteFile + " has not been closed yet.");
-        }
-        return sb.toString();
-      }
-    });
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
-      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
-      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    WAL walNoRep = FACTORY.getWAL(REGION_NO_REP);
-    assertThat(walNoRep, not(instanceOf(DualAsyncFSWAL.class)));
-    DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION);
-    assertEquals(2, FACTORY.getWALs().size());
-    testReadWrite(wal);
-    SynchronousReplicationWALProvider walProvider =
-      (SynchronousReplicationWALProvider) FACTORY.getWALProvider();
-    walProvider.peerRemoved(PEER_ID);
-    assertEquals(1, FACTORY.getWALs().size());
-  }
-}


[17/41] hbase git commit: HBASE-19911 Convert some tests from small to medium because they are timing out: TestNettyRpcServer, TestClientClusterStatus; ADDENDUM changes TestCheckTestClasses to be medium sized test

Posted by zh...@apache.org.
    HBASE-19911 Convert some tests from small to medium because they are timing out: TestNettyRpcServer, TestClientClusterStatus; ADDENDUM changes TestCheckTestClasses to be medium sized test


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

Branch: refs/heads/HBASE-19064
Commit: 9272f40a5c8afeeaa9569688c0fabfcd9264ee5e
Parents: b9cb118
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 31 20:27:34 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jan 31 20:27:34 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/9272f40a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
index 1866981..a20ef73 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.hbase;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
+
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -29,7 +31,7 @@ import org.junit.experimental.categories.Category;
 /**
  * Checks tests are categorized.
  */
-@Category({MiscTests.class, SmallTests.class})
+@Category({MiscTests.class, MediumTests.class})
 public class TestCheckTestClasses {
 
   @ClassRule


[20/41] hbase git commit: HBASE-19839 Fixed flakey tests TestMergeTableRegionsProcedure#testRollbackAndDoubleExecution and TestSplitTableRegionProcedure#testRollbackAndDoubleExecution

Posted by zh...@apache.org.
HBASE-19839 Fixed flakey tests TestMergeTableRegionsProcedure#testRollbackAndDoubleExecution and TestSplitTableRegionProcedure#testRollbackAndDoubleExecution

* Added a comment in MergeTableRegionsProcedure and SplitTableRegionProcedure explaining specific rollbacks has side effect that AssignProcedure/s are submitted asynchronously and those procedures may continue to execute even after rollback() is done.
* Updated comments in tests with correct rollback state to abort
* Added overloaded method MasterProcedureTestingUtility#testRollbackAndDoubleExecution which takes additional argument for waiting on all procedures to finish before asserting conditions
* Updated TestMergeTableRegionsProcedure#testRollbackAndDoubleExecution and TestSplitTableRegionProcedure#testRollbackAndDoubleExecution to use newly added method

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

Branch: refs/heads/HBASE-19064
Commit: 57911d02c65b80eb198530b2a7d1fa39dba11a2c
Parents: b21b8bf
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Mon Jan 22 13:23:41 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 12:01:30 2018 -0800

----------------------------------------------------------------------
 .../assignment/MergeTableRegionsProcedure.java   |  6 ++++++
 .../assignment/SplitTableRegionProcedure.java    |  6 ++++++
 .../TestMergeTableRegionsProcedure.java          |  7 ++++---
 .../TestSplitTableRegionProcedure.java           |  7 ++++---
 .../procedure/MasterProcedureTestingUtility.java | 19 +++++++++++++++++++
 5 files changed, 39 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/57911d02/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 8c59776..4bccab7 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
@@ -263,6 +263,12 @@ public class MergeTableRegionsProcedure
     return Flow.HAS_MORE_STATE;
   }
 
+  /**
+   * To rollback {@link MergeTableRegionsProcedure}, two AssignProcedures are asynchronously
+   * submitted for each region to be merged (rollback doesn't wait on the completion of the
+   * AssignProcedures) . This can be improved by changing rollback() to support sub-procedures.
+   * See HBASE-19851 for details.
+   */
   @Override
   protected void rollbackState(
       final MasterProcedureEnv env,

http://git-wip-us.apache.org/repos/asf/hbase/blob/57911d02/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 1513c25..88e6012 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
@@ -263,6 +263,12 @@ public class SplitTableRegionProcedure
     return Flow.HAS_MORE_STATE;
   }
 
+  /**
+   * To rollback {@link SplitTableRegionProcedure}, an AssignProcedure is asynchronously
+   * submitted for parent region to be split (rollback doesn't wait on the completion of the
+   * AssignProcedure) . This can be improved by changing rollback() to support sub-procedures.
+   * See HBASE-19851 for details.
+   */
   @Override
   protected void rollbackState(final MasterProcedureEnv env, final SplitTableRegionState state)
       throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/57911d02/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 5b70e20..3285d3d 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
@@ -266,11 +266,12 @@ public class TestMergeTableRegionsProcedure {
     long procId = procExec.submitProcedure(
       new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true));
 
-    // Failing before MERGE_TABLE_REGIONS_UPDATE_META we should trigger the rollback
-    // NOTE: the 5 (number before MERGE_TABLE_REGIONS_UPDATE_META step) is
+    // Failing before MERGE_TABLE_REGIONS_CREATE_MERGED_REGION we should trigger the rollback
+    // NOTE: the 5 (number before MERGE_TABLE_REGIONS_CREATE_MERGED_REGION step) is
     // hardcoded, so you have to look at this test at least once when you add a new step.
     int numberOfSteps = 5;
-    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps,
+        true);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/57911d02/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
index 1edb8e5..32b7539 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
@@ -366,12 +366,13 @@ public class TestSplitTableRegionProcedure {
     long procId = procExec.submitProcedure(
       new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
 
-    // Failing before SPLIT_TABLE_REGION_UPDATE_META we should trigger the
+    // Failing before SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS we should trigger the
     // rollback
-    // NOTE: the 3 (number before SPLIT_TABLE_REGION_UPDATE_META step) is
+    // NOTE: the 3 (number before SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS step) is
     // hardcoded, so you have to look at this test at least once when you add a new step.
     int numberOfSteps = 3;
-    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps,
+        true);
     // check that we have only 1 region
     assertEquals(1, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
     List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/57911d02/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 243bb14..fc8953b 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
@@ -427,6 +427,12 @@ public class MasterProcedureTestingUtility {
   public static void testRollbackAndDoubleExecution(
       final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
       final int lastStep) throws Exception {
+    testRollbackAndDoubleExecution(procExec, procId, lastStep, false);
+  }
+
+  public static void testRollbackAndDoubleExecution(
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
+      final int lastStep, boolean waitForAsyncProcs) throws Exception {
     // Execute up to last step
     testRecoveryAndDoubleExecution(procExec, procId, lastStep, false);
 
@@ -448,6 +454,19 @@ public class MasterProcedureTestingUtility {
       assertTrue(procExec.unregisterListener(abortListener));
     }
 
+    if (waitForAsyncProcs) {
+      // Sometimes there are other procedures still executing (including asynchronously spawned by
+      // procId) and due to KillAndToggleBeforeStoreUpdate flag ProcedureExecutor is stopped before
+      // store update. Let all pending procedures finish normally.
+      if (!procExec.isRunning()) {
+        LOG.warn("ProcedureExecutor not running, may have been stopped by pending procedure due to"
+            + " KillAndToggleBeforeStoreUpdate flag.");
+        ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+        restartMasterProcedureExecutor(procExec);
+        ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+      }
+    }
+
     assertEquals(true, procExec.isRunning());
     ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
   }


[03/41] hbase git commit: HBASE-19892 Checking patch attach and yetus 0.7.0 and move to Yetus 0.7.0

Posted by zh...@apache.org.
HBASE-19892 Checking patch attach and yetus 0.7.0 and move to Yetus 0.7.0

One-liner that ups our yetus version from 0.6.0 to 0.7.0.


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

Branch: refs/heads/HBASE-19064
Commit: 67f91f1282369d805df14cec5c99f36c42b192ba
Parents: 9b8d7e0
Author: Michael Stack <st...@apache.org>
Authored: Mon Jan 29 22:34:31 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Jan 29 22:34:40 2018 -0800

----------------------------------------------------------------------
 dev-support/Jenkinsfile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/67f91f12/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index c7adfa5..201783b 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -33,7 +33,7 @@ pipeline {
     TOOLS = "${env.WORKSPACE}/tools"
     // where we check out to across stages
     BASEDIR = "${env.WORKSPACE}/component"
-    YETUS_RELEASE = '0.6.0'
+    YETUS_RELEASE = '0.7.0'
     PROJECT = 'hbase'
     PROJECT_PERSONALITY = 'https://raw.githubusercontent.com/apache/hbase/master/dev-support/hbase-personality.sh'
     // This section of the docs tells folks not to use the javadoc tag. older branches have our old version of the check for said tag.


[34/41] hbase git commit: HBASE-19904 Break dependency of WAL constructor on Replication

Posted by zh...@apache.org.
HBASE-19904 Break dependency of WAL constructor on Replication


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

Branch: refs/heads/HBASE-19064
Commit: fc6d140adf0b382e0b7bfef02ae96be7908036e1
Parents: a2bc19a
Author: zhangduo <zh...@apache.org>
Authored: Fri Feb 2 14:10:29 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 2 14:10:29 2018 +0800

----------------------------------------------------------------------
 .../hbase/mapreduce/TestWALRecordReader.java    |   4 +-
 .../hbase/replication/ReplicationUtils.java     |  10 ++
 .../org/apache/hadoop/hbase/master/HMaster.java |  27 ++-
 .../hadoop/hbase/regionserver/HRegion.java      |  49 +++---
 .../hbase/regionserver/HRegionServer.java       | 108 +++++-------
 .../hbase/regionserver/ReplicationService.java  |  16 +-
 .../regionserver/ReplicationSourceService.java  |   7 -
 .../replication/regionserver/Replication.java   | 176 ++++---------------
 .../regionserver/ReplicationSourceManager.java  |  72 ++++++--
 .../regionserver/ReplicationSyncUp.java         |   3 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   6 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |  43 +++--
 .../hadoop/hbase/wal/DisabledWALProvider.java   |   8 +-
 .../apache/hadoop/hbase/wal/FSHLogProvider.java |  10 +-
 .../hbase/wal/RegionGroupingProvider.java       |  35 ++--
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  42 ++---
 .../apache/hadoop/hbase/wal/WALProvider.java    |  46 +++--
 .../hadoop/hbase/HBaseTestingUtility.java       |   6 +-
 .../hbase/coprocessor/TestWALObserver.java      |   7 +-
 .../hbase/master/cleaner/TestLogsCleaner.java   |   4 +-
 .../cleaner/TestReplicationHFileCleaner.java    |   4 +-
 .../regionserver/TestCacheOnWriteInSchema.java  |   2 +-
 .../TestCompactionArchiveConcurrentClose.java   |   2 +-
 .../TestCompactionArchiveIOException.java       |   2 +-
 .../hbase/regionserver/TestDefaultMemStore.java |   2 +-
 .../hbase/regionserver/TestHMobStore.java       |   2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  23 +--
 .../hadoop/hbase/regionserver/TestHStore.java   |   2 +-
 .../TestStoreFileRefresherChore.java            |   2 +-
 .../TestWALMonotonicallyIncreasingSeqId.java    |   2 +-
 .../wal/AbstractTestLogRolling.java             |   6 +-
 .../wal/AbstractTestProtobufLog.java            |   3 +-
 .../regionserver/wal/AbstractTestWALReplay.java |  11 +-
 .../hbase/regionserver/wal/TestDurability.java  |   6 +-
 .../regionserver/wal/TestLogRollAbort.java      |   2 +-
 .../wal/TestLogRollingNoCluster.java            |   2 +-
 .../wal/TestWALActionsListener.java             |   7 +-
 .../TestReplicationEmptyWALRecovery.java        |   6 +-
 .../replication/TestReplicationSmallTests.java  |  18 --
 .../TestReplicationSourceManager.java           |  51 +++++-
 .../regionserver/TestWALEntryStream.java        |   7 +-
 .../apache/hadoop/hbase/wal/IOTestProvider.java |  69 ++++++--
 .../wal/TestBoundedRegionGroupingStrategy.java  |   2 +-
 .../hadoop/hbase/wal/TestFSHLogProvider.java    |   6 +-
 .../apache/hadoop/hbase/wal/TestSecureWAL.java  |   2 +-
 .../apache/hadoop/hbase/wal/TestWALFactory.java |   2 +-
 .../apache/hadoop/hbase/wal/TestWALMethods.java |   2 +-
 .../hbase/wal/TestWALReaderOnSecureWAL.java     |   7 +-
 .../apache/hadoop/hbase/wal/TestWALRootDir.java |   2 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |   4 +-
 .../hbase/wal/WALPerformanceEvaluation.java     |   2 +-
 51 files changed, 465 insertions(+), 474 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
index 87d100b..e486714 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
@@ -127,7 +127,7 @@ public class TestWALRecordReader {
    */
   @Test
   public void testPartialRead() throws Exception {
-    final WALFactory walfactory = new WALFactory(conf, null, getName());
+    final WALFactory walfactory = new WALFactory(conf, getName());
     WAL log = walfactory.getWAL(info);
     // This test depends on timestamp being millisecond based and the filename of the WAL also
     // being millisecond based.
@@ -186,7 +186,7 @@ public class TestWALRecordReader {
    */
   @Test
   public void testWALRecordReader() throws Exception {
-    final WALFactory walfactory = new WALFactory(conf, null, getName());
+    final WALFactory walfactory = new WALFactory(conf, getName());
     WAL log = walfactory.getWAL(info);
     byte [] value = Bytes.toBytes("value");
     WALEdit edit = new WALEdit();

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 ebe68a7..11507aa 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
@@ -25,6 +25,7 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -122,4 +123,13 @@ public final class ReplicationUtils {
         isTableCFsEqual(rpc1.getTableCFsMap(), rpc2.getTableCFsMap());
     }
   }
+
+  /**
+   * @param c Configuration to look at
+   * @return True if replication for bulk load data is enabled.
+   */
+  public static boolean isReplicationForBulkLoadDataEnabled(final Configuration c) {
+    return c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 d422960..dc1763c 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
+
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Service;
 import java.io.IOException;
@@ -166,8 +168,10 @@ import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
 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.master.ReplicationHFileCleaner;
+import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader;
-import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.trace.TraceUtil;
@@ -484,7 +488,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       // Disable usage of meta replicas in the master
       this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
 
-      Replication.decorateMasterConfiguration(this.conf);
+      decorateMasterConfiguration(this.conf);
 
       // Hack! Maps DFSClient => Master for logs.  HDFS made this
       // config param for task trackers, but we can piggyback off of it.
@@ -3557,4 +3561,23 @@ public class HMaster extends HRegionServer implements MasterServices {
   public ReplicationPeerManager getReplicationPeerManager() {
     return replicationPeerManager;
   }
+
+  /**
+   * This method modifies the master's configuration in order to inject replication-related features
+   */
+  @VisibleForTesting
+  public static void decorateMasterConfiguration(Configuration conf) {
+    String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
+    String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
+    if (!plugins.contains(cleanerClass)) {
+      conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
+    }
+    if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
+      plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
+      cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();
+      if (!plugins.contains(cleanerClass)) {
+        conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 ecc9a74..7a6af75 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -69,7 +71,6 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -148,28 +149,6 @@ import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
-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.common.collect.Lists;
-import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
-import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
-import org.apache.hbase.thirdparty.com.google.protobuf.Service;
-import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
-import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.trace.TraceUtil;
@@ -200,7 +179,29 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import edu.umd.cs.findbugs.annotations.Nullable;
+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.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+import org.apache.hbase.thirdparty.com.google.protobuf.Service;
+import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
+import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
 
 /**
  * Regions store data for a certain region of a table.  It stores all columns

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 3844415..cb7e2d7 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
@@ -131,10 +131,9 @@ import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler;
 import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler;
 import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.replication.regionserver.Replication;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
@@ -158,6 +157,7 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -546,7 +546,7 @@ public class HRegionServer extends HasThread implements
       checkCodecs(this.conf);
       this.userProvider = UserProvider.instantiate(conf);
       FSUtils.setupShortCircuitRead(this.conf);
-      Replication.decorateRegionServerConfiguration(this.conf);
+      decorateRegionServerConfiguration(this.conf);
 
       // Disable usage of meta replicas in the regionserver
       this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
@@ -1781,52 +1781,26 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * Setup WAL log and replication if enabled.
-   * Replication setup is done in here because it wants to be hooked up to WAL.
-   *
-   * @throws IOException
+   * Setup WAL log and replication if enabled. Replication setup is done in here because it wants to
+   * be hooked up to WAL.
    */
   private void setupWALAndReplication() throws IOException {
+    WALFactory factory = new WALFactory(conf, 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);
-    if (LOG.isDebugEnabled()) LOG.debug("logDir=" + logDir);
+    LOG.debug("logDir={}", logDir);
     if (this.walFs.exists(logDir)) {
-      throw new RegionServerRunningException("Region server has already " +
-          "created directory at " + this.serverName.toString());
+      throw new RegionServerRunningException(
+          "Region server has already created directory at " + this.serverName.toString());
     }
-
-    // Instantiate replication if replication enabled.  Pass it the log directories.
-    // In here we create the Replication instances. Later they are initialized and started up.
-    createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir);
-
-    // listeners the wal factory will add to wals it creates.
-    List<WALActionsListener> listeners = new ArrayList<>();
-    listeners.add(new MetricsWAL());
-    if (this.replicationSourceHandler != null &&
-        this.replicationSourceHandler.getWALActionsListener() != null) {
-      // Replication handler is an implementation of WALActionsListener.
-      listeners.add(this.replicationSourceHandler.getWALActionsListener());
-    }
-
-    // There is a cyclic dependency between ReplicationSourceHandler and WALFactory.
-    // We use WALActionsListener to get the newly rolled WALs, so we need to get the
-    // WALActionsListeners from ReplicationSourceHandler before constructing WALFactory. And then
-    // ReplicationSourceHandler need to use WALFactory get the length of the wal file being written.
-    // So we here we need to construct WALFactory first, and then pass it to the initialized method
-    // of ReplicationSourceHandler.
-    // TODO: I can't follow replication; it has initialize and then later on we start it!
-    WALFactory factory = new WALFactory(conf, listeners, serverName.toString());
+    // Instantiate replication if replication enabled. Pass it the log directories.
+    createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir,
+      factory.getWALProvider());
     this.walFactory = factory;
-    if (this.replicationSourceHandler != null) {
-      this.replicationSourceHandler.initialize(this, walFs, logDir, oldLogDir, factory);
-    }
-    if (this.replicationSinkHandler != null &&
-        this.replicationSinkHandler != this.replicationSourceHandler) {
-      this.replicationSinkHandler.initialize(this, walFs, logDir, oldLogDir, factory);
-    }
   }
 
   /**
@@ -2918,15 +2892,13 @@ public class HRegionServer extends HasThread implements
   //
   // Main program and support routines
   //
-
   /**
    * Load the replication executorService objects, if any
    */
   private static void createNewReplicationInstance(Configuration conf, HRegionServer server,
-      FileSystem walFs, Path walDir, Path oldWALDir) throws IOException {
-
-    if ((server instanceof HMaster) && (!LoadBalancer.isTablesOnMaster(conf) ||
-        LoadBalancer.isSystemTablesOnlyOnMaster(conf))) {
+      FileSystem walFs, Path walDir, Path oldWALDir, WALProvider walProvider) throws IOException {
+    if ((server instanceof HMaster) &&
+      (!LoadBalancer.isTablesOnMaster(conf) || LoadBalancer.isSystemTablesOnlyOnMaster(conf))) {
       return;
     }
 
@@ -2941,32 +2913,30 @@ public class HRegionServer extends HasThread implements
     // If both the sink and the source class names are the same, then instantiate
     // only one object.
     if (sourceClassname.equals(sinkClassname)) {
-      server.replicationSourceHandler =
-          (ReplicationSourceService) newReplicationInstance(sourceClassname, conf, server, walFs,
-            walDir, oldWALDir);
+      server.replicationSourceHandler = newReplicationInstance(sourceClassname,
+        ReplicationSourceService.class, conf, server, walFs, walDir, oldWALDir, walProvider);
       server.replicationSinkHandler = (ReplicationSinkService) server.replicationSourceHandler;
     } else {
-      server.replicationSourceHandler =
-          (ReplicationSourceService) newReplicationInstance(sourceClassname, conf, server, walFs,
-            walDir, oldWALDir);
-      server.replicationSinkHandler = (ReplicationSinkService) newReplicationInstance(sinkClassname,
-        conf, server, walFs, walDir, oldWALDir);
+      server.replicationSourceHandler = newReplicationInstance(sourceClassname,
+        ReplicationSourceService.class, conf, server, walFs, walDir, oldWALDir, walProvider);
+      server.replicationSinkHandler = newReplicationInstance(sinkClassname,
+        ReplicationSinkService.class, conf, server, walFs, walDir, oldWALDir, walProvider);
     }
   }
 
-  private static ReplicationService newReplicationInstance(String classname, Configuration conf,
-      HRegionServer server, FileSystem walFs, Path logDir, Path oldLogDir) throws IOException {
-    Class<? extends ReplicationService> clazz = null;
+  private static <T extends ReplicationService> T newReplicationInstance(String classname,
+      Class<T> xface, Configuration conf, HRegionServer server, FileSystem walFs, Path logDir,
+      Path oldLogDir, WALProvider walProvider) throws IOException {
+    Class<? extends T> clazz = null;
     try {
       ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
-      clazz = Class.forName(classname, true, classLoader).asSubclass(ReplicationService.class);
+      clazz = Class.forName(classname, true, classLoader).asSubclass(xface);
     } catch (java.lang.ClassNotFoundException nfe) {
       throw new IOException("Could not find class for " + classname);
     }
-
-    // create an instance of the replication object, but do not initialize it here as we need to use
-    // WALFactory when initializing.
-    return ReflectionUtils.newInstance(clazz, conf);
+    T service = ReflectionUtils.newInstance(clazz, conf);
+    service.initialize(server, walFs, logDir, oldLogDir, walProvider);
+    return service;
   }
 
   /**
@@ -3739,4 +3709,20 @@ public class HRegionServer extends HasThread implements
       throw ProtobufUtil.getRemoteException(se);
     }
   }
+
+  /**
+   * This method modifies the region server's configuration in order to inject replication-related
+   * features
+   * @param conf region server configurations
+   */
+  static void decorateRegionServerConfiguration(Configuration conf) {
+    if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
+      String plugins = conf.get(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, "");
+      String rsCoprocessorClass = ReplicationObserver.class.getCanonicalName();
+      if (!plugins.contains(rsCoprocessorClass)) {
+        conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+          plugins + "," + rsCoprocessorClass);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
index f3bc188..c34231d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.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
@@ -19,13 +18,12 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Server;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
-import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
+import org.apache.hadoop.hbase.wal.WALProvider;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Gateway to Cluster Replication. Used by
@@ -37,14 +35,14 @@ public interface ReplicationService {
 
   /**
    * Initializes the replication service object.
-   * @throws IOException
+   * @param walProvider can be null if not initialized inside a live region server environment, for
+   *          example, {@code ReplicationSyncUp}.
    */
-  void initialize(Server rs, FileSystem fs, Path logdir, Path oldLogDir,
-      WALFileLengthProvider walFileLengthProvider) throws IOException;
+  void initialize(Server rs, FileSystem fs, Path logdir, Path oldLogDir, WALProvider walProvider)
+      throws IOException;
 
   /**
    * Start replication services.
-   * @throws IOException
    */
   void startReplicationService() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 2aef0a8..23ba773 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.replication.regionserver.PeerProcedureHandler;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 
 /**
  * A source for a replication stream has to expose this service.
@@ -28,12 +27,6 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
  */
 @InterfaceAudience.Private
 public interface ReplicationSourceService extends ReplicationService {
-  /**
-   * Returns a WALObserver for the service. This is needed to
-   * observe log rolls and log archival events.
-   */
-  WALActionsListener getWALActionsListener();
-
 
   /**
    * Returns a Handler to handle peer procedures.

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 2fa5a9b..aaf3beb 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
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -30,14 +28,10 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
@@ -46,12 +40,11 @@ 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.ReplicationTracker;
-import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
-import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
@@ -61,16 +54,15 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
 /**
- * Gateway to Replication.  Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
+ * Gateway to Replication. Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
  */
 @InterfaceAudience.Private
-public class Replication implements
-  ReplicationSourceService, ReplicationSinkService, WALActionsListener {
+public class Replication implements ReplicationSourceService, ReplicationSinkService {
   private static final Logger LOG =
       LoggerFactory.getLogger(Replication.class);
-  private boolean replicationForBulkLoadData;
+  private boolean isReplicationForBulkLoadDataEnabled;
   private ReplicationSourceManager replicationManager;
   private ReplicationQueueStorage queueStorage;
   private ReplicationPeers replicationPeers;
@@ -88,18 +80,6 @@ public class Replication implements
   private PeerProcedureHandler peerProcedureHandler;
 
   /**
-   * Instantiate the replication management (if rep is enabled).
-   * @param server Hosting server
-   * @param fs handle to the filesystem
-   * @param logDir
-   * @param oldLogDir directory where logs are archived
-   * @throws IOException
-   */
-  public Replication(Server server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException {
-    initialize(server, fs, logDir, oldLogDir, p -> OptionalLong.empty());
-  }
-
-  /**
    * Empty constructor
    */
   public Replication() {
@@ -107,16 +87,17 @@ public class Replication implements
 
   @Override
   public void initialize(Server server, FileSystem fs, Path logDir, Path oldLogDir,
-      WALFileLengthProvider walFileLengthProvider) throws IOException {
+      WALProvider walProvider) throws IOException {
     this.server = server;
     this.conf = this.server.getConfiguration();
-    this.replicationForBulkLoadData = isReplicationForBulkLoadDataEnabled(this.conf);
+    this.isReplicationForBulkLoadDataEnabled =
+      ReplicationUtils.isReplicationForBulkLoadDataEnabled(this.conf);
     this.scheduleThreadPool = Executors.newScheduledThreadPool(1,
       new ThreadFactoryBuilder()
         .setNameFormat(server.getServerName().toShortString() + "Replication Statistics #%d")
         .setDaemon(true)
         .build());
-    if (this.replicationForBulkLoadData) {
+    if (this.isReplicationForBulkLoadDataEnabled) {
       if (conf.get(HConstants.REPLICATION_CLUSTER_ID) == null
           || conf.get(HConstants.REPLICATION_CLUSTER_ID).isEmpty()) {
         throw new IllegalArgumentException(HConstants.REPLICATION_CLUSTER_ID
@@ -142,9 +123,28 @@ public class Replication implements
     } catch (KeeperException ke) {
       throw new IOException("Could not read cluster id", ke);
     }
-    this.replicationManager =
-        new ReplicationSourceManager(queueStorage, replicationPeers, replicationTracker, conf,
-            this.server, fs, logDir, oldLogDir, clusterId, walFileLengthProvider);
+    this.replicationManager = new ReplicationSourceManager(queueStorage, replicationPeers,
+        replicationTracker, conf, this.server, fs, logDir, oldLogDir, clusterId,
+        walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty());
+    if (walProvider != null) {
+      walProvider.addWALActionsListener(new WALActionsListener() {
+
+        @Override
+        public void preLogRoll(Path oldPath, Path newPath) throws IOException {
+          replicationManager.preLogRoll(newPath);
+        }
+
+        @Override
+        public void postLogRoll(Path oldPath, Path newPath) throws IOException {
+          replicationManager.postLogRoll(newPath);
+        }
+
+        @Override
+        public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
+          replicationManager.scopeWALEdits(logKey, logEdit);
+        }
+      });
+    }
     this.statsThreadPeriod =
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
     LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
@@ -153,23 +153,6 @@ public class Replication implements
     this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager);
   }
 
-  /**
-   * @param c Configuration to look at
-   * @return True if replication for bulk load data is enabled.
-   */
-  public static boolean isReplicationForBulkLoadDataEnabled(final Configuration c) {
-    return c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
-      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
-  }
-
-   /*
-    * Returns an object to listen to new wal changes
-    **/
-  @Override
-  public WALActionsListener getWALActionsListener() {
-    return this;
-  }
-
   @Override
   public PeerProcedureHandler getPeerProcedureHandler() {
     return peerProcedureHandler;
@@ -225,7 +208,7 @@ public class Replication implements
     this.replicationManager.init();
     this.replicationSink = new ReplicationSink(this.conf, this.server);
     this.scheduleThreadPool.scheduleAtFixedRate(
-      new ReplicationStatisticsThread(this.replicationSink, this.replicationManager),
+      new ReplicationStatisticsTask(this.replicationSink, this.replicationManager),
       statsThreadPeriod, statsThreadPeriod, TimeUnit.SECONDS);
   }
 
@@ -237,45 +220,6 @@ public class Replication implements
     return this.replicationManager;
   }
 
-  @Override
-  public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
-    scopeWALEdits(logKey, logEdit, this.conf, this.getReplicationManager());
-  }
-
-  /**
-   * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
-   * compaction WAL edits and if the scope is local.
-   * @param logKey Key that may get scoped according to its edits
-   * @param logEdit Edits used to lookup the scopes
-   * @param replicationManager Manager used to add bulk load events hfile references
-   * @throws IOException If failed to parse the WALEdit
-   */
-  public static void scopeWALEdits(WALKey logKey,
-      WALEdit logEdit, Configuration conf, ReplicationSourceManager replicationManager)
-          throws IOException {
-    boolean replicationForBulkLoadEnabled = isReplicationForBulkLoadDataEnabled(conf);
-    boolean foundOtherEdits = false;
-    for (Cell cell : logEdit.getCells()) {
-      if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
-        foundOtherEdits = true;
-        break;
-      }
-    }
-
-    if (!foundOtherEdits && logEdit.getCells().size() > 0) {
-      WALProtos.RegionEventDescriptor maybeEvent =
-          WALEdit.getRegionEventDescriptor(logEdit.getCells().get(0));
-      if (maybeEvent != null && (maybeEvent.getEventType() ==
-          WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE)) {
-        // In serially replication, we use scopes when reading close marker.
-        foundOtherEdits = true;
-      }
-    }
-    if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
-      ((WALKeyImpl)logKey).serializeReplicationScope(false);
-    }
-  }
-
   void addHFileRefsToQueue(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
       throws IOException {
     try {
@@ -286,62 +230,16 @@ public class Replication implements
     }
   }
 
-  @Override
-  public void preLogRoll(Path oldPath, Path newPath) throws IOException {
-    getReplicationManager().preLogRoll(newPath);
-  }
-
-  @Override
-  public void postLogRoll(Path oldPath, Path newPath) throws IOException {
-    getReplicationManager().postLogRoll(newPath);
-  }
-
   /**
-   * This method modifies the master's configuration in order to inject replication-related features
-   * @param conf
-   */
-  public static void decorateMasterConfiguration(Configuration conf) {
-    String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
-    String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
-    if (!plugins.contains(cleanerClass)) {
-      conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
-    }
-    if (isReplicationForBulkLoadDataEnabled(conf)) {
-      plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
-      cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();
-      if (!plugins.contains(cleanerClass)) {
-        conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);
-      }
-    }
-  }
-
-  /**
-   * This method modifies the region server's configuration in order to inject replication-related
-   * features
-   * @param conf region server configurations
-   */
-  public static void decorateRegionServerConfiguration(Configuration conf) {
-    if (isReplicationForBulkLoadDataEnabled(conf)) {
-      String plugins = conf.get(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, "");
-      String rsCoprocessorClass = ReplicationObserver.class.getCanonicalName();
-      if (!plugins.contains(rsCoprocessorClass)) {
-        conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
-          plugins + "," + rsCoprocessorClass);
-      }
-    }
-  }
-
-  /*
-   * Statistics thread. Periodically prints the cache statistics to the log.
+   * Statistics task. Periodically prints the cache statistics to the log.
    */
-  static class ReplicationStatisticsThread extends Thread {
+  private final static class ReplicationStatisticsTask implements Runnable {
 
     private final ReplicationSink replicationSink;
     private final ReplicationSourceManager replicationManager;
 
-    public ReplicationStatisticsThread(final ReplicationSink replicationSink,
-                            final ReplicationSourceManager replicationManager) {
-      super("ReplicationStatisticsThread");
+    public ReplicationStatisticsTask(ReplicationSink replicationSink,
+        ReplicationSourceManager replicationManager) {
       this.replicationManager = replicationManager;
       this.replicationSink = replicationSink;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 ab86d7c..2147214 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
@@ -43,6 +43,8 @@ import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
@@ -57,9 +59,13 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -67,6 +73,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
 /**
  * This class is responsible to manage all the replication sources. There are two classes of
  * sources:
@@ -86,14 +94,15 @@ import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFacto
  * operations.</li>
  * <li>Need synchronized on {@link #walsById}. There are four methods which modify it,
  * {@link #addPeer(String)}, {@link #removePeer(String)},
- * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}. {@link #walsById}
- * is a ConcurrentHashMap and there is a Lock for peer id in {@link PeerProcedureHandlerImpl}. So
- * there is no race between {@link #addPeer(String)} and {@link #removePeer(String)}.
- * {@link #cleanOldLogs(SortedSet, String, String)} is called by {@link ReplicationSourceInterface}.
- * So no race with {@link #addPeer(String)}. {@link #removePeer(String)} will terminate the
- * {@link ReplicationSourceInterface} firstly, then remove the wals from {@link #walsById}. So no
- * race with {@link #removePeer(String)}. The only case need synchronized is
- * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}.</li>
+ * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}.
+ * {@link #walsById} is a ConcurrentHashMap and there is a Lock for peer id in
+ * {@link PeerProcedureHandlerImpl}. So there is no race between {@link #addPeer(String)} and
+ * {@link #removePeer(String)}. {@link #cleanOldLogs(SortedSet, String, String)} is called by
+ * {@link ReplicationSourceInterface}. So no race with {@link #addPeer(String)}.
+ * {@link #removePeer(String)} will terminate the {@link ReplicationSourceInterface} firstly, then
+ * remove the wals from {@link #walsById}. So no race with {@link #removePeer(String)}. The only
+ * case need synchronized is {@link #cleanOldLogs(SortedSet, String, String)} and
+ * {@link #preLogRoll(Path)}.</li>
  * <li>No need synchronized on {@link #walsByIdRecoveredQueues}. There are three methods which
  * modify it, {@link #removePeer(String)} , {@link #cleanOldLogs(SortedSet, String, String)} and
  * {@link ReplicationSourceManager.NodeFailoverWorker#run()}.
@@ -533,7 +542,9 @@ public class ReplicationSourceManager implements ReplicationListener {
     walSet.clear();
   }
 
-  void preLogRoll(Path newLog) throws IOException {
+  // public because of we call it in TestReplicationEmptyWALRecovery
+  @VisibleForTesting
+  public void preLogRoll(Path newLog) throws IOException {
     String logName = newLog.getName();
     String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(logName);
     // synchronized on latestPaths to avoid the new open source miss the new log
@@ -588,13 +599,52 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
   }
 
-  void postLogRoll(Path newLog) throws IOException {
+  // public because of we call it in TestReplicationEmptyWALRecovery
+  @VisibleForTesting
+  public void postLogRoll(Path newLog) throws IOException {
     // This only updates the sources we own, not the recovered ones
     for (ReplicationSourceInterface source : this.sources.values()) {
       source.enqueueLog(newLog);
     }
   }
 
+  void scopeWALEdits(WALKey logKey, WALEdit logEdit) throws IOException {
+    scopeWALEdits(logKey, logEdit, this.conf);
+  }
+
+  /**
+   * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
+   * compaction WAL edits and if the scope is local.
+   * @param logKey Key that may get scoped according to its edits
+   * @param logEdit Edits used to lookup the scopes
+   * @throws IOException If failed to parse the WALEdit
+   */
+  @VisibleForTesting
+  static void scopeWALEdits(WALKey logKey, WALEdit logEdit, Configuration conf) throws IOException {
+    boolean replicationForBulkLoadEnabled =
+      ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf);
+    boolean foundOtherEdits = false;
+    for (Cell cell : logEdit.getCells()) {
+      if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+        foundOtherEdits = true;
+        break;
+      }
+    }
+
+    if (!foundOtherEdits && logEdit.getCells().size() > 0) {
+      WALProtos.RegionEventDescriptor maybeEvent =
+        WALEdit.getRegionEventDescriptor(logEdit.getCells().get(0));
+      if (maybeEvent != null &&
+        (maybeEvent.getEventType() == WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE)) {
+        // In serially replication, we use scopes when reading close marker.
+        foundOtherEdits = true;
+      }
+    }
+    if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
+      ((WALKeyImpl) logKey).serializeReplicationScope(false);
+    }
+  }
+
   @Override
   public void regionServerRemoved(String regionserver) {
     transferQueues(ServerName.valueOf(regionserver));
@@ -886,7 +936,6 @@ public class ReplicationSourceManager implements ReplicationListener {
    */
   void waitUntilCanBePushed(byte[] encodedName, long seq, String peerId)
       throws IOException, InterruptedException {
-
     /**
      * There are barriers for this region and position for this peer. N barriers form N intervals,
      * (b1,b2) (b2,b3) ... (bn,max). Generally, there is no logs whose seq id is not greater than
@@ -974,5 +1023,4 @@ public class ReplicationSourceManager implements ReplicationListener {
       Thread.sleep(replicationWaitTime);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 9ec244a..01a230d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -99,7 +99,8 @@ public class ReplicationSyncUp extends Configured implements Tool {
     logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
 
     System.out.println("Start Replication Server start");
-    replication = new Replication(new DummyServer(zkw), fs, logDir, oldLogDir);
+    replication = new Replication();
+    replication.initialize(new DummyServer(zkw), fs, logDir, oldLogDir, null);
     manager = replication.getReplicationManager();
     manager.init().get();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index b4a22e4..361bb51 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -114,8 +114,6 @@ import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
@@ -1482,9 +1480,7 @@ public class HBaseFsck extends Configured implements Closeable {
     // unless I pass along via the conf.
     Configuration confForWAL = new Configuration(c);
     confForWAL.set(HConstants.HBASE_DIR, rootdir.toString());
-    WAL wal =
-      new WALFactory(confForWAL, Collections.<WALActionsListener> singletonList(new MetricsWAL()),
-          walFactoryID).getWAL(metaHRI);
+    WAL wal = new WALFactory(confForWAL, walFactoryID).getWAL(metaHRI);
     HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c, metaDescriptor, wal);
     MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, true);
     return meta;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 d9badfa..231afd5 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
@@ -76,13 +76,13 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   }
 
   protected volatile T wal;
-  protected WALFactory factory = null;
-  protected Configuration conf = null;
-  protected List<WALActionsListener> listeners = null;
-  protected String providerId = null;
+  protected WALFactory factory;
+  protected Configuration conf;
+  protected List<WALActionsListener> listeners = new ArrayList<>();
+  protected String providerId;
   protected AtomicBoolean initialized = new AtomicBoolean(false);
   // for default wal provider, logPrefix won't change
-  protected String logPrefix = null;
+  protected String logPrefix;
 
   /**
    * we synchronized on walCreateLock to prevent wal recreation in different threads
@@ -92,19 +92,16 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   /**
    * @param factory factory that made us, identity used for FS layout. may not be null
    * @param conf may not be null
-   * @param listeners may be null
    * @param providerId differentiate between providers from one factory, used for FS layout. may be
    *          null
    */
   @Override
-  public void init(WALFactory factory, Configuration conf, List<WALActionsListener> listeners,
-      String providerId) throws IOException {
+  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.");
     }
     this.factory = factory;
     this.conf = conf;
-    this.listeners = listeners;
     this.providerId = providerId;
     // get log prefix
     StringBuilder sb = new StringBuilder().append(factory.factoryId);
@@ -249,8 +246,8 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
    * Pattern used to validate a WAL file name see {@link #validateWALFilename(String)} for
    * description.
    */
-  private static final Pattern pattern = Pattern
-      .compile(".*\\.\\d*(" + META_WAL_PROVIDER_ID + ")*");
+  private static final Pattern pattern =
+    Pattern.compile(".*\\.\\d*(" + META_WAL_PROVIDER_ID + ")*");
 
   /**
    * A WAL file name is of the format: &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}
@@ -264,8 +261,8 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   }
 
   /**
-   * Construct the directory name for all WALs on a given server. Dir names currently look like
-   * this for WALs: <code>hbase//WALs/kalashnikov.att.net,61634,1486865297088</code>.
+   * Construct the directory name for all WALs on a given server. Dir names currently look like this
+   * for WALs: <code>hbase//WALs/kalashnikov.att.net,61634,1486865297088</code>.
    * @param serverName Server name formatted as described in {@link ServerName}
    * @return the relative WAL directory name, e.g. <code>.logs/1.example.org,60030,12345</code> if
    *         <code>serverName</code> passed is <code>1.example.org,60030,12345</code>
@@ -278,9 +275,9 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   }
 
   /**
-   * Construct the directory name for all old WALs on a given server. The default old WALs dir
-   * looks like: <code>hbase/oldWALs</code>. If you config hbase.separate.oldlogdir.by.regionserver
-   * to true, it looks like <code>hbase//oldWALs/kalashnikov.att.net,61634,1486865297088</code>.
+   * Construct the directory name for all old WALs on a given server. The default old WALs dir looks
+   * like: <code>hbase/oldWALs</code>. If you config hbase.separate.oldlogdir.by.regionserver to
+   * true, it looks like <code>hbase//oldWALs/kalashnikov.att.net,61634,1486865297088</code>.
    * @param conf
    * @param serverName Server name formatted as described in {@link ServerName}
    * @return the relative WAL directory name
@@ -372,7 +369,7 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
     }
     try {
       serverName = ServerName.parseServerName(logDirName);
-    } catch (IllegalArgumentException|IllegalStateException ex) {
+    } catch (IllegalArgumentException | IllegalStateException ex) {
       serverName = null;
       LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
     }
@@ -430,16 +427,14 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   }
 
   /**
-   * Opens WAL reader with retries and
-   * additional exception handling
+   * Opens WAL reader with retries and additional exception handling
    * @param path path to WAL file
    * @param conf configuration
    * @return WAL Reader instance
    * @throws IOException
    */
-  public static org.apache.hadoop.hbase.wal.WAL.Reader
-    openReader(Path path, Configuration conf)
-        throws IOException
+  public static org.apache.hadoop.hbase.wal.WAL.Reader openReader(Path path, Configuration conf)
+      throws IOException
 
   {
     long retryInterval = 2000; // 2 sec
@@ -503,6 +498,10 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
     }
   }
 
+  @Override
+  public void addWALActionsListener(WALActionsListener listener) {
+    listeners.add(listener);
+  }
 
   /**
    * Get prefix of the log from its name, assuming WAL name in format of

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index 2105490..1e750e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -55,8 +55,7 @@ class DisabledWALProvider implements WALProvider {
   WAL disabled;
 
   @Override
-  public void init(final WALFactory factory, final Configuration conf,
-      final List<WALActionsListener> listeners, String providerId) throws IOException {
+  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
     if (null != disabled) {
       throw new IllegalStateException("WALProvider.init should only be called once.");
     }
@@ -250,4 +249,9 @@ class DisabledWALProvider implements WALProvider {
   public long getLogFileSize() {
     return 0;
   }
+
+  @Override
+  public void addWALActionsListener(WALActionsListener listener) {
+    disabled.registerWALActionsListener(listener);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
index f1662bc..b0a924f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
@@ -19,19 +19,17 @@
 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;
-import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-// imports for things that haven't moved from regionserver.wal yet.
-import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
-import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
-import org.apache.hadoop.hbase.util.CommonFSUtils;
 
 /**
  * A WAL provider that use {@link FSHLog}.

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 a0ef817..28817e9 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
@@ -23,11 +23,11 @@ import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DE
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.Lock;
+import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -130,20 +130,18 @@ public class RegionGroupingProvider implements WALProvider {
 
   private final KeyLocker<String> createLock = new KeyLocker<>();
 
-  private RegionGroupingStrategy strategy = null;
-  private WALFactory factory = null;
-  private List<WALActionsListener> listeners = null;
-  private String providerId = null;
+  private RegionGroupingStrategy strategy;
+  private WALFactory factory;
+  private List<WALActionsListener> listeners = new ArrayList<>();
+  private String providerId;
   private Class<? extends WALProvider> providerClass;
 
   @Override
-  public void init(final WALFactory factory, final Configuration conf,
-      final List<WALActionsListener> listeners, final String providerId) throws IOException {
+  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
     if (null != strategy) {
       throw new IllegalStateException("WALProvider.init should only be called once.");
     }
     this.factory = factory;
-    this.listeners = null == listeners ? null : Collections.unmodifiableList(listeners);
     StringBuilder sb = new StringBuilder().append(factory.factoryId);
     if (providerId != null) {
       if (providerId.startsWith(WAL_FILE_NAME_DELIMITER)) {
@@ -159,19 +157,15 @@ public class RegionGroupingProvider implements WALProvider {
 
   private WALProvider createProvider(String group) throws IOException {
     if (META_WAL_PROVIDER_ID.equals(providerId)) {
-      return factory.createProvider(providerClass, listeners, META_WAL_PROVIDER_ID);
+      return factory.createProvider(providerClass, META_WAL_PROVIDER_ID);
     } else {
-      return factory.createProvider(providerClass, listeners, group);
+      return factory.createProvider(providerClass, group);
     }
   }
 
   @Override
   public List<WAL> getWALs() {
-    List<WAL> wals = new ArrayList<>();
-    for (WALProvider provider : cached.values()) {
-      wals.addAll(provider.getWALs());
-    }
-    return wals;
+    return cached.values().stream().flatMap(p -> p.getWALs().stream()).collect(Collectors.toList());
   }
 
   private WAL getWAL(String group) throws IOException {
@@ -182,6 +176,7 @@ public class RegionGroupingProvider implements WALProvider {
         provider = cached.get(group);
         if (provider == null) {
           provider = createProvider(group);
+          listeners.forEach(provider::addWALActionsListener);
           cached.put(group, provider);
         }
       } finally {
@@ -277,4 +272,14 @@ public class RegionGroupingProvider implements WALProvider {
     }
     return logFileSize;
   }
+
+  @Override
+  public void addWALActionsListener(WALActionsListener listener) {
+    // Notice that there is an assumption that this method must be called before the getWAL above,
+    // so we can make sure there is no sub WALProvider yet, so we only add the listener to our
+    // listeners list without calling addWALActionListener for each WALProvider. Although it is no
+    // hurt to execute an extra loop to call addWALActionListener for each WALProvider, but if the
+    // extra code actually works, then we will have other big problems. So leave it as is.
+    listeners.add(listener);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 d59c824..1410b53 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
@@ -19,19 +19,14 @@ package org.apache.hadoop.hbase.wal;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.util.Collections;
 import java.util.List;
-import java.util.OptionalLong;
 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.client.RegionInfo;
-// imports for things that haven't moved from regionserver.wal yet.
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -62,7 +57,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  * Alternatively, you may provide a custom implementation of {@link WALProvider} by class name.
  */
 @InterfaceAudience.Private
-public class WALFactory implements WALFileLengthProvider {
+public class WALFactory {
 
   private static final Logger LOG = LoggerFactory.getLogger(WALFactory.class);
 
@@ -135,12 +130,12 @@ public class WALFactory implements WALFileLengthProvider {
     }
   }
 
-  WALProvider createProvider(Class<? extends WALProvider> clazz,
-      List<WALActionsListener> listeners, String providerId) throws IOException {
+  WALProvider createProvider(Class<? extends WALProvider> clazz, String providerId)
+      throws IOException {
     LOG.info("Instantiating WALProvider of type " + clazz);
     try {
       final WALProvider result = clazz.getDeclaredConstructor().newInstance();
-      result.init(this, conf, listeners, providerId);
+      result.init(this, conf, providerId);
       return result;
     } catch (Exception e) {
       LOG.error("couldn't set up WALProvider, the configured class is " + clazz);
@@ -150,24 +145,23 @@ public class WALFactory implements WALFileLengthProvider {
   }
 
   /**
-   * instantiate a provider from a config property.
-   * requires conf to have already been set (as well as anything the provider might need to read).
+   * 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(final String key, final String defaultValue,
-      final List<WALActionsListener> listeners, final String providerId) throws IOException {
+  WALProvider getProvider(String key, String defaultValue, String providerId) throws IOException {
     Class<? extends WALProvider> clazz = getProviderClass(key, defaultValue);
-    return createProvider(clazz, listeners, providerId);
+    WALProvider provider = createProvider(clazz, 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 listeners may be null. will be given to all created wals (and not meta-wals)
+   *          instances.
    * @param factoryId a unique identifier for this factory. used i.e. by filesystem implementations
-   *     to make a directory
+   *          to make a directory
    */
-  public WALFactory(final Configuration conf, final List<WALActionsListener> listeners,
-      final String factoryId) throws IOException {
+  public WALFactory(Configuration conf, String factoryId) throws IOException {
     // until we've moved reader/writer construction down into providers, this initialization must
     // happen prior to provider initialization, in case they need to instantiate a reader/writer.
     timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000);
@@ -178,12 +172,12 @@ public class WALFactory implements WALFileLengthProvider {
     this.factoryId = factoryId;
     // end required early initialization
     if (conf.getBoolean("hbase.regionserver.hlog.enabled", true)) {
-      provider = getProvider(WAL_PROVIDER, DEFAULT_WAL_PROVIDER, listeners, null);
+      provider = getProvider(WAL_PROVIDER, DEFAULT_WAL_PROVIDER, null);
     } else {
       // special handling of existing configuration behavior.
       LOG.warn("Running with WAL disabled.");
       provider = new DisabledWALProvider();
-      provider.init(this, conf, null, factoryId);
+      provider.init(this, conf, factoryId);
     }
   }
 
@@ -236,7 +230,6 @@ public class WALFactory implements WALFileLengthProvider {
         return provider;
       }
       provider = getProvider(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER,
-        Collections.<WALActionsListener> singletonList(new MetricsWAL()),
         AbstractFSWALProvider.META_WAL_PROVIDER_ID);
       if (metaProvider.compareAndSet(null, provider)) {
         return provider;
@@ -448,9 +441,4 @@ public class WALFactory implements WALFileLengthProvider {
   public final WALProvider getMetaWALProvider() {
     return this.metaProvider.get();
   }
-
-  @Override
-  public OptionalLong getLogFileSizeIfBeingWritten(Path path) {
-    return getWALs().stream().map(w -> w.getLogFileSizeIfBeingWritten(path)).filter(o -> o.isPresent()).findAny().orElse(OptionalLong.empty());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
index 0586d1d..7ad815e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
@@ -21,34 +21,31 @@ package org.apache.hadoop.hbase.wal;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
+import java.util.OptionalLong;
 import java.util.concurrent.CompletableFuture;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * The Write Ahead Log (WAL) stores all durable edits to the HRegion.
- * This interface provides the entry point for all WAL implementors.
+ * The Write Ahead Log (WAL) stores all durable edits to the HRegion. This interface provides the
+ * entry point for all WAL implementors.
  * <p>
- * See {@link FSHLogProvider} for an example implementation.
- *
- * A single WALProvider will be used for retrieving multiple WALs in a particular region server
- * and must be threadsafe.
+ * See {@link FSHLogProvider} for an example implementation. A single WALProvider will be used for
+ * retrieving multiple WALs in a particular region server and must be threadsafe.
  */
 @InterfaceAudience.Private
 public interface WALProvider {
 
   /**
-   * Set up the provider to create wals.
-   * will only be called once per instance.
+   * Set up the provider to create wals. will only be called once per instance.
    * @param factory factory that made us may not be null
    * @param conf may not be null
-   * @param listeners may be null
    * @param providerId differentiate between providers from one factory. may be null
    */
-  void init(WALFactory factory, Configuration conf, List<WALActionsListener> listeners,
-      String providerId) throws IOException;
+  void init(WALFactory factory, Configuration conf, String providerId) throws IOException;
 
   /**
    * @param region the region which we want to get a WAL for it. Could be null.
@@ -62,16 +59,16 @@ public interface WALProvider {
   List<WAL> getWALs();
 
   /**
-   * persist outstanding WALs to storage and stop accepting new appends.
-   * This method serves as shorthand for sending a sync to every WAL provided by a given
-   * implementation. Those WALs will also stop accepting new writes.
+   * persist outstanding WALs to storage and stop accepting new appends. This method serves as
+   * shorthand for sending a sync to every WAL provided by a given implementation. Those WALs will
+   * also stop accepting new writes.
    */
   void shutdown() throws IOException;
 
   /**
-   * shutdown utstanding WALs and clean up any persisted state.
-   * Call this method only when you will not need to replay any of the edits to the WALs from
-   * this provider. After this call completes, the underlying resources should have been reclaimed.
+   * shutdown utstanding WALs and clean up any persisted state. Call this method only when you will
+   * not need to replay any of the edits to the WALs from this provider. After this call completes,
+   * the underlying resources should have been reclaimed.
    */
   void close() throws IOException;
 
@@ -83,11 +80,13 @@ public interface WALProvider {
   // interface provided by WAL.
   interface Writer extends WriterBase {
     void sync() throws IOException;
+
     void append(WAL.Entry entry) throws IOException;
   }
 
   interface AsyncWriter extends WriterBase {
     CompletableFuture<Long> sync();
+
     void append(WAL.Entry entry);
   }
 
@@ -101,4 +100,17 @@ public interface WALProvider {
    */
   long getLogFileSize();
 
+  /**
+   * Add a {@link WALActionsListener}.
+   * <p>
+   * Notice that you must call this method before calling {@link #getWAL(RegionInfo)} as this method
+   * will not effect the {@link WAL} which has already been created. And as long as we can only it
+   * when initialization, it is not thread safe.
+   */
+  void addWALActionsListener(WALActionsListener listener);
+
+  default WALFileLengthProvider getWALFileLengthProvider() {
+    return path -> getWALs().stream().map(w -> w.getLogFileSizeIfBeingWritten(path))
+        .filter(o -> o.isPresent()).findAny().orElse(OptionalLong.empty());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 09e6935..6007b07 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -112,8 +112,6 @@ import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.VisibilityLabelsCache;
@@ -2326,9 +2324,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     // unless I pass along via the conf.
     Configuration confForWAL = new Configuration(conf);
     confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
-    return (new WALFactory(confForWAL,
-        Collections.<WALActionsListener> singletonList(new MetricsWAL()),
-        "hregion-" + RandomStringUtils.randomNumeric(8))).getWAL(hri);
+    return new WALFactory(confForWAL, "hregion-" + RandomStringUtils.randomNumeric(8)).getWAL(hri);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
index df80fa0..3ee7020 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
@@ -155,7 +155,7 @@ public class TestWALObserver {
     if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseWALRootDir)) {
       TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseWALRootDir, true);
     }
-    this.wals = new WALFactory(conf, null, serverName);
+    this.wals = new WALFactory(conf, serverName);
   }
 
   @After
@@ -353,8 +353,9 @@ public class TestWALObserver {
         Path p = runWALSplit(newConf);
         LOG.info("WALSplit path == " + p);
         // Make a new wal for new region open.
-        final WALFactory wals2 = new WALFactory(conf, null,
-            ServerName.valueOf(currentTest.getMethodName()+"2", 16010, System.currentTimeMillis()).toString());
+        final WALFactory wals2 = new WALFactory(conf,
+            ServerName.valueOf(currentTest.getMethodName() + "2", 16010, System.currentTimeMillis())
+                .toString());
         WAL wal2 = wals2.getWAL(null);
         HRegion region = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir,
             hri, htd, wal2, TEST_UTIL.getHBaseCluster().getRegionServer(0), null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index d30fe9f..2f518c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -45,10 +45,10 @@ import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
-import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -114,7 +114,7 @@ public class TestLogsCleaner {
     conf.setLong("hbase.master.logcleaner.ttl", ttlWAL);
     conf.setLong("hbase.master.procedurewalcleaner.ttl", ttlProcedureWAL);
 
-    Replication.decorateMasterConfiguration(conf);
+    HMaster.decorateMasterConfiguration(conf);
     Server server = new DummyServer();
     ReplicationQueueStorage queueStorage =
         ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/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 d3385e7..08dd428 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -50,7 +51,6 @@ 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.master.ReplicationHFileCleaner;
-import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Pair;
@@ -93,7 +93,7 @@ public class TestReplicationHFileCleaner {
     TEST_UTIL.startMiniZKCluster();
     server = new DummyServer();
     conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
-    Replication.decorateMasterConfiguration(conf);
+    HMaster.decorateMasterConfiguration(conf);
     rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf);
     rp.init();
     rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index 5792106..f26998b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -178,7 +178,7 @@ public class TestCacheOnWriteInSchema {
     fs.delete(logdir, true);
 
     RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
-    walFactory = new WALFactory(conf, null, id);
+    walFactory = new WALFactory(conf, id);
 
     region = TEST_UTIL.createLocalHRegion(info, htd, walFactory.getWAL(info));
     store = new HStore(region, hcd, conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
index aa5365c..225c723 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -175,7 +175,7 @@ public class TestCompactionArchiveConcurrentClose {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
-    final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());
+    final WALFactory wals = new WALFactory(walConf, "log_" + info.getEncodedName());
     HRegion region = new HRegion(fs, wals.getWAL(info), conf, htd, null);
 
     region.initialize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
index b8780af..4c6cf6a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
@@ -196,7 +196,7 @@ public class TestCompactionArchiveIOException {
     HRegionFileSystem fs = new HRegionFileSystem(conf, errFS, tableDir, info);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
-    final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());
+    final WALFactory wals = new WALFactory(walConf, "log_" + info.getEncodedName());
     HRegion region = new HRegion(fs, wals.getWAL(info), conf, htd, null);
 
     region.initialize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index cd8539a..b7e0164 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -946,7 +946,7 @@ public class TestDefaultMemStore {
     EnvironmentEdgeForMemstoreTest edge = new EnvironmentEdgeForMemstoreTest();
     EnvironmentEdgeManager.injectEdge(edge);
     edge.setCurrentTimeMillis(1234);
-    WALFactory wFactory = new WALFactory(conf, null, "1234");
+    WALFactory wFactory = new WALFactory(conf, "1234");
     HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir,
         conf, FSTableDescriptors.createMetaTableDescriptor(conf),
         wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO));

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc6d140a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index e8dbdac..9c5a667 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -165,7 +165,7 @@ public class TestHMobStore {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
-    final WALFactory wals = new WALFactory(walConf, null, methodName);
+    final WALFactory wals = new WALFactory(walConf, methodName);
     region = new HRegion(tableDir, wals.getWAL(info), fs, conf, info, htd, null);
     store = new HMobStore(region, hcd, conf);
     if(testStore) {


[26/41] hbase git commit: HBASE-19913 Split TestStochasticLoadBalancer2

Posted by zh...@apache.org.
HBASE-19913 Split TestStochasticLoadBalancer2


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

Branch: refs/heads/HBASE-19064
Commit: 94dad844af239eafd09dac951bbc9ac2243fac82
Parents: 3622bb0
Author: zhangduo <zh...@apache.org>
Authored: Thu Feb 1 15:56:25 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 2 08:29:28 2018 +0800

----------------------------------------------------------------------
 .../master/balancer/BalancerTestBase2.java      | 40 ++++++++
 .../balancer/TestStochasticLoadBalancer2.java   | 96 --------------------
 ...TestStochasticLoadBalancerRegionReplica.java |  4 +-
 ...oadBalancerRegionReplicaHighReplication.java | 46 ++++++++++
 ...icLoadBalancerRegionReplicaLargeCluster.java | 43 +++++++++
 ...sticLoadBalancerRegionReplicaMidCluster.java | 43 +++++++++
 ...onReplicaReplicationGreaterThanNumNodes.java | 46 ++++++++++
 7 files changed, 220 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/94dad844/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase2.java
new file mode 100644
index 0000000..dc952da
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase2.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.balancer;
+
+import org.junit.After;
+import org.junit.Before;
+
+public class BalancerTestBase2 extends BalancerTestBase {
+
+  @Before
+  public void before() {
+    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
+    conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
+    conf.setFloat("hbase.master.balancer.stochastic.minCostNeedBalance", 0.05f);
+    loadBalancer.setConf(conf);
+  }
+
+  @After
+  public void after() {
+    // reset config to make sure balancer run
+    loadBalancer.setConf(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/94dad844/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java
deleted file mode 100644
index da573da..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.balancer;
-
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.testclassification.FlakeyTests;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({FlakeyTests.class, LargeTests.class})
-public class TestStochasticLoadBalancer2 extends BalancerTestBase {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestStochasticLoadBalancer2.class);
-
-  @Before
-  public void before() {
-    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
-    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
-    conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
-    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
-    conf.setFloat("hbase.master.balancer.stochastic.minCostNeedBalance", 0.05f);
-    loadBalancer.setConf(conf);
-  }
-
-  @After
-  public void after() {
-    // reset config to make sure balancer run
-    loadBalancer.setConf(conf);
-  }
-
-  @Test
-  public void testRegionReplicasOnMidCluster() {
-    int numNodes = 200;
-    int numRegions = 40 * 200;
-    int replication = 3; // 3 replicas per region
-    int numRegionsPerServer = 30; //all regions are mostly balanced
-    int numTables = 10;
-    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
-  }
-
-  @Test
-  public void testRegionReplicasOnLargeCluster() {
-    int numNodes = 1000;
-    int numRegions = 20 * numNodes; // 20 * replication regions per RS
-    int numRegionsPerServer = 19; // all servers except one
-    int numTables = 100;
-    int replication = 3;
-    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
-  }
-
-  @Test
-  public void testRegionReplicasOnMidClusterHighReplication() {
-    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 4000000L);
-    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
-    loadBalancer.setConf(conf);
-    int numNodes = 80;
-    int numRegions = 6 * numNodes;
-    int replication = 80; // 80 replicas per region, one for each server
-    int numRegionsPerServer = 5;
-    int numTables = 10;
-    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, true);
-  }
-
-  @Test
-  public void testRegionReplicationOnMidClusterReplicationGreaterThanNumNodes() {
-    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
-    loadBalancer.setConf(conf);
-    int numNodes = 40;
-    int numRegions = 6 * 50;
-    int replication = 50; // 50 replicas per region, more than numNodes
-    int numRegionsPerServer = 6;
-    int numTables = 10;
-    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, false);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/94dad844/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java
index cb48e1d..5b06f86 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java
@@ -35,13 +35,13 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.master.RackManager;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-@Category({ MasterTests.class, LargeTests.class })
+@Category({ MasterTests.class, MediumTests.class })
 public class TestStochasticLoadBalancerRegionReplica extends BalancerTestBase {
 
   @ClassRule

http://git-wip-us.apache.org/repos/asf/hbase/blob/94dad844/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java
new file mode 100644
index 0000000..1253304
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java
@@ -0,0 +1,46 @@
+/**
+ * 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.balancer;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestStochasticLoadBalancerRegionReplicaHighReplication extends BalancerTestBase2 {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestStochasticLoadBalancerRegionReplicaHighReplication.class);
+
+  @Test
+  public void testRegionReplicasOnMidClusterHighReplication() {
+    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 4000000L);
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
+    loadBalancer.setConf(conf);
+    int numNodes = 80;
+    int numRegions = 6 * numNodes;
+    int replication = 80; // 80 replicas per region, one for each server
+    int numRegionsPerServer = 5;
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/94dad844/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaLargeCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaLargeCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaLargeCluster.java
new file mode 100644
index 0000000..f43af33
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaLargeCluster.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.master.balancer;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestStochasticLoadBalancerRegionReplicaLargeCluster extends BalancerTestBase2 {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestStochasticLoadBalancerRegionReplicaLargeCluster.class);
+
+  @Test
+  public void testRegionReplicasOnLargeCluster() {
+    int numNodes = 1000;
+    int numRegions = 20 * numNodes; // 20 * replication regions per RS
+    int numRegionsPerServer = 19; // all servers except one
+    int numTables = 100;
+    int replication = 3;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/94dad844/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaMidCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaMidCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaMidCluster.java
new file mode 100644
index 0000000..99c46a5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaMidCluster.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.master.balancer;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestStochasticLoadBalancerRegionReplicaMidCluster extends BalancerTestBase2 {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestStochasticLoadBalancerRegionReplicaMidCluster.class);
+
+  @Test
+  public void testRegionReplicasOnMidCluster() {
+    int numNodes = 200;
+    int numRegions = 40 * 200;
+    int replication = 3; // 3 replicas per region
+    int numRegionsPerServer = 30; // all regions are mostly balanced
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/94dad844/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.java
new file mode 100644
index 0000000..fd0cc98
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.java
@@ -0,0 +1,46 @@
+/**
+ * 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.balancer;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes
+    extends BalancerTestBase2 {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule
+      .forClass(TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.class);
+
+  @Test
+  public void testRegionReplicationOnMidClusterReplicationGreaterThanNumNodes() {
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
+    loadBalancer.setConf(conf);
+    int numNodes = 40;
+    int numRegions = 6 * 50;
+    int replication = 50; // 50 replicas per region, more than numNodes
+    int numRegionsPerServer = 6;
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, false);
+  }
+}


[28/41] hbase git commit: Revert "HBASE-19901 HBASE-19901 Up yetus proclimit on nightlies"

Posted by zh...@apache.org.
Revert "HBASE-19901 HBASE-19901 Up yetus proclimit on nightlies"

This reverts commit 2e6bc12441f50a4de9ee87e96132cb067556a528.


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

Branch: refs/heads/HBASE-19064
Commit: 18eec8c1a58574ae3ff529904f677288ff696765
Parents: 5f5ddf5
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 1 18:04:23 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 18:04:23 2018 -0800

----------------------------------------------------------------------
 dev-support/hbase-personality.sh   | 8 --------
 dev-support/hbase_nightly_yetus.sh | 1 -
 2 files changed, 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/18eec8c1/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index ddd0df4..e24930d 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -64,14 +64,6 @@ function personality_globals
 
   # Override the maven options
   MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
-
-  # Yetus 0.7.0 enforces limits. Default proclimit is 1000.
-  # Up it. See HBASE-19902 for how we arrived at this number. 
-  PROCLIMIT=10000
-
-  # Set docker container to run with 20g. Default is 4g in yetus.
-  # See HBASE-19902 for how we arrived at 20g.
-  # UNUSED AT MOMENT DOCKERMEMLIMIT=20g
 }
 
 ## @description  Parse extra arguments required by personalities, if any.

http://git-wip-us.apache.org/repos/asf/hbase/blob/18eec8c1/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index 3e6fc7c..4e67354 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -67,7 +67,6 @@ YETUS_ARGS=("--whitespace-tabs-ignore-list=${WHITESPACE_IGNORE_LIST}" "${YETUS_A
 YETUS_ARGS=("--sentinel" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--proclimit=${PROCLIMIT}" "${YETUS_ARGS[@]}")
 
 # Currently, flaky list is calculated only for master branch.
 UNDERSCORED_BRANCH_NAME=$(echo ${BRANCH_NAME} | tr '.-' '_')


[22/41] hbase git commit: HBASE-19884 BucketEntryGroup's equals, hashCode and compareTo methods are not consistent

Posted by zh...@apache.org.
HBASE-19884 BucketEntryGroup's equals, hashCode and compareTo methods are not consistent

Move back to default equals and hashCode.
Remove compareTo and Comparator to PriorityQueue.

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

Branch: refs/heads/HBASE-19064
Commit: d47242247552bbad2df578c89501a6c6789c89c8
Parents: 38c8144
Author: Peter Somogyi <ps...@cloudera.com>
Authored: Tue Jan 30 10:14:08 2018 +0100
Committer: Michael Stack <st...@apache.org>
Committed: Thu Feb 1 13:21:58 2018 -0800

----------------------------------------------------------------------
 .../hbase/io/hfile/bucket/BucketCache.java      | 29 ++------------------
 1 file changed, 3 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d4724224/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index bd2b9c8..e9129d2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -36,7 +36,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
-import java.util.Objects;
 import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
@@ -821,7 +820,8 @@ public class BucketCache implements BlockCache, HeapSize {
         }
       }
 
-      PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<>(3);
+      PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<>(3,
+          Comparator.comparingLong(BucketEntryGroup::overflow));
 
       bucketQueue.add(bucketSingle);
       bucketQueue.add(bucketMulti);
@@ -1350,7 +1350,7 @@ public class BucketCache implements BlockCache, HeapSize {
    * the eviction algorithm takes the appropriate number of elements out of each
    * according to configuration parameters and their relative sizes.
    */
-  private class BucketEntryGroup implements Comparable<BucketEntryGroup> {
+  private class BucketEntryGroup {
 
     private CachedEntryQueue queue;
     private long totalSize = 0;
@@ -1390,29 +1390,6 @@ public class BucketCache implements BlockCache, HeapSize {
     public long totalSize() {
       return totalSize;
     }
-
-    @Override
-    public int compareTo(BucketEntryGroup that) {
-      return Long.compare(this.overflow(), that.overflow());
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      BucketEntryGroup that = (BucketEntryGroup) o;
-      return totalSize == that.totalSize && bucketSize == that.bucketSize
-          && Objects.equals(queue, that.queue);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(queue, totalSize, bucketSize);
-    }
   }
 
   /**