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/03/25 12:15:37 UTC

svn commit: r1460617 [2/2] - in /hbase/branches/0.95/hbase-server/src: main/java/org/apache/hadoop/hbase/backup/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/snapshot/ main/...

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java Mon Mar 25 11:15:36 2013
@@ -43,7 +43,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -57,8 +56,7 @@ import org.apache.hadoop.hbase.catalog.M
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.NoServerForRegionException;
-import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -123,7 +121,8 @@ import com.google.common.collect.Sets;
  * <p>
  * The more complicated answer is that this depends upon the largest storefile
  * in your region. With a growing data size, this will get larger over time. You
- * want the largest region to be just big enough that the {@link HStore} compact
+ * want the largest region to be just big enough that the
+ * {@link org.apache.hadoop.hbase.regionserver.HStore} compact
  * selection algorithm only compacts it due to a timed major. If you don't, your
  * cluster can be prone to compaction storms as the algorithm decides to run
  * major compactions on a large series of regions all at once. Note that
@@ -628,9 +627,10 @@ public class RegionSplitter {
     LinkedList<Pair<byte[], byte[]>> physicalSplitting = Lists.newLinkedList();
 
     // get table info
-    Path hbDir = new Path(table.getConfiguration().get(HConstants.HBASE_DIR));
-    Path tableDir = HTableDescriptor.getTableDir(hbDir, table.getTableName());
+    Path rootDir = FSUtils.getRootDir(table.getConfiguration());
+    Path tableDir = HTableDescriptor.getTableDir(rootDir, table.getTableName());
     FileSystem fs = tableDir.getFileSystem(table.getConfiguration());
+    HTableDescriptor htd = table.getTableDescriptor();
 
     // clear the cache to forcibly refresh region information
     table.clearRegionCache();
@@ -661,25 +661,22 @@ public class RegionSplitter {
         check.add(table.getRegionLocation(start).getRegionInfo());
         check.add(table.getRegionLocation(split).getRegionInfo());
         for (HRegionInfo hri : check.toArray(new HRegionInfo[] {})) {
-          boolean refFound = false;
           byte[] sk = hri.getStartKey();
           if (sk.length == 0)
             sk = splitAlgo.firstRow();
           String startKey = splitAlgo.rowToStr(sk);
-          HTableDescriptor htd = table.getTableDescriptor();
+
+          HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
+              table.getConfiguration(), fs, tableDir, hri, true);
+
           // check every Column Family for that region
+          boolean refFound = false;
           for (HColumnDescriptor c : htd.getFamilies()) {
-            Path cfDir = HStore.getStoreHomedir(tableDir, hri, c.getName());
-            if (fs.exists(cfDir)) {
-              for (FileStatus file : fs.listStatus(cfDir)) {
-                refFound |= StoreFileInfo.isReference(file.getPath());
-                if (refFound)
-                  break;
-              }
-            }
-            if (refFound)
+            if ((refFound = regionFs.hasReferences(htd.getNameAsString()))) {
               break;
+            }
           }
+
           // compaction is completed when all reference files are gone
           if (!refFound) {
             check.remove(hri);
@@ -691,8 +688,7 @@ public class RegionSplitter {
           physicalSplitting.add(region);
         }
       } catch (NoServerForRegionException nsfre) {
-        LOG.debug("No Server Exception thrown for: "
-            + splitAlgo.rowToStr(start));
+        LOG.debug("No Server Exception thrown for: " + splitAlgo.rowToStr(start));
         physicalSplitting.add(region);
         table.clearRegionCache();
       }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java Mon Mar 25 11:15:36 2013
@@ -142,7 +142,8 @@ public class TestHFileArchiving {
     FileSystem fs = UTIL.getTestFileSystem();
 
     // now attempt to depose the region
-    Path regionDir = HRegion.getRegionDir(region.getTableDir().getParent(), region.getRegionInfo());
+    Path rootDir = region.getRegionFileSystem().getTableDir().getParent();
+    Path regionDir = HRegion.getRegionDir(rootDir, region.getRegionInfo());
 
     HFileArchiver.archiveRegion(UTIL.getConfiguration(), fs, region.getRegionInfo());
 
@@ -175,7 +176,7 @@ public class TestHFileArchiving {
     assertEquals(1, servingRegions.size());
     HRegion region = servingRegions.get(0);
 
-    FileSystem fs = region.getFilesystem();
+    FileSystem fs = region.getRegionFileSystem().getFileSystem();
 
     // make sure there are some files in the regiondir
     Path rootDir = FSUtils.getRootDir(fs.getConf());
@@ -241,8 +242,7 @@ public class TestHFileArchiving {
     clearArchiveDirectory();
 
     // then get the current store files
-    Path regionDir = region.getRegionDir();
-    List<String> storeFiles = getRegionStoreFiles(fs, regionDir);
+    List<String> storeFiles = getRegionStoreFiles(region);
 
     // then delete the table so the hfiles get archived
     UTIL.deleteTable(TABLE_NAME);
@@ -302,8 +302,7 @@ public class TestHFileArchiving {
     clearArchiveDirectory();
 
     // then get the current store files
-    Path regionDir = region.getRegionDir();
-    List<String> storeFiles = getRegionStoreFiles(fs, regionDir);
+    List<String> storeFiles = getRegionStoreFiles(region);
 
     // then delete the table so the hfiles get archived
     UTIL.getHBaseAdmin().deleteColumn(TABLE_NAME, TEST_FAM);
@@ -419,8 +418,9 @@ public class TestHFileArchiving {
     return fileNames;
   }
 
-  private List<String> getRegionStoreFiles(final FileSystem fs, final Path regionDir) 
-      throws IOException {
+  private List<String> getRegionStoreFiles(final HRegion region) throws IOException {
+    Path regionDir = region.getRegionFileSystem().getRegionDir();
+    FileSystem fs = region.getRegionFileSystem().getFileSystem();
     List<String> storeFiles = getAllFileNames(fs, regionDir);
     // remove all the non-storefile named files for the region
     for (int i = 0; i < storeFiles.size(); i++) {

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java Mon Mar 25 11:15:36 2013
@@ -70,9 +70,9 @@ public class TestHFileLinkCleaner {
 
     Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
     Path archiveStoreDir = HFileArchiveUtil.getStoreArchivePath(conf,
-          tableName, hri, familyName);
+          tableName, hri.getEncodedName(), familyName);
     Path archiveLinkStoreDir = HFileArchiveUtil.getStoreArchivePath(conf,
-          tableLinkName, hriLink, familyName);
+          tableLinkName, hriLink.getEncodedName(), familyName);
 
     // Create hfile /hbase/table-link/region/cf/getEncodedName.HFILE(conf);
     Path familyPath = getFamilyDirPath(archiveDir, tableName, hri.getEncodedName(), familyName);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Mon Mar 25 11:15:36 2013
@@ -245,8 +245,8 @@ public class TestHRegion extends HBaseTe
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, conf, family);
     try {
-      Path regiondir = region.getRegionDir();
-      FileSystem fs = region.getFilesystem();
+      Path regiondir = region.getRegionFileSystem().getRegionDir();
+      FileSystem fs = region.getRegionFileSystem().getFileSystem();
       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
 
       Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
@@ -257,8 +257,7 @@ public class TestHRegion extends HBaseTe
       for (long i = minSeqId; i <= maxSeqId; i += 10) {
         Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
         fs.create(recoveredEdits);
-        HLog.Writer writer = HLogFactory.createWriter(fs,
-            recoveredEdits, conf);
+        HLog.Writer writer = HLogFactory.createWriter(fs, recoveredEdits, conf);
 
         long time = System.nanoTime();
         WALEdit edit = new WALEdit();
@@ -273,8 +272,7 @@ public class TestHRegion extends HBaseTe
       Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
           Bytes.BYTES_COMPARATOR);
       for (Store store : region.getStores().values()) {
-        maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(),
-            minSeqId - 1);
+        maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1);
       }
       long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
       assertEquals(maxSeqId, seqId);
@@ -297,8 +295,8 @@ public class TestHRegion extends HBaseTe
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, conf, family);
     try {
-      Path regiondir = region.getRegionDir();
-      FileSystem fs = region.getFilesystem();
+      Path regiondir = region.getRegionFileSystem().getRegionDir();
+      FileSystem fs = region.getRegionFileSystem().getFileSystem();
       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
 
       Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
@@ -309,8 +307,7 @@ public class TestHRegion extends HBaseTe
       for (long i = minSeqId; i <= maxSeqId; i += 10) {
         Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
         fs.create(recoveredEdits);
-        HLog.Writer writer = HLogFactory.createWriter(fs,
-            recoveredEdits, conf);
+        HLog.Writer writer = HLogFactory.createWriter(fs, recoveredEdits, conf);
 
         long time = System.nanoTime();
         WALEdit edit = new WALEdit();
@@ -354,13 +351,12 @@ public class TestHRegion extends HBaseTe
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, conf, family);
     try {
-      Path regiondir = region.getRegionDir();
-      FileSystem fs = region.getFilesystem();
+      Path regiondir = region.getRegionFileSystem().getRegionDir();
+      FileSystem fs = region.getRegionFileSystem().getFileSystem();
 
       Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
       for (int i = 1000; i < 1050; i += 10) {
-        Path recoveredEdits = new Path(
-            recoveredEditsDir, String.format("%019d", i));
+        Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
         FSDataOutputStream dos=  fs.create(recoveredEdits);
         dos.writeInt(i);
         dos.close();
@@ -1713,9 +1709,9 @@ public class TestHRegion extends HBaseTe
           openClosedRegion(subregions[i]);
           subregions[i].compactStores();
         }
-        Path oldRegionPath = region.getRegionDir();
-        Path oldRegion1 = subregions[0].getRegionDir();
-        Path oldRegion2 = subregions[1].getRegionDir();
+        Path oldRegionPath = region.getRegionFileSystem().getRegionDir();
+        Path oldRegion1 = subregions[0].getRegionFileSystem().getRegionDir();
+        Path oldRegion2 = subregions[1].getRegionFileSystem().getRegionDir();
         long startTime = System.currentTimeMillis();
         region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
         LOG.info("Merge regions elapsed time: " +
@@ -3491,8 +3487,8 @@ public class TestHRegion extends HBaseTe
 
     // Create a region and skip the initialization (like CreateTableHandler)
     HRegion region = HRegion.createHRegion(hri, rootDir, conf, htd, null, false, true);
-    Path regionDir = region.getRegionDir();
-    FileSystem fs = region.getFilesystem();
+    Path regionDir = region.getRegionFileSystem().getRegionDir();
+    FileSystem fs = region.getRegionFileSystem().getFileSystem();
     HRegion.closeHRegion(region);
 
     Path regionInfoFile = new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE);
@@ -3503,7 +3499,7 @@ public class TestHRegion extends HBaseTe
 
     // Try to open the region
     region = HRegion.openHRegion(rootDir, hri, htd, null, conf);
-    assertEquals(regionDir, region.getRegionDir());
+    assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
     HRegion.closeHRegion(region);
 
     // Verify that the .regioninfo file is still there
@@ -3516,7 +3512,7 @@ public class TestHRegion extends HBaseTe
       fs.exists(regionInfoFile));
 
     region = HRegion.openHRegion(rootDir, hri, htd, null, conf);
-    assertEquals(regionDir, region.getRegionDir());
+    assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
     HRegion.closeHRegion(region);
 
     // Verify that the .regioninfo file is still there

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java Mon Mar 25 11:15:36 2013
@@ -63,7 +63,7 @@ public class TestHRegionFileSystem {
     assertEquals(hri, hriVerify);
 
     // Open the region
-    regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs, rootDir, hri);
+    regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs, rootDir, hri, false);
     assertEquals(regionDir, regionFs.getRegionDir());
 
     // Delete the region

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java Mon Mar 25 11:15:36 2013
@@ -68,13 +68,13 @@ public class TestHRegionInfo {
     assertEquals(modtime, modtime2);
     // Now load the file.
     HRegionInfo deserializedHri = HRegionFileSystem.loadRegionInfoFileContent(
-        FileSystem.get(htu.getConfiguration()), r.getRegionDir());
+        r.getRegionFileSystem().getFileSystem(), r.getRegionFileSystem().getRegionDir());
     assertTrue(hri.equals(deserializedHri));
   }
 
   long getModTime(final HRegion r) throws IOException {
-    FileStatus [] statuses =
-      r.getFilesystem().listStatus(new Path(r.getRegionDir(), HRegionFileSystem.REGION_INFO_FILE));
+    FileStatus[] statuses = r.getRegionFileSystem().getFileSystem().listStatus(
+      new Path(r.getRegionFileSystem().getRegionDir(), HRegionFileSystem.REGION_INFO_FILE));
     assertTrue(statuses != null && statuses.length == 1);
     return statuses[0].getModificationTime();
   }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java Mon Mar 25 11:15:36 2013
@@ -89,9 +89,10 @@ public class TestRegionMergeTransaction 
   public void teardown() throws IOException {
     for (HRegion region : new HRegion[] { region_a, region_b, region_c }) {
       if (region != null && !region.isClosed()) region.close();
-      if (this.fs.exists(region.getRegionDir())
-          && !this.fs.delete(region.getRegionDir(), true)) {
-        throw new IOException("Failed deleting of " + region.getRegionDir());
+      if (this.fs.exists(region.getRegionFileSystem().getRegionDir())
+          && !this.fs.delete(region.getRegionFileSystem().getRegionDir(), true)) {
+        throw new IOException("Failed deleting of "
+            + region.getRegionFileSystem().getRegionDir());
       }
     }
     if (this.wal != null)
@@ -335,7 +336,8 @@ public class TestRegionMergeTransaction 
     // Make sure that merged region is still in the filesystem, that
     // they have not been removed; this is supposed to be the case if we go
     // past point of no return.
-    Path tableDir = this.region_a.getTableDir();
+    Path tableDir = this.region_a.getRegionFileSystem().getRegionDir()
+        .getParent();
     Path mergedRegionDir = new Path(tableDir, mt.getMergedRegionInfo()
         .getEncodedName());
     assertTrue(TEST_UTIL.getTestFileSystem().exists(mergedRegionDir));
@@ -422,3 +424,4 @@ public class TestRegionMergeTransaction 
   }
 
 }
+

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java Mon Mar 25 11:15:36 2013
@@ -162,15 +162,15 @@ public class TestRegionMergeTransactionO
       admin.compact(mergedRegionInfo.getRegionName());
       // wait until merged region doesn't have reference file
       long timeout = System.currentTimeMillis() + waitTime;
+      HRegionFileSystem hrfs = new HRegionFileSystem(
+          TEST_UTIL.getConfiguration(), fs, tabledir, mergedRegionInfo);
       while (System.currentTimeMillis() < timeout) {
-        if (!HRegion.hasReferences(fs, rootDir, mergedRegionInfo,
-            tableDescritor)) {
+        if (!hrfs.hasReferences(tableDescritor)) {
           break;
         }
         Thread.sleep(50);
       }
-      assertFalse(HRegion.hasReferences(fs, rootDir, mergedRegionInfo,
-          tableDescritor));
+      assertFalse(hrfs.hasReferences(tableDescritor));
 
       // run CatalogJanitor to clean merge references in META and archive the
       // files of merging regions
@@ -295,3 +295,4 @@ public class TestRegionMergeTransactionO
   }
 
 }
+

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java Mon Mar 25 11:15:36 2013
@@ -95,9 +95,9 @@ public class TestSplitTransaction {
 
   @After public void teardown() throws IOException {
     if (this.parent != null && !this.parent.isClosed()) this.parent.close();
-    if (this.fs.exists(this.parent.getRegionDir()) &&
-        !this.fs.delete(this.parent.getRegionDir(), true)) {
-      throw new IOException("Failed delete of " + this.parent.getRegionDir());
+    Path regionDir = this.parent.getRegionFileSystem().getRegionDir();
+    if (this.fs.exists(regionDir) && !this.fs.delete(regionDir, true)) {
+      throw new IOException("Failed delete of " + regionDir);
     }
     if (this.wal != null) this.wal.closeAndDelete();
     this.fs.delete(this.testdir, true);
@@ -136,11 +136,9 @@ public class TestSplitTransaction {
     // Make sure that region a and region b are still in the filesystem, that
     // they have not been removed; this is supposed to be the case if we go
     // past point of no return.
-    Path tableDir =  this.parent.getRegionDir().getParent();
-    Path daughterADir =
-      new Path(tableDir, spiedUponSt.getFirstDaughter().getEncodedName());
-    Path daughterBDir =
-      new Path(tableDir, spiedUponSt.getSecondDaughter().getEncodedName());
+    Path tableDir =  this.parent.getRegionFileSystem().getTableDir();
+    Path daughterADir = new Path(tableDir, spiedUponSt.getFirstDaughter().getEncodedName());
+    Path daughterBDir = new Path(tableDir, spiedUponSt.getSecondDaughter().getEncodedName());
     assertTrue(TEST_UTIL.getTestFileSystem().exists(daughterADir));
     assertTrue(TEST_UTIL.getTestFileSystem().exists(daughterBDir));
   }
@@ -154,7 +152,11 @@ public class TestSplitTransaction {
   }
 
   private SplitTransaction prepareGOOD_SPLIT_ROW() {
-    SplitTransaction st = new SplitTransaction(this.parent, GOOD_SPLIT_ROW);
+    return prepareGOOD_SPLIT_ROW(this.parent);
+  }
+
+  private SplitTransaction prepareGOOD_SPLIT_ROW(final HRegion parentRegion) {
+    SplitTransaction st = new SplitTransaction(parentRegion, GOOD_SPLIT_ROW);
     assertTrue(st.prepare());
     return st;
   }
@@ -165,6 +167,7 @@ public class TestSplitTransaction {
   @Test public void testPrepareWithRegionsWithReference() throws IOException {
     HStore storeMock = Mockito.mock(HStore.class);
     when(storeMock.hasReferences()).thenReturn(true);
+    when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
     when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
     this.parent.stores.put(Bytes.toBytes(""), storeMock);
 
@@ -214,13 +217,13 @@ public class TestSplitTransaction {
     when(mockServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration());
     PairOfSameType<HRegion> daughters = st.execute(mockServer, null);
     // Do some assertions about execution.
-    assertTrue(this.fs.exists(st.getSplitDir()));
+    assertTrue(this.fs.exists(this.parent.getRegionFileSystem().getSplitsDir()));
     // Assert the parent region is closed.
     assertTrue(this.parent.isClosed());
 
     // Assert splitdir is empty -- because its content will have been moved out
     // to be under the daughter region dirs.
-    assertEquals(0, this.fs.listStatus(st.getSplitDir()).length);
+    assertEquals(0, this.fs.listStatus(this.parent.getRegionFileSystem().getSplitsDir()).length);
     // Check daughters have correct key span.
     assertTrue(Bytes.equals(this.parent.getStartKey(), daughters.getFirst().getStartKey()));
     assertTrue(Bytes.equals(GOOD_SPLIT_ROW, daughters.getFirst().getEndKey()));
@@ -249,9 +252,10 @@ public class TestSplitTransaction {
     assertEquals(rowcount, parentRowCount);
 
     // Start transaction.
-    SplitTransaction st = prepareGOOD_SPLIT_ROW();
+    HRegion spiedRegion = spy(this.parent);
+    SplitTransaction st = prepareGOOD_SPLIT_ROW(spiedRegion);
     SplitTransaction spiedUponSt = spy(st);
-    when(spiedUponSt.createDaughterRegion(spiedUponSt.getSecondDaughter())).
+    when(spiedRegion.createDaughterRegionFromSplits(spiedUponSt.getSecondDaughter())).
       thenThrow(new MockedFailedDaughterCreation());
     // Run the execute.  Look at what it returns.
     boolean expectedException = false;

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java Mon Mar 25 11:15:36 2013
@@ -68,10 +68,10 @@ public class TestStoreFile extends HBase
   static final Log LOG = LogFactory.getLog(TestStoreFile.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private CacheConfig cacheConf =  new CacheConfig(TEST_UTIL.getConfiguration());
-  private static String ROOT_DIR =
-    TEST_UTIL.getDataTestDir("TestStoreFile").toString();
+  private static String ROOT_DIR = TEST_UTIL.getDataTestDir("TestStoreFile").toString();
   private static final ChecksumType CKTYPE = ChecksumType.CRC32;
   private static final int CKBYTES = 512;
+  private static String TEST_FAMILY = "cf";
 
   @Override
   public void setUp() throws Exception {
@@ -89,16 +89,19 @@ public class TestStoreFile extends HBase
    * @throws Exception
    */
   public void testBasicHalfMapFile() throws Exception {
-    // Make up a directory hierarchy that has a regiondir ("7e0102") and familyname.
-    Path outputDir = new Path(new Path(this.testDir, "7e0102"),
-        "familyname");
-    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf,
-        this.fs, 2 * 1024)
-            .withOutputDir(outputDir)
+    final HRegionInfo hri = new HRegionInfo(Bytes.toBytes("testBasicHalfMapFileTb"));
+    HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
+      conf, fs, new Path(this.testDir, hri.getTableNameAsString()), hri);
+
+    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs, 2 * 1024)
+            .withFilePath(regionFs.createTempName())
             .build();
     writeStoreFile(writer);
-    checkHalfHFile(new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
-        BloomType.NONE, NoOpDataBlockEncoder.INSTANCE));
+
+    Path sfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
+    StoreFile sf = new StoreFile(this.fs, sfPath, conf, cacheConf,
+        BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
+    checkHalfHFile(regionFs, sf);
   }
 
   private void writeStoreFile(final StoreFile.Writer writer) throws IOException {
@@ -134,17 +137,19 @@ public class TestStoreFile extends HBase
    * store files in other regions works.
    * @throws IOException
    */
-  public void testReference()
-  throws IOException {
-    // Make up a directory hierarchy that has a regiondir ("7e0102") and familyname.
-    Path storedir = new Path(new Path(this.testDir, "7e0102"), "familyname");
+  public void testReference() throws IOException {
+    final HRegionInfo hri = new HRegionInfo(Bytes.toBytes("testReferenceTb"));
+    HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
+      conf, fs, new Path(this.testDir, hri.getTableNameAsString()), hri);
+
     // Make a store file and write data to it.
-    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf,
-        this.fs, 8 * 1024)
-            .withOutputDir(storedir)
+    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs, 8 * 1024)
+            .withFilePath(regionFs.createTempName())
             .build();
     writeStoreFile(writer);
-    StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
+
+    Path hsfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
+    StoreFile hsf = new StoreFile(this.fs, hsfPath, conf, cacheConf,
         BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
     StoreFile.Reader reader = hsf.createReader();
     // Split on a row, not in middle of row.  Midkey returned by reader
@@ -155,7 +160,8 @@ public class TestStoreFile extends HBase
     kv = KeyValue.createKeyValueFromKey(reader.getLastKey());
     byte [] finalRow = kv.getRow();
     // Make a reference
-    Path refPath = StoreFile.split(fs, storedir, hsf, midRow, true);
+    HRegionInfo splitHri = new HRegionInfo(hri.getTableName(), null, midRow);
+    Path refPath = splitStoreFile(regionFs, splitHri, TEST_FAMILY, hsf, midRow, true);
     StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
         BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
     // Now confirm that I can read from the reference and that it only gets
@@ -173,26 +179,21 @@ public class TestStoreFile extends HBase
   }
 
   public void testHFileLink() throws IOException {
-    final String columnFamily = "f";
-
-    // force temp data in hbase/target/test-data instead of /tmp/hbase-xxxx/ 
-    Configuration testConf = new Configuration(this.conf); 
-    FSUtils.setRootDir(testConf, this.testDir);  
-
-    HRegionInfo hri = new HRegionInfo(Bytes.toBytes("table-link"));
-    Path storedir = new Path(new Path(this.testDir,
-        new Path(hri.getTableNameAsString(), hri.getEncodedName())), columnFamily);
+    final HRegionInfo hri = new HRegionInfo(Bytes.toBytes("testHFileLinkTb"));
+    // force temp data in hbase/target/test-data instead of /tmp/hbase-xxxx/
+    Configuration testConf = new Configuration(this.conf);
+    FSUtils.setRootDir(testConf, this.testDir);
+    HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
+      testConf, fs, new Path(this.testDir, hri.getTableNameAsString()), hri);
 
     // Make a store file and write data to it.
-    StoreFile.Writer writer = new StoreFile.WriterBuilder(testConf, cacheConf,
-         this.fs, 8 * 1024)
-            .withOutputDir(storedir)
+    StoreFile.Writer writer = new StoreFile.WriterBuilder(testConf, cacheConf, this.fs, 8 * 1024)
+            .withFilePath(regionFs.createTempName())
             .build();
-    Path storeFilePath = writer.getPath();
     writeStoreFile(writer);
-    writer.close();
 
-    Path dstPath = new Path(this.testDir, new Path("test-region", columnFamily));
+    Path storeFilePath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
+    Path dstPath = new Path(regionFs.getTableDir(), new Path("test-region", TEST_FAMILY));
     HFileLink.create(testConf, this.fs, dstPath, hri, storeFilePath.getName());
     Path linkFilePath = new Path(dstPath,
                   HFileLink.createHFileLinkName(hri, storeFilePath.getName()));
@@ -218,53 +219,49 @@ public class TestStoreFile extends HBase
    * to hfilelinks (created by snapshot clones) can be properly interpreted.
    */
   public void testReferenceToHFileLink() throws IOException {
-    final String columnFamily = "f";
-
-    Path rootDir = FSUtils.getRootDir(conf);
-
-    String tablename = "_original-evil-name"; // adding legal table name chars to verify regex handles it.
-    HRegionInfo hri = new HRegionInfo(Bytes.toBytes(tablename));
-    // store dir = <root>/<tablename>/<rgn>/<cf>
-    Path storedir = new Path(new Path(rootDir,
-      new Path(hri.getTableNameAsString(), hri.getEncodedName())), columnFamily);
+    // force temp data in hbase/target/test-data instead of /tmp/hbase-xxxx/
+    Configuration testConf = new Configuration(this.conf);
+    FSUtils.setRootDir(testConf, this.testDir);
+
+    // adding legal table name chars to verify regex handles it.
+    HRegionInfo hri = new HRegionInfo(Bytes.toBytes("_original-evil-name"));
+    HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
+      testConf, fs, new Path(this.testDir, hri.getTableNameAsString()), hri);
 
     // Make a store file and write data to it. <root>/<tablename>/<rgn>/<cf>/<file>
-    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf,
-         this.fs, 8 * 1024)
-            .withOutputDir(storedir)
+    StoreFile.Writer writer = new StoreFile.WriterBuilder(testConf, cacheConf, this.fs, 8 * 1024)
+            .withFilePath(regionFs.createTempName())
             .build();
-    Path storeFilePath = writer.getPath();
     writeStoreFile(writer);
-    writer.close();
+    Path storeFilePath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
 
     // create link to store file. <root>/clone/region/<cf>/<hfile>-<region>-<table>
-    String target = "clone";
-    Path dstPath = new Path(rootDir, new Path(new Path(target, "7e0102"), columnFamily));
-    HFileLink.create(conf, this.fs, dstPath, hri, storeFilePath.getName());
+    HRegionInfo hriClone = new HRegionInfo(Bytes.toBytes("clone"));
+    HRegionFileSystem cloneRegionFs = HRegionFileSystem.createRegionOnFileSystem(
+      testConf, fs, new Path(this.testDir, hri.getTableNameAsString()), hriClone);
+    Path dstPath = cloneRegionFs.getStoreDir(TEST_FAMILY);
+    HFileLink.create(testConf, this.fs, dstPath, hri, storeFilePath.getName());
     Path linkFilePath = new Path(dstPath,
                   HFileLink.createHFileLinkName(hri, storeFilePath.getName()));
 
     // create splits of the link.
     // <root>/clone/splitA/<cf>/<reftohfilelink>,
     // <root>/clone/splitB/<cf>/<reftohfilelink>
-    Path splitDirA = new Path(new Path(rootDir,
-        new Path(target, "571A")), columnFamily);
-    Path splitDirB = new Path(new Path(rootDir,
-        new Path(target, "571B")), columnFamily);
-    StoreFile f = new StoreFile(fs, linkFilePath, conf, cacheConf, BloomType.NONE,
+    HRegionInfo splitHriA = new HRegionInfo(hri.getTableName(), null, SPLITKEY);
+    HRegionInfo splitHriB = new HRegionInfo(hri.getTableName(), SPLITKEY, null);
+    StoreFile f = new StoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE,
         NoOpDataBlockEncoder.INSTANCE);
-    byte[] splitRow = SPLITKEY;
-    Path pathA = StoreFile.split(fs, splitDirA, f, splitRow, true); // top
-    Path pathB = StoreFile.split(fs, splitDirB, f, splitRow, false); // bottom
+    Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top
+    Path pathB = splitStoreFile(cloneRegionFs, splitHriB, TEST_FAMILY, f, SPLITKEY, false);// bottom
 
     // OK test the thing
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    FSUtils.logFileSystemState(fs, this.testDir, LOG);
 
     // There is a case where a file with the hfilelink pattern is actually a daughter
     // reference to a hfile link.  This code in StoreFile that handles this case.
-    
+
     // Try to open store file from link
-    StoreFile hsfA = new StoreFile(this.fs, pathA,  conf, cacheConf,
+    StoreFile hsfA = new StoreFile(this.fs, pathA, testConf, cacheConf,
         BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
 
     // Now confirm that I can read from the ref to link
@@ -275,9 +272,9 @@ public class TestStoreFile extends HBase
       count++;
     }
     assertTrue(count > 0); // read some rows here
-    
+
     // Try to open store file from link
-    StoreFile hsfB = new StoreFile(this.fs, pathB,  conf, cacheConf,
+    StoreFile hsfB = new StoreFile(this.fs, pathB, testConf, cacheConf,
         BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
 
     // Now confirm that I can read from the ref to link
@@ -291,31 +288,21 @@ public class TestStoreFile extends HBase
     assertEquals((LAST_CHAR - FIRST_CHAR + 1) * (LAST_CHAR - FIRST_CHAR + 1), count);
   }
 
-  private void checkHalfHFile(final StoreFile f)
-  throws IOException {
+  private void checkHalfHFile(final HRegionFileSystem regionFs, final StoreFile f)
+      throws IOException {
     byte [] midkey = f.createReader().midkey();
     KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
     byte [] midRow = midKV.getRow();
     // Create top split.
-    Path topDir = HStore.getStoreHomedir(this.testDir, "1",
-      Bytes.toBytes(f.getPath().getParent().getName()));
-    if (this.fs.exists(topDir)) {
-      this.fs.delete(topDir, true);
-    }
-    Path topPath = StoreFile.split(this.fs, topDir, f, midRow, true);
+    HRegionInfo topHri = new HRegionInfo(regionFs.getRegionInfo().getTableName(), null, midRow);
+    Path topPath = splitStoreFile(regionFs, topHri, TEST_FAMILY, f, midRow, true);
     // Create bottom split.
-    Path bottomDir = HStore.getStoreHomedir(this.testDir, "2",
-      Bytes.toBytes(f.getPath().getParent().getName()));
-    if (this.fs.exists(bottomDir)) {
-      this.fs.delete(bottomDir, true);
-    }
-    Path bottomPath = StoreFile.split(this.fs, bottomDir, f, midRow, false);
+    HRegionInfo bottomHri = new HRegionInfo(regionFs.getRegionInfo().getTableName(), midRow, null);
+    Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false);
     // Make readers on top and bottom.
-    StoreFile.Reader top =
-        new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE,
-            NoOpDataBlockEncoder.INSTANCE).createReader();
-    StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath,
-        conf, cacheConf, BloomType.NONE,
+    StoreFile.Reader top = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE,
+        NoOpDataBlockEncoder.INSTANCE).createReader();
+    StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE,
         NoOpDataBlockEncoder.INSTANCE).createReader();
     ByteBuffer previous = null;
     LOG.info("Midkey: " + midKV.toString());
@@ -329,7 +316,7 @@ public class TestStoreFile extends HBase
       ByteBuffer key = null;
       HFileScanner topScanner = top.getScanner(false, false);
       while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
-          (topScanner.isSeeked() && topScanner.next())) {
+             (topScanner.isSeeked() && topScanner.next())) {
         key = topScanner.getKey();
 
         if (topScanner.getReader().getComparator().compare(key.array(),
@@ -361,20 +348,19 @@ public class TestStoreFile extends HBase
         LOG.info("Last in bottom: " + Bytes.toString(Bytes.toBytes(previous)));
       }
       // Remove references.
-      this.fs.delete(topPath, false);
-      this.fs.delete(bottomPath, false);
+      regionFs.cleanupDaughterRegion(topHri);
+      regionFs.cleanupDaughterRegion(bottomHri);
 
       // Next test using a midkey that does not exist in the file.
       // First, do a key that is < than first key. Ensure splits behave
       // properly.
       byte [] badmidkey = Bytes.toBytes("  .");
-      topPath = StoreFile.split(this.fs, topDir, f, badmidkey, true);
-      bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey, false);
-      top = new StoreFile(this.fs, topPath, conf, cacheConf,
-          BloomType.NONE,
+      assertTrue(fs.exists(f.getPath()));
+      topPath = splitStoreFile(regionFs, topHri, TEST_FAMILY, f, badmidkey, true);
+      bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, badmidkey, false);
+      top = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE,
           NoOpDataBlockEncoder.INSTANCE).createReader();
-      bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
-          BloomType.NONE,
+      bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE,
           NoOpDataBlockEncoder.INSTANCE).createReader();
       bottomScanner = bottom.getScanner(false, false);
       int count = 0;
@@ -409,18 +395,16 @@ public class TestStoreFile extends HBase
         assertTrue(tmp.charAt(i) == 'z');
       }
       // Remove references.
-      this.fs.delete(topPath, false);
-      this.fs.delete(bottomPath, false);
+      regionFs.cleanupDaughterRegion(topHri);
+      regionFs.cleanupDaughterRegion(bottomHri);
 
       // Test when badkey is > than last key in file ('||' > 'zz').
       badmidkey = Bytes.toBytes("|||");
-      topPath = StoreFile.split(this.fs, topDir, f, badmidkey, true);
-      bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey, false);
-      top = new StoreFile(this.fs, topPath, conf, cacheConf,
-          BloomType.NONE,
+      topPath = splitStoreFile(regionFs,topHri, TEST_FAMILY, f, badmidkey, true);
+      bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, badmidkey, false);
+      top = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE,
           NoOpDataBlockEncoder.INSTANCE).createReader();
-      bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
-          BloomType.NONE,
+      bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE,
           NoOpDataBlockEncoder.INSTANCE).createReader();
       first = true;
       bottomScanner = bottom.getScanner(false, false);
@@ -463,10 +447,8 @@ public class TestStoreFile extends HBase
 
   private static final String localFormatter = "%010d";
 
-  private void bloomWriteRead(StoreFile.Writer writer, FileSystem fs)
-  throws Exception {
-    float err = conf.getFloat(
-        BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0);
+  private void bloomWriteRead(StoreFile.Writer writer, FileSystem fs) throws Exception {
+    float err = conf.getFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0);
     Path f = writer.getPath();
     long now = System.currentTimeMillis();
     for (int i = 0; i < 2000; i += 2) {
@@ -511,8 +493,7 @@ public class TestStoreFile extends HBase
 
   public void testBloomFilter() throws Exception {
     FileSystem fs = FileSystem.getLocal(conf);
-    conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
-        (float) 0.01);
+    conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, (float) 0.01);
     conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
 
     // write the file
@@ -530,11 +511,9 @@ public class TestStoreFile extends HBase
 
   public void testDeleteFamilyBloomFilter() throws Exception {
     FileSystem fs = FileSystem.getLocal(conf);
-    conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
-        (float) 0.01);
+    conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, (float) 0.01);
     conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
-    float err = conf.getFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
-        0);
+    float err = conf.getFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0);
 
     // write the file
     Path f = new Path(ROOT_DIR, getName());
@@ -624,9 +603,8 @@ public class TestStoreFile extends HBase
     int versions = 2;
 
     // run once using columns and once using rows
-    BloomType[] bt =
-      {BloomType.ROWCOL, BloomType.ROW};
-    int[] expKeys    = {rowCount*colCount, rowCount};
+    BloomType[] bt = {BloomType.ROWCOL, BloomType.ROW};
+    int[] expKeys  = {rowCount*colCount, rowCount};
     // below line deserves commentary.  it is expected bloom false positives
     //  column = rowCount*2*colCount inserts
     //  row-level = only rowCount*2 inserts, but failures will be magnified by
@@ -946,6 +924,15 @@ public class TestStoreFile extends HBase
     assertEquals(startEvicted, cs.getEvictedCount());
   }
 
+  private Path splitStoreFile(final HRegionFileSystem regionFs, final HRegionInfo hri,
+      final String family, final StoreFile sf, final byte[] splitKey, boolean isTopRef)
+      throws IOException {
+    FileSystem fs = regionFs.getFileSystem();
+    Path path = regionFs.splitStoreFile(hri, family, sf, splitKey, isTopRef);
+    Path regionDir = regionFs.commitDaughterRegion(hri);
+    return new Path(new Path(regionDir, family), path.getName());
+  }
+
   private StoreFile.Writer writeStoreFile(Configuration conf,
       CacheConfig cacheConf, Path path, int numBlocks)
   throws IOException {
@@ -961,8 +948,7 @@ public class TestStoreFile extends HBase
       totalSize += kv.getLength() + 1;
     }
     int blockSize = totalSize / numBlocks;
-    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs,
-        blockSize)
+    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs, blockSize)
             .withFilePath(path)
             .withMaxKeyCount(2000)
             .withChecksumType(CKTYPE)
@@ -1003,16 +989,14 @@ public class TestStoreFile extends HBase
             .withBytesPerChecksum(CKBYTES)
             .build();
     writer.close();
-    
+
     StoreFile storeFile = new StoreFile(fs, writer.getPath(), conf,
         cacheConf, BloomType.NONE, dataBlockEncoder);
     StoreFile.Reader reader = storeFile.createReader();
-    
+
     Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
     byte[] value = fileInfo.get(HFileDataBlockEncoder.DATA_BLOCK_ENCODING);
-
     assertEquals(dataBlockEncoderAlgo.getNameInBytes(), value);
   }
-
 }
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Mon Mar 25 11:15:36 2013
@@ -530,8 +530,7 @@ public class TestWALReplay {
     for (HColumnDescriptor hcd: htd.getFamilies()) {
       cf_count++;
       if (cf_count == 2) {
-        this.fs.delete(new Path(region.getRegionDir(), Bytes.toString(hcd.getName()))
-            , true);
+        region.getRegionFileSystem().deleteFamily(hcd.getNameAsString());
       }
     }
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java Mon Mar 25 11:15:36 2013
@@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.client.HC
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
@@ -156,19 +156,21 @@ public class TestRestoreSnapshotHelper {
   private void createSnapshot(final Path rootDir, final Path snapshotDir, final HTableDescriptor htd)
       throws IOException {
     // First region, simple with one plain hfile.
-    HRegion r0 = HRegion.createHRegion(new HRegionInfo(htd.getName()), archiveDir,
-        conf, htd, null, true, true);
-    Path storeFile = new Path(new Path(r0.getRegionDir(), TEST_FAMILY), TEST_HFILE);
+    HRegionInfo hri = new HRegionInfo(htd.getName());
+    HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf,
+      fs, new Path(archiveDir, hri.getTableNameAsString()), hri);
+    Path storeFile = new Path(rootDir, TEST_HFILE);
     fs.createNewFile(storeFile);
-    r0.close();
+    r0fs.commitStoreFile(TEST_FAMILY, storeFile);
 
     // Second region, used to test the split case.
     // This region contains a reference to the hfile in the first region.
-    HRegion r1 = HRegion.createHRegion(new HRegionInfo(htd.getName()), archiveDir,
-        conf, htd, null, true, true);
-    fs.createNewFile(new Path(new Path(r1.getRegionDir(), TEST_FAMILY),
-        storeFile.getName() + '.' + r0.getRegionInfo().getEncodedName()));
-    r1.close();
+    hri = new HRegionInfo(htd.getName());
+    HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf,
+      fs, new Path(archiveDir, hri.getTableNameAsString()), hri);
+    storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
+    fs.createNewFile(storeFile);
+    r1fs.commitStoreFile(TEST_FAMILY, storeFile);
 
     Path tableDir = HTableDescriptor.getTableDir(archiveDir, htd.getName());
     FileUtil.copy(fs, tableDir, fs, snapshotDir, false, conf);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java Mon Mar 25 11:15:36 2013
@@ -213,7 +213,8 @@ public class HFileArchiveTestingUtil {
    * @return {@link Path} to the archive directory for the given region
    */
   public static Path getRegionArchiveDir(Configuration conf, HRegion region) {
-    return HFileArchiveUtil.getRegionArchiveDir(conf, region.getTableDir(), region.getRegionDir());
+    return HFileArchiveUtil.getRegionArchiveDir(region.getRegionFileSystem().getTableDir(),
+        region.getRegionFileSystem().getRegionDir());
   }
 
   /**
@@ -225,9 +226,8 @@ public class HFileArchiveTestingUtil {
    */
   public static Path getStoreArchivePath(Configuration conf, HRegion region, Store store)
       throws IOException {
-    HRegionInfo hri = region.getRegionInfo();
-    return HFileArchiveUtil.getStoreArchivePath(conf, hri.getTableNameAsString(), hri,
-        store.getFamily().getNameAsString());
+    return HFileArchiveUtil.getStoreArchivePath(conf, region.getRegionInfo(),
+        region.getRegionFileSystem().getTableDir(), store.getFamily().getName());
   }
 
   public static Path getStoreArchivePath(HBaseTestingUtility util, String tableName,

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java?rev=1460617&r1=1460616&r2=1460617&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java Mon Mar 25 11:15:36 2013
@@ -50,10 +50,9 @@ public class TestHFileArchiveUtil {
   
   @Test
   public void testRegionArchiveDir() {
-    Configuration conf = null;
     Path tableDir = new Path("table");
     Path regionDir = new Path("region");
-    assertNotNull(HFileArchiveUtil.getRegionArchiveDir(conf, tableDir, regionDir));
+    assertNotNull(HFileArchiveUtil.getRegionArchiveDir(tableDir, regionDir));
   }
   
   @Test