You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2013/02/08 21:14:03 UTC

svn commit: r1444212 - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/util/ test/java/org/apache/hadoop/hbase/ test/java/org/apache/hadoop/hbase/coprocessor/ test/java/org/apache/hado...

Author: mbertozzi
Date: Fri Feb  8 20:14:02 2013
New Revision: 1444212

URL: http://svn.apache.org/r1444212
Log:
HBASE-7786 Consolidate HRegion creation/opening API

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1444212&r1=1444211&r2=1444212&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri Feb  8 20:14:02 2013
@@ -391,16 +391,6 @@ public class HRegion implements HeapSize
   private final MetricsRegionWrapperImpl metricsRegionWrapper;
 
   /**
-   * HRegion copy constructor. Useful when reopening a closed region (normally
-   * for unit tests)
-   * @param other original object
-   */
-  public HRegion(HRegion other) {
-    this(other.getTableDir(), other.getLog(), other.getFilesystem(),
-        other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
-  }
-
-  /**
    * HRegion constructor.  his constructor should only be used for testing and
    * extensions.  Instances of HRegion should be instantiated with the
    * {@link HRegion#newHRegion(Path, HLog, FileSystem, Configuration, HRegionInfo, HTableDescriptor, RegionServerServices)} method.
@@ -516,9 +506,13 @@ public class HRegion implements HeapSize
 
   /**
    * Initialize this region.
+   * Used only by tests and SplitTransaction to reopen the region.
+   * You should use createHRegion() or openHRegion()
    * @return What the next sequence (edit) id should be.
    * @throws IOException e
+   * @deprecated use HRegion.createHRegion() or HRegion.openHRegion()
    */
+  @Deprecated
   public long initialize() throws IOException {
     return initialize(null);
   }
@@ -530,8 +524,7 @@ public class HRegion implements HeapSize
    * @return What the next sequence (edit) id should be.
    * @throws IOException e
    */
-  public long initialize(final CancelableProgressable reporter)
-  throws IOException {
+  private long initialize(final CancelableProgressable reporter) throws IOException {
     MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
     long nextSeqId = -1;
     try {
@@ -3800,7 +3793,7 @@ public class HRegion implements HeapSize
    * @param rsServices
    * @return the new instance
    */
-  public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
+  static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
       Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
       RegionServerServices rsServices) {
     try {
@@ -3967,7 +3960,7 @@ public class HRegion implements HeapSize
    * HLog#setSequenceNumber(long) passing the result of the call to
    * HRegion#getMinSequenceId() to ensure the log id is properly kept
    * up.  HRegionStore does this every time it opens a new region.
-   * @param conf
+   * @param conf The Configuration object to use.
    * @param rsServices An interface we can request flushes against.
    * @param reporter An interface we can report progress against.
    * @return new HRegion
@@ -3979,26 +3972,22 @@ public class HRegion implements HeapSize
     final RegionServerServices rsServices,
     final CancelableProgressable reporter)
   throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Opening region: " + info);
-    }
-    if (info == null) {
-      throw new NullPointerException("Passed region info is null");
-    }
-    Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
-      info.getTableName());
-    FileSystem fs = null;
-    if (rsServices != null) {
-      fs = rsServices.getFileSystem();
-    }
-    if (fs == null) {
-      fs = FileSystem.get(conf);
-    }
-    HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info,
-      htd, rsServices);
-    return r.openHRegion(reporter);
+    return openHRegion(FSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
   }
 
+  /**
+   * Open a Region.
+   * @param rootDir Root directory for HBase instance
+   * @param info Info for region to be opened.
+   * @param htd the table descriptor
+   * @param wal HLog for region to use. This method will call
+   * HLog#setSequenceNumber(long) passing the result of the call to
+   * HRegion#getMinSequenceId() to ensure the log id is properly kept
+   * up.  HRegionStore does this every time it opens a new region.
+   * @param conf The Configuration object to use.
+   * @return new HRegion
+   * @throws IOException
+   */
   public static HRegion openHRegion(Path rootDir, final HRegionInfo info,
       final HTableDescriptor htd, final HLog wal, final Configuration conf)
   throws IOException {
@@ -4009,14 +3998,15 @@ public class HRegion implements HeapSize
    * Open a Region.
    * @param rootDir Root directory for HBase instance
    * @param info Info for region to be opened.
+   * @param htd the table descriptor
    * @param wal HLog for region to use. This method will call
    * HLog#setSequenceNumber(long) passing the result of the call to
    * HRegion#getMinSequenceId() to ensure the log id is properly kept
    * up.  HRegionStore does this every time it opens a new region.
-   * @param conf
+   * @param conf The Configuration object to use.
+   * @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 Path rootDir, final HRegionInfo info,
@@ -4024,16 +4014,79 @@ public class HRegion implements HeapSize
       final RegionServerServices rsServices,
       final CancelableProgressable reporter)
   throws IOException {
+    FileSystem fs = null;
+    if (rsServices != null) {
+      fs = rsServices.getFileSystem();
+    }
+    if (fs == null) {
+      fs = FileSystem.get(conf);
+    }
+    return openHRegion(conf, fs, rootDir, 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 HLog for region to use. This method will call
+   * HLog#setSequenceNumber(long) passing the result of the call to
+   * HRegion#getMinSequenceId() to ensure the log id is properly kept
+   * up.  HRegionStore does this every time it opens a new region.
+   * @return new HRegion
+   * @throws IOException
+   */
+  public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
+      final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final HLog wal)
+      throws IOException {
+    return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
+  }
+
+  /**
+   * 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 HLog for region to use. This method will call
+   * HLog#setSequenceNumber(long) passing the result of the call to
+   * HRegion#getMinSequenceId() to ensure the log 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 HRegionInfo info, final HTableDescriptor htd, final HLog wal,
+      final RegionServerServices rsServices, final CancelableProgressable reporter)
+      throws IOException {
     if (info == null) throw new NullPointerException("Passed region info is null");
     LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
     if (LOG.isDebugEnabled()) {
       LOG.debug("Opening region: " + info);
     }
     Path dir = HTableDescriptor.getTableDir(rootDir, info.getTableName());
-    HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info, htd, rsServices);
+    HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info, htd, rsServices);
     return r.openHRegion(reporter);
   }
 
+  /**
+   * Useful when reopening a closed region (normally for unit tests)
+   * @param other original object
+   * @param reporter An interface we can report progress against.
+   * @return new HRegion
+   * @throws IOException
+   */
+  public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
+      throws IOException {
+    HRegion r = newHRegion(other.getTableDir(), other.getLog(), other.getFilesystem(),
+        other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
+    return r.openHRegion(reporter);
+  }
 
   /**
    * Open HRegion.
@@ -4062,6 +4115,22 @@ public class HRegion implements HeapSize
   }
 
   /**
+   * Create a daughter region from given a temp directory with the region data.
+   * @param hri Spec. for daughter region to open.
+   * @param daughterTmpDir Directory that contains region files.
+   * @throws IOException
+   */
+  HRegion createDaughterRegion(final HRegionInfo hri, final Path daughterTmpDir)
+      throws IOException {
+    HRegion r = HRegion.newHRegion(this.getTableDir(), this.getLog(), fs,
+        this.getBaseConf(), hri, this.getTableDesc(), rsServices);
+    r.readRequestsCount.set(this.getReadRequestsCount() / 2);
+    r.writeRequestsCount.set(this.getWriteRequestsCount() / 2);
+    moveInitialFilesIntoPlace(fs, daughterTmpDir, r.getRegionDir());
+    return r;
+  }
+
+  /**
    * Inserts a new region's meta information into the passed
    * <code>meta</code> region. Used by the HMaster bootstrap code adding
    * new table to ROOT table.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java?rev=1444212&r1=1444211&r2=1444212&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java Fri Feb  8 20:14:02 2013
@@ -313,11 +313,11 @@ public class SplitTransaction {
     // stuff in fs that needs cleanup -- a storefile or two.  Thats why we
     // add entry to journal BEFORE rather than AFTER the change.
     this.journal.add(JournalEntry.STARTED_REGION_A_CREATION);
-    HRegion a = createDaughterRegion(this.hri_a, this.parent.rsServices);
+    HRegion a = createDaughterRegion(this.hri_a);
 
     // Ditto
     this.journal.add(JournalEntry.STARTED_REGION_B_CREATION);
-    HRegion b = createDaughterRegion(this.hri_b, this.parent.rsServices);
+    HRegion b = createDaughterRegion(this.hri_b);
 
     // This is the point of no return.  Adding subsequent edits to .META. as we
     // do below when we do the daughter opens adding each to .META. can fail in
@@ -696,20 +696,10 @@ public class SplitTransaction {
    * @throws IOException
    * @see #cleanupDaughterRegion(FileSystem, Path, String)
    */
-  HRegion createDaughterRegion(final HRegionInfo hri,
-      final RegionServerServices rsServices)
-  throws IOException {
+  HRegion createDaughterRegion(final HRegionInfo hri) throws IOException {
     // Package private so unit tests have access.
-    FileSystem fs = this.parent.getFilesystem();
-    Path regionDir = getSplitDirForDaughter(this.parent.getFilesystem(),
-      this.splitdir, hri);
-    HRegion r = HRegion.newHRegion(this.parent.getTableDir(),
-      this.parent.getLog(), fs, this.parent.getBaseConf(),
-      hri, this.parent.getTableDesc(), rsServices);
-    r.readRequestsCount.set(this.parent.getReadRequestsCount() / 2);
-    r.writeRequestsCount.set(this.parent.getWriteRequestsCount() / 2);
-    HRegion.moveInitialFilesIntoPlace(fs, regionDir, r.getRegionDir());
-    return r;
+    Path regionDir = getSplitDirForDaughter(this.splitdir, hri);
+    return this.parent.createDaughterRegion(hri, regionDir);
   }
 
   private static void cleanupDaughterRegion(final FileSystem fs,
@@ -723,15 +713,13 @@ public class SplitTransaction {
   /*
    * Get the daughter directories in the splits dir.  The splits dir is under
    * the parent regions' directory.
-   * @param fs
    * @param splitdir
    * @param hri
    * @return Path to daughter split dir.
    * @throws IOException
    */
-  private static Path getSplitDirForDaughter(final FileSystem fs,
-      final Path splitdir, final HRegionInfo hri)
-  throws IOException {
+  private static Path getSplitDirForDaughter(final Path splitdir, final HRegionInfo hri)
+      throws IOException {
     return new Path(splitdir, hri.getEncodedName());
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java?rev=1444212&r1=1444211&r2=1444212&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java Fri Feb  8 20:14:02 2013
@@ -138,7 +138,7 @@ class HMerge {
   private static abstract class Merger {
     protected final Configuration conf;
     protected final FileSystem fs;
-    protected final Path tabledir;
+    protected final Path rootDir;
     protected final HTableDescriptor htd;
     protected final HLog hlog;
     private final long maxFilesize;
@@ -151,11 +151,9 @@ class HMerge {
       this.maxFilesize = conf.getLong(HConstants.HREGION_MAX_FILESIZE,
           HConstants.DEFAULT_MAX_FILE_SIZE);
 
-      this.tabledir = new Path(
-          fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR))),
-          Bytes.toString(tableName)
-      );
-      this.htd = FSTableDescriptors.getTableDescriptor(this.fs, this.tabledir);
+      this.rootDir = FSUtils.getRootDir(conf);
+      Path tabledir = HTableDescriptor.getTableDir(this.rootDir, tableName);
+      this.htd = FSTableDescriptors.getTableDescriptor(this.fs, tabledir);
       String logname = "merge_" + System.currentTimeMillis() + HConstants.HREGION_LOGDIR_NAME;
 
       this.hlog = HLogFactory.createHLog(fs, tabledir, logname, conf);
@@ -192,14 +190,10 @@ class HMerge {
       long nextSize = 0;
       for (int i = 0; i < info.length - 1; i++) {
         if (currentRegion == null) {
-          currentRegion = HRegion.newHRegion(tabledir, hlog, fs, conf, info[i],
-            this.htd, null);
-          currentRegion.initialize();
+          currentRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i], this.htd, hlog);
           currentSize = currentRegion.getLargestHStoreSize();
         }
-        nextRegion = HRegion.newHRegion(tabledir, hlog, fs, conf, info[i + 1],
-          this.htd, null);
-        nextRegion.initialize();
+        nextRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i + 1], this.htd, hlog);
         nextSize = nextRegion.getLargestHStoreSize();
 
         if ((currentSize + nextSize) <= (maxFilesize / 2)) {
@@ -349,21 +343,15 @@ class HMerge {
         throws IOException {
       super(conf, fs, HConstants.META_TABLE_NAME);
 
-      Path rootTableDir = HTableDescriptor.getTableDir(
-          fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR))),
-          HConstants.ROOT_TABLE_NAME);
+      Path rootDir = FSUtils.getRootDir(conf);
 
       // Scan root region to find all the meta regions
-
-      root = HRegion.newHRegion(rootTableDir, hlog, fs, conf,
-          HRegionInfo.ROOT_REGIONINFO, HTableDescriptor.ROOT_TABLEDESC, null);
-      root.initialize();
+      root = HRegion.openHRegion(conf, fs, rootDir, HRegionInfo.ROOT_REGIONINFO,
+          HTableDescriptor.ROOT_TABLEDESC, hlog);
 
       Scan scan = new Scan();
-      scan.addColumn(HConstants.CATALOG_FAMILY,
-          HConstants.REGIONINFO_QUALIFIER);
-      InternalScanner rootScanner =
-        root.getScanner(scan);
+      scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
+      InternalScanner rootScanner = root.getScanner(scan);
 
       try {
         List<KeyValue> results = new ArrayList<KeyValue>();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java?rev=1444212&r1=1444211&r2=1444212&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java Fri Feb  8 20:14:02 2013
@@ -169,9 +169,7 @@ public abstract class HBaseTestCase exte
 
   protected HRegion openClosedRegion(final HRegion closedRegion)
   throws IOException {
-    HRegion r = new HRegion(closedRegion);
-    r.initialize();
-    return r;
+    return HRegion.openHRegion(closedRegion, null);
   }
 
   /**

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java?rev=1444212&r1=1444211&r2=1444212&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java Fri Feb  8 20:14:02 2013
@@ -401,8 +401,7 @@ public class TestCoprocessorInterface ex
   HRegion reopenRegion(final HRegion closedRegion, Class<?> ... implClasses)
       throws IOException {
     //HRegionInfo info = new HRegionInfo(tableName, null, null, false);
-    HRegion r = new HRegion(closedRegion);
-    r.initialize();
+    HRegion r = HRegion.openHRegion(closedRegion, null);
 
     // this following piece is a hack. currently a coprocessorHost
     // is secretly loaded at OpenRegionHandler. we don't really

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java?rev=1444212&r1=1444211&r2=1444212&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java Fri Feb  8 20:14:02 2013
@@ -262,12 +262,10 @@ public class TestWALObserver {
         FileSystem newFS = FileSystem.get(newConf);
         // Make a new wal for new region open.
         HLog wal2 = createWAL(newConf);
-        Path tableDir =
-          HTableDescriptor.getTableDir(hbaseRootDir, hri.getTableName());
-        HRegion region = new HRegion(tableDir, wal2, FileSystem.get(newConf),
-          newConf, hri, htd, TEST_UTIL.getHBaseCluster().getRegionServer(0));
+        HRegion region = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir,
+            hri, htd, wal2, TEST_UTIL.getHBaseCluster().getRegionServer(0), null);
+        long seqid2 = region.getOpenSeqNum();
 
-        long seqid2 = region.initialize();
         SampleRegionWALObserver cp2 =
           (SampleRegionWALObserver)region.getCoprocessorHost().findCoprocessor(
               SampleRegionWALObserver.class.getName());

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java?rev=1444212&r1=1444211&r2=1444212&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java Fri Feb  8 20:14:02 2013
@@ -258,7 +258,7 @@ public class TestSplitTransaction {
     // Start transaction.
     SplitTransaction st = prepareGOOD_SPLIT_ROW();
     SplitTransaction spiedUponSt = spy(st);
-    when(spiedUponSt.createDaughterRegion(spiedUponSt.getSecondDaughter(), null)).
+    when(spiedUponSt.createDaughterRegion(spiedUponSt.getSecondDaughter())).
       thenThrow(new MockedFailedDaughterCreation());
     // Run the execute.  Look at what it returns.
     boolean expectedException = false;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1444212&r1=1444211&r2=1444212&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Fri Feb  8 20:14:02 2013
@@ -268,10 +268,10 @@ public class TestWALReplay {
     HLog wal3 = createWAL(this.conf);
     wal3.setSequenceNumber(wal2.getSequenceNumber());
     try {
-      final HRegion region = new HRegion(basedir, wal3, this.fs, this.conf, hri,
-        htd, null);
-      long seqid = region.initialize();
-      assertTrue(seqid > wal3.getSequenceNumber());
+      long wal3SeqId = wal3.getSequenceNumber();
+      HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal3);
+      long seqid = region.getOpenSeqNum();
+      assertTrue(seqid > wal3SeqId);
 
       // TODO: Scan all.
       region.close();
@@ -325,9 +325,10 @@ public class TestWALReplay {
       public Object run() throws Exception {
         runWALSplit(newConf);
         HLog wal2 = createWAL(newConf);
-        HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
-          newConf, hri, htd, null);
-        long seqid2 = region2.initialize();
+
+        HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf),
+          hbaseRootDir, hri, htd, wal2);
+        long seqid2 = region2.getOpenSeqNum();
         assertTrue(seqid2 > -1);
 
         // I can't close wal1.  Its been appropriated when we split.
@@ -365,8 +366,8 @@ public class TestWALReplay {
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
     HLog wal = createWAL(this.conf);
-    HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
-    long seqid = region.initialize();
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    long seqid = region.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal.setSequenceNumber(seqid);
     boolean first = true;
@@ -390,8 +391,8 @@ public class TestWALReplay {
     wal.close();
     runWALSplit(this.conf);
     HLog wal2 = createWAL(this.conf);
-    HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
-    long seqid2 = region2.initialize();
+    HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal2);
+    long seqid2 = region2.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal2.setSequenceNumber(seqid2);
     assertTrue(seqid + result.size() < seqid2);
@@ -482,8 +483,8 @@ public class TestWALReplay {
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
     HLog wal = createWAL(this.conf);
-    HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
-    long seqid = region.initialize();
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    long seqid = region.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal.setSequenceNumber(seqid);
     for (HColumnDescriptor hcd: htd.getFamilies()) {
@@ -518,8 +519,8 @@ public class TestWALReplay {
     // Let us try to split and recover
     runWALSplit(this.conf);
     HLog wal2 = createWAL(this.conf);
-    HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
-    long seqid2 = region2.initialize();
+    HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal2);
+    long seqid2 = region2.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal2.setSequenceNumber(seqid2);
     assertTrue(seqid + result.size() < seqid2);
@@ -638,9 +639,9 @@ public class TestWALReplay {
 
     // Mock the HLog
     MockHLog wal = createMockWAL(this.conf);
-    
-    HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
-    long seqid = region.initialize();
+
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    long seqid = region.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all
     // regions.
     wal.setSequenceNumber(seqid);