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:40 UTC

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

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());