You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2016/09/29 16:08:27 UTC

[06/18] hbase git commit: HBASE-14439 break everything HRegion

HBASE-14439 break everything HRegion


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

Branch: refs/heads/hbase-14439
Commit: c347b84cd4e832402b5f4cc1c332f3d8f560bdd2
Parents: 7546190
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Tue Oct 20 16:22:07 2015 -0700
Committer: Sean Busbey <bu...@apache.org>
Committed: Thu Sep 29 11:07:29 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/fs/RegionFileSystem.java       |  132 ++-
 .../hadoop/hbase/fs/legacy/LegacyLayout.java    |    2 +-
 .../hbase/fs/legacy/LegacyMasterFileSystem.java |    2 +-
 .../hbase/fs/legacy/LegacyRegionFileSystem.java |  101 +-
 .../hadoop/hbase/master/CatalogJanitor.java     |   18 +-
 .../hbase/regionserver/CompactionTool.java      |   18 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  113 +-
 .../hbase/regionserver/HRegionFileSystem.java   | 1109 ------------------
 .../hadoop/hbase/regionserver/HStore.java       |    5 +-
 .../regionserver/RegionMergeTransaction.java    |    6 +-
 .../RegionMergeTransactionImpl.java             |   19 +-
 .../regionserver/SplitTransactionImpl.java      |    3 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java   |    8 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |    6 +-
 .../hbase/snapshot/SnapshotManifestV1.java      |   24 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   27 +-
 .../hadoop/hbase/util/HBaseFsckRepair.java      |    2 +-
 .../hadoop/hbase/util/ModifyRegionUtils.java    |    2 +-
 .../hadoop/hbase/util/RegionSplitter.java       |   12 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |    6 +-
 .../org/apache/hadoop/hbase/TestIOFencing.java  |   25 +-
 .../client/TestRestoreSnapshotFromClient.java   |   16 +-
 .../TestRegionObserverScannerOpenHook.java      |   11 +-
 .../TableSnapshotInputFormatTestBase.java       |   38 +-
 .../MasterProcedureTestingUtility.java          |   19 +-
 .../hbase/regionserver/TestAtomicOperation.java |   27 +-
 .../hadoop/hbase/regionserver/TestBulkLoad.java |    8 +-
 .../TestDefaultCompactSelection.java            |  163 ++-
 .../hbase/regionserver/TestDefaultMemStore.java |    2 +-
 .../hbase/regionserver/TestHMobStore.java       |    7 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |   23 +-
 .../regionserver/TestHRegionFileSystem.java     |   30 +-
 .../hbase/regionserver/TestHRegionInfo.java     |   10 +-
 .../regionserver/TestHRegionReplayEvents.java   |    5 +-
 .../hbase/regionserver/TestRecoveredEdits.java  |    6 +-
 .../TestRegionMergeTransactionOnCluster.java    |   18 +-
 .../TestScannerHeartbeatMessages.java           |   12 +-
 .../TestSplitTransactionOnCluster.java          |   16 +-
 .../hadoop/hbase/regionserver/TestStore.java    |    9 +-
 .../hbase/regionserver/TestStoreFile.java       |   24 +-
 .../TestStoreFileRefresherChore.java            |   12 +-
 .../hbase/regionserver/wal/TestDurability.java  |    4 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |    8 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    |    6 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java   |   12 +-
 .../hbase/wal/WALPerformanceEvaluation.java     |    8 +-
 46 files changed, 643 insertions(+), 1491 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java
index 3b94355..154d648 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.TableDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.fs.legacy.LegacyRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -46,6 +46,61 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.fs.FSUtilsWithRetries;
+import org.apache.hadoop.hbase.fs.FsContext;
+import org.apache.hadoop.hbase.fs.RegionFileSystem;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.MetaUtils;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobUtils;
+
+import org.apache.hadoop.hbase.Cell;
+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.KeyValueUtil;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.regionserver.*;
+import org.apache.hadoop.hbase.io.Reference;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSHDFSUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+
 @InterfaceAudience.Private
 public abstract class RegionFileSystem {
   private static Log LOG = LogFactory.getLog(RegionFileSystem.class);
@@ -135,6 +190,78 @@ public abstract class RegionFileSystem {
   protected abstract void destroy() throws IOException;
 
   // ==========================================================================
+  //  NOOOOO
+  // ==========================================================================
+  public abstract void checkRegionInfoOnFilesystem() throws IOException;
+  public abstract Path getRegionDir();
+  public abstract Path getTableDir();
+
+  public abstract Path getTempDir();
+
+  public HRegionInfo getRegionInfoForFS() { return hri; }
+
+  public abstract Path getStoreDir(final String familyName);
+  public abstract Path createTempName();
+  public abstract Path createStoreDir(final String familyName) throws IOException;
+  public abstract Path bulkLoadStoreFile(final String familyName, Path srcPath, long seqNum)
+      throws IOException;
+
+  public abstract void cleanupTempDir() throws IOException;
+  public abstract void cleanupSplitsDir() throws IOException;
+  public abstract void cleanupMergesDir() throws IOException;
+  public abstract void cleanupAnySplitDetritus() throws IOException;
+
+  public abstract Path commitDaughterRegion(final HRegionInfo regionInfo)
+      throws IOException;
+  public abstract void commitMergedRegion(final HRegionInfo mergedRegionInfo) throws IOException;
+  public abstract StoreFileInfo getStoreFileInfo(final String familyName, final String fileName)
+      throws IOException;
+
+  public abstract Path commitStoreFile(final String familyName, final Path buildPath) throws IOException;
+  public abstract void commitStoreFiles(final Map<byte[], List<StoreFile>> storeFiles) throws IOException;
+
+  public abstract void removeStoreFile(final String familyName, final Path filePath)
+      throws IOException;
+  public abstract void removeStoreFiles(final String familyName, final Collection<StoreFile> storeFiles)
+      throws IOException;
+
+  public abstract boolean hasReferences(final String familyName) throws IOException;
+  public abstract boolean hasReferences(final HTableDescriptor htd) throws IOException;
+
+  public abstract Path getStoreFilePath(final String familyName, final String fileName);
+
+  public abstract void logFileSystemState(final Log LOG) throws IOException;
+
+  public abstract void createSplitsDir() throws IOException;
+  public abstract Path getSplitsDir();
+  public abstract Path getSplitsDir(final HRegionInfo hri);
+
+  public abstract Path getMergesDir();
+  public abstract void createMergesDir() throws IOException;
+
+  public abstract Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName,
+      final StoreFile f, final Path mergedDir)
+      throws IOException;
+
+  public abstract void cleanupMergedRegion(final HRegionInfo mergedRegion) throws IOException;
+
+  public abstract Path splitStoreFile(final HRegionInfo hri, final String familyName,
+      final StoreFile f, final byte[] splitRow, final boolean top, RegionSplitPolicy splitPolicy)
+          throws IOException;
+
+  public abstract void cleanupDaughterRegion(final HRegionInfo regionInfo) throws IOException;
+
+  public static HRegionInfo loadRegionInfoFileContent(FileSystem fs, Path regionDir)
+      throws IOException {
+    FSDataInputStream in = fs.open(new Path(regionDir, ".regioninfo"));
+    try {
+      return HRegionInfo.parseFrom(in);
+    } finally {
+      in.close();
+    }
+  }
+
+  // ==========================================================================
   //  PUBLIC
   // ==========================================================================
   public static RegionFileSystem open(Configuration conf, HRegionInfo regionInfo, boolean bootstrap)
@@ -155,8 +282,9 @@ public abstract class RegionFileSystem {
     RegionFileSystem rfs = getInstance(conf, fs, rootDir, regionInfo);
     if (bootstrap) {
       // TODO: are bootstrap and create two different things?
-      // should switch to bootstrap & read-only 
+      // should switch to bootstrap & read-only
       // legacy region wants to recover the .regioninfo :(
+      rfs.bootstrap();
     }
     return rfs;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java
index 8c26cdf..53f4fb0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.mob.MobConstants;
 
 public final class LegacyLayout {
   /** Name of the region info file that resides just under the region directory. */
-  private final static String REGION_INFO_FILE = ".regioninfo";
+  public final static String REGION_INFO_FILE = ".regioninfo";
 
   /** Temporary subdirectory of the region directory used for merges. */
   public static final String REGION_MERGES_DIR = ".merges";

http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java
index b2675a0..f6201a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java
@@ -238,7 +238,7 @@ public class LegacyMasterFileSystem extends MasterFileSystem {
       HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
       HTableDescriptor metaDescriptor = HTableDescriptor.metaTableDescriptor(c);
       MetaUtils.setInfoFamilyCachingForMeta(metaDescriptor, false);
-      HRegion meta = HRegion.createHRegion(metaHRI, rd, c, metaDescriptor, null);
+      HRegion meta = HRegion.createHRegion(c, rd, metaDescriptor, metaHRI, null);
       MetaUtils.setInfoFamilyCachingForMeta(metaDescriptor, true);
       meta.close();
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyRegionFileSystem.java
index d07a1f1..ce914a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyRegionFileSystem.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
@@ -45,7 +46,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableDescriptor;
 import org.apache.hadoop.hbase.fs.FSUtilsWithRetries;
 import org.apache.hadoop.hbase.fs.FsContext;
 import org.apache.hadoop.hbase.fs.RegionFileSystem;
@@ -61,12 +61,10 @@ import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 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.KeyValueUtil;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.regionserver.*;
@@ -101,10 +99,14 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
     this.regionInfoForFs = ServerRegionReplicaUtil.getRegionInfoForFs(hri);
   }
 
-  private Path getRegionDir() {
+  public Path getRegionDir() {
     return regionDir;
   }
 
+  public Path getTableDir() {
+    return tableDir;
+  }
+
   // ==========================================================================
   //  PUBLIC Methods - Families Related
   // ==========================================================================
@@ -112,7 +114,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
   public Collection<String> getFamilies() throws IOException {
     FileSystem fs = getFileSystem();
     FileStatus[] fds = FSUtils.listStatus(fs, regionDir, new FSUtils.FamilyDirFilter(fs));
-    if (fds == null) return null;
+    if (fds == null) return Collections.emptyList();
 
     ArrayList<String> families = new ArrayList<String>(fds.length);
     for (FileStatus status: fds) {
@@ -161,14 +163,14 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
   //  Temp Helpers
   // ===========================================================================
   /** @return {@link Path} to the region's temp directory, used for file creations */
-  Path getTempDir() {
+  public Path getTempDir() {
     return LegacyLayout.getRegionTempDir(regionDir);
   }
 
   /**
    * Clean up any temp detritus that may have been left around from previous operation attempts.
    */
-  void cleanupTempDir() throws IOException {
+  public void cleanupTempDir() throws IOException {
     fsWithRetries.deleteDir(getTempDir());
   }
 
@@ -190,7 +192,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @return {@link Path} to the directory of the specified family
    * @throws IOException if the directory creation fails.
    */
-  Path createStoreDir(final String familyName) throws IOException {
+  public Path createStoreDir(final String familyName) throws IOException {
     Path storeDir = getStoreDir(familyName);
     if (!fsWithRetries.createDir(storeDir))
       throw new IOException("Failed creating "+storeDir);
@@ -207,7 +209,9 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
     Path familyDir = getStoreDir(familyName);
     FileStatus[] files = FSUtils.listStatus(getFileSystem(), familyDir);
     if (files == null) {
-      LOG.debug("No StoreFiles for: " + familyDir);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("No StoreFiles for: " + familyDir);
+      }
       return null;
     }
 
@@ -233,7 +237,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @param fileName File Name
    * @return The qualified Path for the specified family/file
    */
-  Path getStoreFilePath(final String familyName, final String fileName) {
+  public Path getStoreFilePath(final String familyName, final String fileName) {
     Path familyDir = getStoreDir(familyName);
     return LegacyLayout.getStoreFile(familyDir, fileName).makeQualified(getFileSystem());
   }
@@ -245,7 +249,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @param fileName File Name
    * @return The {@link StoreFileInfo} for the specified family/file
    */
-  StoreFileInfo getStoreFileInfo(final String familyName, final String fileName)
+  public StoreFileInfo getStoreFileInfo(final String familyName, final String fileName)
       throws IOException {
     Path familyDir = getStoreDir(familyName);
     return ServerRegionReplicaUtil.getStoreFileInfo(getConfiguration(),
@@ -260,9 +264,18 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @throws IOException
    */
   public boolean hasReferences(final String familyName) throws IOException {
-    FileStatus[] files = FSUtils.listStatus(getFileSystem(),
-        getStoreDir(familyName), new FSUtils.ReferenceFileFilter(getFileSystem()));
-    return files != null && files.length > 0;
+    FileStatus[] files = FSUtils.listStatus(getFileSystem(), getStoreDir(familyName));
+    if (files != null) {
+      for(FileStatus stat: files) {
+        if(stat.isDirectory()) {
+          continue;
+        }
+        if(StoreFileInfo.isReference(stat.getPath())) {
+          return true;
+        }
+      }
+    }
+    return false;
   }
 
   /**
@@ -357,7 +370,9 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
     if (!fsWithRetries.exists(buildPath)) {
       throw new FileNotFoundException(buildPath.toString());
     }
-    LOG.debug("Committing store file " + buildPath + " as " + dstPath);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Committing store file " + buildPath + " as " + dstPath);
+    }
     // buildPath exists, therefore not doing an exists() check.
     if (!fsWithRetries.rename(buildPath, dstPath)) {
       throw new IOException("Failed rename of " + buildPath + " to " + dstPath);
@@ -371,7 +386,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @param storeFiles list of store files divided by family
    * @throws IOException
    */
-  void commitStoreFiles(final Map<byte[], List<StoreFile>> storeFiles) throws IOException {
+  public void commitStoreFiles(final Map<byte[], List<StoreFile>> storeFiles) throws IOException {
     for (Map.Entry<byte[], List<StoreFile>> es: storeFiles.entrySet()) {
       String familyName = Bytes.toString(es.getKey());
       for (StoreFile sf: es.getValue()) {
@@ -415,7 +430,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @return The destination {@link Path} of the bulk loaded file
    * @throws IOException
    */
-  Path bulkLoadStoreFile(final String familyName, Path srcPath, long seqNum)
+  public Path bulkLoadStoreFile(final String familyName, Path srcPath, long seqNum)
       throws IOException {
     // Copy the file if it's on another filesystem
     FileSystem fs = getFileSystem();
@@ -441,18 +456,18 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
   //  Splits Helpers
   // ===========================================================================
   /** @return {@link Path} to the temp directory used during split operations */
-  Path getSplitsDir() {
+  public Path getSplitsDir() {
     return LegacyLayout.getRegionSplitsDir(getRegionDir());
   }
 
-  Path getSplitsDir(final HRegionInfo hri) {
+  public Path getSplitsDir(final HRegionInfo hri) {
     return LegacyLayout.getRegionSplitsDir(getSplitsDir(), hri);
   }
 
   /**
    * Clean up any split detritus that may have been left around from previous split attempts.
    */
-  void cleanupSplitsDir() throws IOException {
+  public void cleanupSplitsDir() throws IOException {
     fsWithRetries.deleteDir(getSplitsDir());
   }
 
@@ -462,7 +477,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * Call this method on initial region deploy.
    * @throws IOException
    */
-  void cleanupAnySplitDetritus() throws IOException {
+  public void cleanupAnySplitDetritus() throws IOException {
     Path splitdir = this.getSplitsDir();
     if (!fsWithRetries.exists(splitdir)) return;
     // Look at the splitdir.  It could have the encoded names of the daughter
@@ -491,7 +506,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @param regionInfo daughter {@link HRegionInfo}
    * @throws IOException
    */
-  void cleanupDaughterRegion(final HRegionInfo regionInfo) throws IOException {
+  public void cleanupDaughterRegion(final HRegionInfo regionInfo) throws IOException {
     Path regionDir = LegacyLayout.getRegionDir(tableDir, regionInfo);
     if (!fsWithRetries.deleteDir(regionDir)) {
       throw new IOException("Failed delete of " + regionDir);
@@ -505,7 +520,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @param regionInfo                 daughter {@link org.apache.hadoop.hbase.HRegionInfo}
    * @throws IOException
    */
-  Path commitDaughterRegion(final HRegionInfo regionInfo)
+  public Path commitDaughterRegion(final HRegionInfo regionInfo)
       throws IOException {
     Path regionDir = LegacyLayout.getRegionDir(tableDir, regionInfo);
     Path daughterTmpDir = this.getSplitsDir(regionInfo);
@@ -529,7 +544,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
   /**
    * Create the region splits directory.
    */
-  void createSplitsDir() throws IOException {
+  public void createSplitsDir() throws IOException {
     createTempDir(getSplitsDir());
   }
 
@@ -558,7 +573,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @return Path to created reference.
    * @throws IOException
    */
-  Path splitStoreFile(final HRegionInfo hri, final String familyName, final StoreFile f,
+  public Path splitStoreFile(final HRegionInfo hri, final String familyName, final StoreFile f,
       final byte[] splitRow, final boolean top, RegionSplitPolicy splitPolicy)
           throws IOException {
 
@@ -568,24 +583,24 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
       try {
         if (top) {
           //check if larger than last key.
-          KeyValue splitKey = KeyValueUtil.createFirstOnRow(splitRow);
-          Cell lastKey = f.createReader().getLastKey();
+          Cell splitKey = CellUtil.createFirstOnRow(splitRow);
+          Cell lastKey = f.getLastKey();
           // If lastKey is null means storefile is empty.
           if (lastKey == null) {
             return null;
           }
-          if (f.getReader().getComparator().compare(splitKey, lastKey) > 0) {
+          if (f.getComparator().compare(splitKey, lastKey) > 0) {
             return null;
           }
         } else {
           //check if smaller than first key
-          KeyValue splitKey = KeyValueUtil.createLastOnRow(splitRow);
-          Cell firstKey = f.createReader().getFirstKey();
+          Cell splitKey = CellUtil.createLastOnRow(splitRow);
+          Cell firstKey = f.getFirstKey();
           // If firstKey is null means storefile is empty.
           if (firstKey == null) {
             return null;
           }
-          if (f.getReader().getComparator().compare(splitKey, firstKey) < 0) {
+          if (f.getComparator().compare(splitKey, firstKey) < 0) {
             return null;
           }
         }
@@ -613,7 +628,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
   //  Merge Helpers
   // ===========================================================================
   /** @return {@link Path} to the temp directory used during merge operations */
-  Path getMergesDir() {
+  public Path getMergesDir() {
     return LegacyLayout.getRegionMergesDir(getRegionDir());
   }
 
@@ -624,7 +639,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
   /**
    * Clean up any merge detritus that may have been left around from previous merge attempts.
    */
-  void cleanupMergesDir() throws IOException {
+  public void cleanupMergesDir() throws IOException {
     fsWithRetries.deleteDir(getMergesDir());
   }
 
@@ -633,7 +648,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @param mergedRegion {@link HRegionInfo}
    * @throws IOException
    */
-  void cleanupMergedRegion(final HRegionInfo mergedRegion) throws IOException {
+  public void cleanupMergedRegion(final HRegionInfo mergedRegion) throws IOException {
     Path regionDir = LegacyLayout.getRegionDir(tableDir, mergedRegion);
     if (fsWithRetries.deleteDir(regionDir)) {
       throw new IOException("Failed delete of " + regionDir);
@@ -645,7 +660,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @throws IOException If merges dir already exists or we fail to create it.
    * @see HRegionFileSystem#cleanupMergesDir()
    */
-  void createMergesDir() throws IOException {
+  public void createMergesDir() throws IOException {
     createTempDir(getMergesDir());
   }
 
@@ -659,7 +674,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @return Path to created reference.
    * @throws IOException
    */
-  Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName,
+  public Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName,
       final StoreFile f, final Path mergedDir)
       throws IOException {
     Path referenceDir = new Path(new Path(mergedDir,
@@ -684,7 +699,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @param mergedRegionInfo merged region {@link HRegionInfo}
    * @throws IOException
    */
-  void commitMergedRegion(final HRegionInfo mergedRegionInfo) throws IOException {
+  public void commitMergedRegion(final HRegionInfo mergedRegionInfo) throws IOException {
     Path regionDir = new Path(this.tableDir, mergedRegionInfo.getEncodedName());
     Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo);
     // Move the tmp dir in the expected location
@@ -704,7 +719,7 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
    * @param LOG log to output information
    * @throws IOException if an unexpected exception occurs
    */
-  void logFileSystemState(final Log LOG) throws IOException {
+  public void logFileSystemState(final Log LOG) throws IOException {
     FSUtils.logFileSystemState(getFileSystem(), this.getRegionDir(), LOG);
   }
 
@@ -738,6 +753,8 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
   // ==========================================================================
   @Override
   protected void bootstrap() throws IOException {
+    fsWithRetries.createDir(getRegionDir());
+
     // Cleanup temporary directories
     cleanupTempDir();
     cleanupSplitsDir();
@@ -747,8 +764,10 @@ public class LegacyRegionFileSystem extends RegionFileSystem {
     checkRegionInfoOnFilesystem();
   }
 
-  private void checkRegionInfoOnFilesystem() throws IOException {
-    // TODO
+  public void checkRegionInfoOnFilesystem() throws IOException {
+    writeRegionInfoFileContent(getConfiguration(), getFileSystem(),
+      LegacyLayout.getRegionInfoFile(getRegionDir()),
+      getRegionInfoFileContent(getRegionInfo()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index e748c3b..1005449 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.fs.RegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -199,10 +199,10 @@ public class CatalogJanitor extends ScheduledChore {
     Path rootdir = this.services.getMasterFileSystem().getRootDir();
     Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
     HTableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
-    HRegionFileSystem regionFs = null;
+    RegionFileSystem regionFs = null;
     try {
-      regionFs = HRegionFileSystem.openRegionFromFileSystem(
-          this.services.getConfiguration(), fs, tabledir, mergedRegion, true);
+      regionFs = RegionFileSystem.open(
+          this.services.getConfiguration(), fs, tabledir, mergedRegion, false);
     } catch (IOException e) {
       LOG.warn("Merged region does not exist: " + mergedRegion.getEncodedName());
     }
@@ -397,8 +397,6 @@ public class CatalogJanitor extends ScheduledChore {
 
     Path daughterRegionDir = new Path(tabledir, daughter.getEncodedName());
 
-    HRegionFileSystem regionFs = null;
-
     try {
       if (!FSUtils.isExists(fs, daughterRegionDir)) {
         return new Pair<Boolean, Boolean>(Boolean.FALSE, Boolean.FALSE);
@@ -410,10 +408,10 @@ public class CatalogJanitor extends ScheduledChore {
     }
 
     boolean references = false;
-    HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
     try {
-      regionFs = HRegionFileSystem.openRegionFromFileSystem(
-          this.services.getConfiguration(), fs, tabledir, daughter, true);
+      final RegionFileSystem regionFs = RegionFileSystem.open(this.services.getConfiguration(),
+          fs, tabledir, daughter, false);
+      final HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
 
       for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
         if ((references = regionFs.hasReferences(family.getNameAsString()))) {
@@ -461,4 +459,4 @@ public class CatalogJanitor extends ScheduledChore {
     return cleanMergeRegion(region, mergeRegions.getFirst(),
         mergeRegions.getSecond());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
index 65b32a4..73ed9e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
@@ -52,9 +52,9 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.fs.RegionFileSystem;
 import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.mapreduce.JobUtil;
@@ -112,7 +112,7 @@ public class CompactionTool extends Configured implements Tool {
         Path regionDir = path.getParent();
         Path tableDir = regionDir.getParent();
         HTableDescriptor htd = LegacyTableDescriptor.getTableDescriptorFromFs(fs, tableDir);
-        HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+        HRegionInfo hri = RegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
         compactStoreFiles(tableDir, htd, hri,
             path.getName(), compactOnce, major);
       } else if (isRegionDir(fs, path)) {
@@ -138,7 +138,7 @@ public class CompactionTool extends Configured implements Tool {
     private void compactRegion(final Path tableDir, final HTableDescriptor htd,
         final Path regionDir, final boolean compactOnce, final boolean major)
         throws IOException {
-      HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+      HRegionInfo hri = RegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
       for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
         compactStoreFiles(tableDir, htd, hri, familyDir.getName(), compactOnce, major);
       }
@@ -181,20 +181,14 @@ public class CompactionTool extends Configured implements Tool {
     private static HStore getStore(final Configuration conf, final FileSystem fs,
         final Path tableDir, final HTableDescriptor htd, final HRegionInfo hri,
         final String familyName, final Path tempDir) throws IOException {
-      HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, hri) {
-        @Override
-        public Path getTempDir() {
-          return tempDir;
-        }
-      };
-      HRegion region = new HRegion(regionFs, null, conf, htd, null);
+      RegionFileSystem regionFs = null;
+      HRegion region = new HRegion(regionFs, htd, null, null);
       return new HStore(region, htd.getFamily(Bytes.toBytes(familyName)), conf);
     }
   }
 
   private static boolean isRegionDir(final FileSystem fs, final Path path) throws IOException {
-    Path regionInfo = new Path(path, HRegionFileSystem.REGION_INFO_FILE);
-    return fs.exists(regionInfo);
+    return fs.exists(null);
   }
 
   private static boolean isTableDir(final FileSystem fs, final Path path) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c347b84c/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 f6d2e36..c284082 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
@@ -133,6 +133,7 @@ import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.FilterWrapper;
 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
+import org.apache.hadoop.hbase.fs.RegionFileSystem;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -287,7 +288,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   final AtomicLong compactionNumBytesCompacted = new AtomicLong(0L);
 
   private final WAL wal;
-  private final HRegionFileSystem fs;
+  private final RegionFileSystem fs;
   protected final Configuration conf;
   private final Configuration baseConf;
   private final int rowLockWaitDuration;
@@ -616,11 +617,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   @Deprecated
   @VisibleForTesting
-  public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
+  private HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
       final Configuration confParam, final HRegionInfo regionInfo,
-      final HTableDescriptor htd, final RegionServerServices rsServices) {
-    this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo),
-      wal, confParam, htd, rsServices);
+      final HTableDescriptor htd, final RegionServerServices rsServices)
+      throws IOException {
+    this(RegionFileSystem.open(confParam, fs, tableDir, regionInfo, false), htd, wal, rsServices);
   }
 
   /**
@@ -639,7 +640,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param htd the table descriptor
    * @param rsServices reference to {@link RegionServerServices} or null
    */
-  public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration confParam,
+  public HRegion(final RegionFileSystem rfs, final HTableDescriptor htd, final WAL wal,
+      final RegionServerServices rsServices) {
+    this(rfs, wal, rfs.getConfiguration(), htd, rsServices);
+  }
+
+  private HRegion(final RegionFileSystem fs, final WAL wal, final Configuration confParam,
       final HTableDescriptor htd, final RegionServerServices rsServices) {
     if (htd == null) {
       throw new IllegalArgumentException("Need table descriptor");
@@ -842,9 +848,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       // Remove temporary data left over from old regions
       status.setStatus("Cleaning up temporary data from old regions");
       fs.cleanupTempDir();
-    }
 
-    if (this.writestate.writesEnabled) {
       status.setStatus("Cleaning up detritus from prior splits");
       // Get rid of any splits or merges that were lost in-progress.  Clean out
       // these directories here on open.  We may be opening a region that was
@@ -1097,7 +1101,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
     FileSystem fs = tablePath.getFileSystem(conf);
 
-    HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo);
+    RegionFileSystem regionFs = RegionFileSystem.open(conf, fs, tablePath, regionInfo, false);
     for (HColumnDescriptor family: tableDescriptor.getFamilies()) {
       Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family.getNameAsString());
       if (storeFiles == null) continue;
@@ -1704,7 +1708,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   /** @return the {@link HRegionFileSystem} used by this region */
-  public HRegionFileSystem getRegionFileSystem() {
+  public RegionFileSystem getRegionFileSystem() {
     return this.fs;
   }
 
@@ -6296,20 +6300,27 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param htd the table descriptor
    * @return the new instance
    */
-  static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
-      Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
-      RegionServerServices rsServices) {
+  static HRegion newHRegion(Configuration conf, FileSystem fs, Path rootDir,
+      HTableDescriptor htd, HRegionInfo regionInfo, WAL wal, RegionServerServices rsServices)
+      throws IOException {
+    RegionFileSystem rfs = RegionFileSystem.open(conf, fs, rootDir, regionInfo, false);
+    return newHRegion(rfs, htd, wal, rsServices);
+  }
+
+  private static HRegion newHRegion(RegionFileSystem rfs, HTableDescriptor htd, WAL wal,
+      RegionServerServices rsServices) throws IOException {
     try {
+      Configuration conf = rfs.getConfiguration();
+
       @SuppressWarnings("unchecked")
       Class<? extends HRegion> regionClass =
           (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
 
       Constructor<? extends HRegion> c =
-          regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
-              Configuration.class, HRegionInfo.class, HTableDescriptor.class,
-              RegionServerServices.class);
+          regionClass.getConstructor(RegionFileSystem.class, HTableDescriptor.class,
+              WAL.class, RegionServerServices.class);
 
-      return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
+      return c.newInstance(rfs, htd, wal, rsServices);
     } catch (Throwable e) {
       // todo: what should I throw here?
       throw new IllegalStateException("Could not instantiate a region instance.", e);
@@ -6326,27 +6337,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @return new HRegion
    * @throws IOException
    */
-  public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
-        final Configuration conf, final HTableDescriptor hTableDescriptor,
-        final WAL wal, final boolean initialize)
-  throws IOException {
+  public static HRegion createHRegion(final Configuration conf, final Path rootDir,
+        final HTableDescriptor hTableDescriptor, final HRegionInfo info,
+        final WAL wal, final boolean initialize) throws IOException {
     LOG.info("creating HRegion " + info.getTable().getNameAsString()
         + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
         " Table name == " + info.getTable().getNameAsString());
-    FileSystem fs = FileSystem.get(conf);
-    Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
-    HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
-    HRegion region = HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, null);
+    FileSystem fs = rootDir.getFileSystem(conf);
+    RegionFileSystem rfs = RegionFileSystem.open(conf, fs, rootDir, info, true);
+    HRegion region = HRegion.newHRegion(rfs, hTableDescriptor, wal, null);
     if (initialize) region.initialize(null);
     return region;
   }
 
-  public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
-                                      final Configuration conf,
-                                      final HTableDescriptor hTableDescriptor,
-                                      final WAL wal)
-    throws IOException {
-    return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
+  public static HRegion createHRegion(final Configuration conf, final Path rootDir,
+      final HTableDescriptor hTableDescriptor, final HRegionInfo info, final WAL wal)
+      throws IOException {
+    return createHRegion(conf, rootDir, hTableDescriptor, info, wal, true);
   }
 
 
@@ -6480,36 +6487,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal,
       final RegionServerServices rsServices, final CancelableProgressable reporter)
       throws IOException {
-    Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
-    return openHRegion(conf, fs, rootDir, tableDir, info, htd, wal, rsServices, reporter);
-  }
-
-  /**
-   * Open a Region.
-   * @param conf The Configuration object to use.
-   * @param fs Filesystem to use
-   * @param rootDir Root directory for HBase instance
-   * @param info Info for region to be opened.
-   * @param htd the table descriptor
-   * @param wal WAL for region to use. This method will call
-   * WAL#setSequenceNumber(long) passing the result of the call to
-   * HRegion#getMinSequenceId() to ensure the wal id is properly kept
-   * up.  HRegionStore does this every time it opens a new region.
-   * @param rsServices An interface we can request flushes against.
-   * @param reporter An interface we can report progress against.
-   * @return new HRegion
-   * @throws IOException
-   */
-  public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
-      final Path rootDir, final Path tableDir, final HRegionInfo info, final HTableDescriptor htd,
-      final WAL wal, final RegionServerServices rsServices,
-      final CancelableProgressable reporter)
-      throws IOException {
     if (info == null) throw new NullPointerException("Passed region info is null");
     if (LOG.isDebugEnabled()) {
       LOG.debug("Opening region: " + info);
     }
-    HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
+    HRegion r = HRegion.newHRegion(conf, fs, rootDir, htd, info, wal, rsServices);
     return r.openHRegion(reporter);
   }
 
@@ -6527,9 +6509,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
       throws IOException {
-    HRegionFileSystem regionFs = other.getRegionFileSystem();
-    HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
-        other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
+    RegionFileSystem regionFs = other.getRegionFileSystem();
+    HRegion r = newHRegion(other.baseConf, regionFs.getFileSystem(),
+      regionFs.getRootDir(), other.getTableDesc(), other.getRegionInfo(),
+      other.getWAL(), null);
     return r.openHRegion(reporter);
   }
 
@@ -6578,7 +6561,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
 
     Path rootDir = FSUtils.getRootDir(conf);
-    Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
 
     FileSystem fs = null;
     if (rsServices != null) {
@@ -6621,8 +6603,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     fs.commitDaughterRegion(hri);
 
     // Create the daughter HRegion instance
-    HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(), fs.getFileSystem(),
-        this.getBaseConf(), hri, this.getTableDesc(), rsServices);
+    HRegion r = HRegion.newHRegion(this.getBaseConf(), fs.getFileSystem(), this.fs.getRootDir(),
+        this.getTableDesc(), hri, this.getWAL(), rsServices);
     r.readRequestsCount.add(this.getReadRequestsCount() / 2);
     r.filteredReadRequestsCount.add(this.getFilteredReadRequestsCount() / 2);
     r.writeRequestsCount.add(this.getWriteRequestsCount() / 2);
@@ -6637,9 +6619,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   HRegion createMergedRegionFromMerges(final HRegionInfo mergedRegionInfo,
       final HRegion region_b) throws IOException {
-    HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(),
-        fs.getFileSystem(), this.getBaseConf(), mergedRegionInfo,
-        this.getTableDesc(), this.rsServices);
+    HRegion r = HRegion.newHRegion(this.getBaseConf(), fs.getFileSystem(), this.fs.getRootDir(),
+        this.getTableDesc(), mergedRegionInfo, this.getWAL(), this.rsServices);
     r.readRequestsCount.add(this.getReadRequestsCount()
         + region_b.getReadRequestsCount());
     r.filteredReadRequestsCount.add(this.getFilteredReadRequestsCount()