You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2019/01/08 18:56:04 UTC

hbase git commit: HBASE-21688: Address WAL filesystem issues

Repository: hbase
Updated Branches:
  refs/heads/master 7d6ce3569 -> d26acbe1d


HBASE-21688: Address WAL filesystem issues

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


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

Branch: refs/heads/master
Commit: d26acbe1dbc0852fd580c926ffe2d53353c582f1
Parents: 7d6ce35
Author: Vladimir Rodionov <vr...@hortonworks.com>
Authored: Mon Jan 7 19:47:17 2019 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jan 8 13:56:11 2019 -0500

----------------------------------------------------------------------
 .../backup/impl/IncrementalBackupManager.java   | 11 ++++----
 .../backup/master/TestBackupLogCleaner.java     |  6 ++---
 .../test/IntegrationTestBigLinkedList.java      |  8 +++---
 .../test/IntegrationTestLoadAndVerify.java      |  7 ++---
 .../org/apache/hadoop/hbase/io/WALLink.java     |  4 +--
 .../hadoop/hbase/master/MasterWalManager.java   | 10 ++++---
 .../ReplicationSourceWALReader.java             |  2 +-
 .../regionserver/WALEntryStream.java            | 14 +++++-----
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  6 ++---
 .../hbase/fs/TestBlockReorderMultiBlocks.java   |  2 +-
 .../hadoop/hbase/master/AbstractTestDLS.java    |  4 +--
 .../regionserver/TestWALEntryStream.java        | 28 ++++++++++----------
 12 files changed, 55 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
index 853f458..93d264a 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.backup.util.BackupUtils;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -243,10 +243,11 @@ public class IncrementalBackupManager extends BackupManager {
       throws IOException {
     LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps
         + "\n newestTimestamps: " + newestTimestamps);
-    Path rootdir = FSUtils.getRootDir(conf);
-    Path logDir = new Path(rootdir, HConstants.HREGION_LOGDIR_NAME);
-    Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
-    FileSystem fs = rootdir.getFileSystem(conf);
+
+    Path walRootDir = CommonFSUtils.getWALRootDir(conf);
+    Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    FileSystem fs = walRootDir.getFileSystem(conf);
     NewestLogFilter pathFilter = new NewestLogFilter();
 
     List<String> resultLogFiles = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
index b37c7a9..9273487 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -151,8 +151,8 @@ public class TestBackupLogCleaner extends TestBackupBase {
   }
 
   private List<FileStatus> getListOfWALFiles(Configuration c) throws IOException {
-    Path logRoot = new Path(FSUtils.getRootDir(c), HConstants.HREGION_LOGDIR_NAME);
-    FileSystem fs = FileSystem.get(c);
+    Path logRoot = new Path(CommonFSUtils.getWALRootDir(c), HConstants.HREGION_LOGDIR_NAME);
+    FileSystem fs = logRoot.getFileSystem(c);
     RemoteIterator<LocatedFileStatus> it = fs.listFiles(logRoot, true);
     List<FileStatus> logFiles = new ArrayList<FileStatus>();
     while (it.hasNext()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index b965bca..1e00450 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.regionserver.FlushPolicyFactory;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Random64;
 import org.apache.hadoop.hbase.util.RegionSplitter;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -981,10 +982,11 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
       if (keys.isEmpty()) throw new RuntimeException("No keys to find");
       LOG.info("Count of keys to find: " + keys.size());
       for(byte [] key: keys)  LOG.info("Key: " + Bytes.toStringBinary(key));
-      Path hbaseDir = new Path(getConf().get(HConstants.HBASE_DIR));
       // Now read all WALs. In two dirs. Presumes certain layout.
-      Path walsDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
-      Path oldWalsDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
+      Path walsDir = new Path(
+          CommonFSUtils.getWALRootDir(getConf()), HConstants.HREGION_LOGDIR_NAME);
+      Path oldWalsDir = new Path(
+          CommonFSUtils.getWALRootDir(getConf()), HConstants.HREGION_OLDLOGDIR_NAME);
       LOG.info("Running Search with keys inputDir=" + inputDir +", numMappers=" + numMappers +
         " against " + getConf().get(HConstants.HBASE_DIR));
       int ret = ToolRunner.run(getConf(), new WALSearcher(getConf()),

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
index ffdfa86..ffad7d3 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.mapreduce.TableMapper;
 import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
@@ -506,10 +507,10 @@ public void cleanUpCluster() throws Exception {
     if (keys.isEmpty()) throw new RuntimeException("No keys to find");
     LOG.info("Count of keys to find: " + keys.size());
     for(byte [] key: keys)  LOG.info("Key: " + Bytes.toStringBinary(key));
-    Path hbaseDir = new Path(getConf().get(HConstants.HBASE_DIR));
     // Now read all WALs. In two dirs. Presumes certain layout.
-    Path walsDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
-    Path oldWalsDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    Path walsDir = new Path(CommonFSUtils.getWALRootDir(getConf()), HConstants.HREGION_LOGDIR_NAME);
+    Path oldWalsDir = new Path(
+        CommonFSUtils.getWALRootDir(getConf()), HConstants.HREGION_OLDLOGDIR_NAME);
     LOG.info("Running Search with keys inputDir=" + inputDir +
       " against " + getConf().get(HConstants.HBASE_DIR));
     int ret = ToolRunner.run(new WALSearcher(getConf()), new String [] {walsDir.toString(), ""});

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java
index aac2a87..c495201 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/WALLink.java
@@ -24,7 +24,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 
 /**
  * WALLink describes a link to a WAL.
@@ -45,7 +45,7 @@ public class WALLink extends FileLink {
    */
   public WALLink(final Configuration conf,
       final String serverName, final String logName) throws IOException {
-    this(FSUtils.getWALRootDir(conf), serverName, logName);
+    this(CommonFSUtils.getWALRootDir(conf), serverName, logName);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
index fbf4594..8fdcb17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
@@ -187,8 +188,9 @@ public class MasterWalManager {
    * @return List of all RegionServer WAL dirs; i.e. this.rootDir/HConstants.HREGION_LOGDIR_NAME.
    */
   public FileStatus[] getWALDirPaths(final PathFilter filter) throws IOException {
-    Path walDirPath = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
-    FileStatus[] walDirForServerNames = FSUtils.listStatus(fs, walDirPath, filter);
+    Path walDirPath = new Path(CommonFSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME);
+    FileStatus[] walDirForServerNames = FSUtils.listStatus(
+        CommonFSUtils.getWALFileSystem(conf), walDirPath, filter);
     return walDirForServerNames == null? new FileStatus[0]: walDirForServerNames;
   }
 
@@ -201,12 +203,12 @@ public class MasterWalManager {
    *             it.
    */
   @Deprecated
-  public Set<ServerName> getFailedServersFromLogFolders() {
+  public Set<ServerName> getFailedServersFromLogFolders() throws IOException {
     boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors",
         WALSplitter.SPLIT_SKIP_ERRORS_DEFAULT);
 
     Set<ServerName> serverNames = new HashSet<>();
-    Path logsDirPath = new Path(this.rootDir, HConstants.HREGION_LOGDIR_NAME);
+    Path logsDirPath = new Path(CommonFSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME);
 
     do {
       if (services.isStopped()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index b3bdb02..d9f1785 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -124,7 +124,7 @@ class ReplicationSourceWALReader extends Thread {
     int sleepMultiplier = 1;
     while (isReaderRunning()) { // we only loop back here if something fatal happened to our stream
       try (WALEntryStream entryStream =
-          new WALEntryStream(logQueue, fs, conf, currentPosition,
+          new WALEntryStream(logQueue, conf, currentPosition,
               source.getWALFileLengthProvider(), source.getServerWALsBelongTo(),
               source.getSourceMetrics())) {
         while (isReaderRunning()) { // loop here to keep reusing stream while we can

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
index 0393af4..2201932 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -80,11 +81,11 @@ class WALEntryStream implements Closeable {
    * @param metrics replication metrics
    * @throws IOException
    */
-  public WALEntryStream(PriorityBlockingQueue<Path> logQueue, FileSystem fs, Configuration conf,
+  public WALEntryStream(PriorityBlockingQueue<Path> logQueue, Configuration conf,
       long startPosition, WALFileLengthProvider walFileLengthProvider, ServerName serverName,
       MetricsSource metrics) throws IOException {
     this.logQueue = logQueue;
-    this.fs = fs;
+    this.fs = CommonFSUtils.getWALFileSystem(conf);
     this.conf = conf;
     this.currentPositionOfEntry = startPosition;
     this.walFileLengthProvider = walFileLengthProvider;
@@ -312,10 +313,10 @@ class WALEntryStream implements Closeable {
   }
 
   private Path getArchivedLog(Path path) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path walRootDir = CommonFSUtils.getWALRootDir(conf);
 
     // Try found the log in old dir
-    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
     Path archivedLogLocation = new Path(oldLogDir, path.getName());
     if (fs.exists(archivedLogLocation)) {
       LOG.info("Log " + path + " was moved to " + archivedLogLocation);
@@ -324,7 +325,7 @@ class WALEntryStream implements Closeable {
 
     // Try found the log in the seperate old log dir
     oldLogDir =
-        new Path(rootDir, new StringBuilder(HConstants.HREGION_OLDLOGDIR_NAME)
+        new Path(walRootDir, new StringBuilder(HConstants.HREGION_OLDLOGDIR_NAME)
             .append(Path.SEPARATOR).append(serverName.getServerName()).toString());
     archivedLogLocation = new Path(oldLogDir, path.getName());
     if (fs.exists(archivedLogLocation)) {
@@ -381,7 +382,8 @@ class WALEntryStream implements Closeable {
   // For HBASE-15019
   private void recoverLease(final Configuration conf, final Path path) {
     try {
-      final FileSystem dfs = FSUtils.getCurrentFileSystem(conf);
+
+      final FileSystem dfs = CommonFSUtils.getWALFileSystem(conf);
       FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
       fsUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
         @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/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 8176942..9e5f9e8 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
@@ -1680,9 +1680,9 @@ public class HBaseFsck extends Configured implements Closeable {
    *          Meta recovery WAL directory inside WAL directory path.
    */
   private void removeHBCKMetaRecoveryWALDir(String walFactoryId) throws IOException {
-    Path rootdir = FSUtils.getRootDir(getConf());
-    Path walLogDir = new Path(new Path(rootdir, HConstants.HREGION_LOGDIR_NAME), walFactoryId);
-    FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
+    Path walLogDir = new Path(new Path(CommonFSUtils.getWALRootDir(getConf()),
+          HConstants.HREGION_LOGDIR_NAME), walFactoryId);
+    FileSystem fs = CommonFSUtils.getWALFileSystem(getConf());
     FileStatus[] walFiles = FSUtils.listStatus(fs, walLogDir, null);
     if (walFiles == null || walFiles.length == 0) {
       LOG.info("HBCK meta recovery WAL directory is empty, removing it now.");

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
index ad2b2d4..df760f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
@@ -141,7 +141,7 @@ public class TestBlockReorderMultiBlocks {
 
     // Now we need to find the log file, its locations, and look at it
 
-    String rootDir = new Path(FSUtils.getRootDir(conf) + "/" + HConstants.HREGION_LOGDIR_NAME +
+    String rootDir = new Path(FSUtils.getWALRootDir(conf) + "/" + HConstants.HREGION_LOGDIR_NAME +
             "/" + targetRs.getServerName().toString()).toUri().getPath();
 
     DistributedFileSystem mdfs = (DistributedFileSystem)

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
index d72e756..350534c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
@@ -420,8 +420,8 @@ public abstract class AbstractTestDLS {
     startCluster(1);
     final SplitLogManager slm = master.getMasterWalManager().getSplitLogManager();
     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
-    final Path logDir = new Path(new Path(FSUtils.getRootDir(conf), HConstants.HREGION_LOGDIR_NAME),
-        ServerName.valueOf("x", 1, 1).toString());
+    final Path rootLogDir = new Path(FSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME);
+    final Path logDir = new Path(rootLogDir, ServerName.valueOf("x", 1, 1).toString());
     fs.mkdirs(logDir);
     ExecutorService executor = null;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d26acbe1/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 fac6f74..f9f70cd 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
@@ -156,7 +156,7 @@ public class TestWALEntryStream {
           log.rollWriter();
 
           try (WALEntryStream entryStream =
-              new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
+              new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) {
             int i = 0;
             while (entryStream.hasNext()) {
               assertNotNull(entryStream.next());
@@ -183,7 +183,7 @@ public class TestWALEntryStream {
     appendToLogAndSync();
     long oldPos;
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) {
       // There's one edit in the log, read it. Reading past it needs to throw exception
       assertTrue(entryStream.hasNext());
       WAL.Entry entry = entryStream.peek();
@@ -197,7 +197,7 @@ public class TestWALEntryStream {
 
     appendToLogAndSync();
 
-    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, CONF, oldPos,
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, CONF, oldPos,
         log, null, new MetricsSource("1"))) {
       // Read the newly added entry, make sure we made progress
       WAL.Entry entry = entryStream.next();
@@ -211,7 +211,7 @@ public class TestWALEntryStream {
     log.rollWriter();
     appendToLogAndSync();
 
-    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, CONF, oldPos,
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, CONF, oldPos,
         log, null, new MetricsSource("1"))) {
       WAL.Entry entry = entryStream.next();
       assertNotEquals(oldPos, entryStream.getPosition());
@@ -237,7 +237,7 @@ public class TestWALEntryStream {
     appendToLog("1");
     appendToLog("2");// 2
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) {
       assertEquals("1", getRow(entryStream.next()));
 
       appendToLog("3"); // 3 - comes in after reader opened
@@ -262,7 +262,7 @@ public class TestWALEntryStream {
   public void testNewEntriesWhileStreaming() throws Exception {
     appendToLog("1");
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) {
       entryStream.next(); // we've hit the end of the stream at this point
 
       // some new entries come in while we're streaming
@@ -285,7 +285,7 @@ public class TestWALEntryStream {
     long lastPosition = 0;
     appendToLog("1");
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) {
       entryStream.next(); // we've hit the end of the stream at this point
       appendToLog("2");
       appendToLog("3");
@@ -293,7 +293,7 @@ public class TestWALEntryStream {
     }
     // next stream should picks up where we left off
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, CONF, lastPosition, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, CONF, lastPosition, log, null, new MetricsSource("1"))) {
       assertEquals("2", getRow(entryStream.next()));
       assertEquals("3", getRow(entryStream.next()));
       assertFalse(entryStream.hasNext()); // done
@@ -310,14 +310,14 @@ public class TestWALEntryStream {
     long lastPosition = 0;
     appendEntriesToLogAndSync(3);
     // read only one element
-    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, CONF, lastPosition,
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, CONF, lastPosition,
         log, null, new MetricsSource("1"))) {
       entryStream.next();
       lastPosition = entryStream.getPosition();
     }
     // there should still be two more entries from where we left off
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, CONF, lastPosition, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, CONF, lastPosition, log, null, new MetricsSource("1"))) {
       assertNotNull(entryStream.next());
       assertNotNull(entryStream.next());
       assertFalse(entryStream.hasNext());
@@ -328,7 +328,7 @@ public class TestWALEntryStream {
   @Test
   public void testEmptyStream() throws Exception {
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) {
       assertFalse(entryStream.hasNext());
     }
   }
@@ -361,7 +361,7 @@ public class TestWALEntryStream {
     // get ending position
     long position;
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) {
       entryStream.next();
       entryStream.next();
       entryStream.next();
@@ -476,7 +476,7 @@ public class TestWALEntryStream {
     // get ending position
     long position;
     try (WALEntryStream entryStream =
-      new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
+      new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) {
       entryStream.next();
       entryStream.next();
       entryStream.next();
@@ -592,7 +592,7 @@ public class TestWALEntryStream {
     appendToLog("2");
     long size = log.getLogFileSizeIfBeingWritten(walQueue.peek()).getAsLong();
     AtomicLong fileLength = new AtomicLong(size - 1);
-    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, CONF, 0,
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue,  CONF, 0,
         p -> OptionalLong.of(fileLength.get()), null, new MetricsSource("1"))) {
       assertTrue(entryStream.hasNext());
       assertNotNull(entryStream.next());