Browse Source

HADOOP-2478 Restructure how HBase lays out files in the file system (phase 2) Includes migration tool org.apache.hadoop.hbase.util.Migrate

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@611519 13f79535-47bb-0310-9956-ffa450edef68
Jim Kellerman 17 years ago
parent
commit
2b9450f335
37 changed files with 1871 additions and 1254 deletions
  1. 2 0
      src/contrib/hbase/CHANGES.txt
  2. 16 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
  3. 0 3
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
  4. 89 84
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java
  5. 30 30
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
  6. 92 94
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java
  7. 71 81
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
  8. 11 4
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java
  9. 17 12
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
  10. 248 197
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
  11. 291 482
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
  12. 10 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
  13. 437 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Migrate.java
  14. 7 0
      src/contrib/hbase/src/test/hbase-site.xml
  15. 32 26
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java
  16. 1 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java
  17. 185 46
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
  18. 3 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java
  19. 4 6
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/MultiRegionTable.java
  20. 13 7
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompaction.java
  21. 12 16
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteAll.java
  22. 8 10
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteFamily.java
  23. 14 42
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java
  24. 8 14
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet2.java
  25. 3 10
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java
  26. 14 13
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
  27. 6 9
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java
  28. 6 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestInfoServers.java
  29. 1 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestLogRolling.java
  30. 1 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeMeta.java
  31. 2 2
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeTable.java
  32. 14 28
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
  33. 8 11
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
  34. 25 3
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
  35. 11 9
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java
  36. 11 10
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
  37. 168 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/util/TestMigrate.java

+ 2 - 0
src/contrib/hbase/CHANGES.txt

@@ -10,6 +10,8 @@ Trunk (unreleased changes)
    HADOOP-2519 Performance improvements: Customized RPC serialization
    HADOOP-2519 Performance improvements: Customized RPC serialization
    HADOOP-2478 Restructure how HBase lays out files in the file system (phase 1)
    HADOOP-2478 Restructure how HBase lays out files in the file system (phase 1)
                (test input data)
                (test input data)
+   HADOOP-2478 Restructure how HBase lays out files in the file system (phase 2)
+               Includes migration tool org.apache.hadoop.hbase.util.Migrate
 
 
   NEW FEATURES
   NEW FEATURES
     HADOOP-2061 Add new Base64 dialects
     HADOOP-2061 Add new Base64 dialects

+ 16 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java

@@ -28,6 +28,8 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparable;
 
 
+import org.apache.hadoop.hbase.io.TextSequence;
+
 /**
 /**
  * An HColumnDescriptor contains information about a column family such as the
  * An HColumnDescriptor contains information about a column family such as the
  * number of versions, compression settings, etc.
  * number of versions, compression settings, etc.
@@ -101,6 +103,8 @@ public class HColumnDescriptor implements WritableComparable {
   private BloomFilterDescriptor bloomFilter;
   private BloomFilterDescriptor bloomFilter;
   // Version number of this class
   // Version number of this class
   private byte versionNumber;
   private byte versionNumber;
+  // Family name without the ':'
+  private transient Text familyName = null;
   
   
   /**
   /**
    * Default constructor. Must be present for Writable.
    * Default constructor. Must be present for Writable.
@@ -172,6 +176,17 @@ public class HColumnDescriptor implements WritableComparable {
   public Text getName() {
   public Text getName() {
     return name;
     return name;
   }
   }
+
+  /** @return name of column family without trailing ':' */
+  public synchronized Text getFamilyName() {
+    if (name != null) {
+      if (familyName == null) {
+        familyName = new TextSequence(name, 0, name.getLength() - 1).toText();
+      }
+      return familyName;
+    }
+    return null;
+  }
   
   
   /** @return compression type being used for the column family */
   /** @return compression type being used for the column family */
   public CompressionType getCompression() {
   public CompressionType getCompression() {
@@ -211,6 +226,7 @@ public class HColumnDescriptor implements WritableComparable {
     return this.bloomFilter;
     return this.bloomFilter;
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
     // Output a name minus ':'.
     // Output a name minus ':'.

+ 0 - 3
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java

@@ -78,9 +78,6 @@ public interface HConstants {
   /** Default HBase instance root directory */
   /** Default HBase instance root directory */
   static final String DEFAULT_HBASE_DIR = "/hbase";
   static final String DEFAULT_HBASE_DIR = "/hbase";
   
   
-  /** Used to construct the name of the directory in which a HRegion resides */
-  static final String HREGIONDIR_PREFIX = "hregion_";
-  
   /** Used to construct the name of the log directory for a region server */
   /** Used to construct the name of the log directory for a region server */
   static final String HREGION_LOGDIR_NAME = "log";
   static final String HREGION_LOGDIR_NAME = "log";
 
 

+ 89 - 84
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLog.java

@@ -130,88 +130,6 @@ public class HLog implements HConstants {
   // during an update
   // during an update
   private final Integer updateLock = new Integer(0);
   private final Integer updateLock = new Integer(0);
 
 
-  /**
-   * Split up a bunch of log files, that are no longer being written to, into
-   * new files, one per region. Delete the old log files when finished.
-   *
-   * @param rootDir Root directory of the HBase instance
-   * @param srcDir Directory of log files to split: e.g.
-   *                <code>${ROOTDIR}/log_HOST_PORT</code>
-   * @param fs FileSystem
-   * @param conf HBaseConfiguration
-   * @throws IOException
-   */
-  static void splitLog(Path rootDir, Path srcDir, FileSystem fs,
-    Configuration conf)
-  throws IOException {
-    Path logfiles[] = fs.listPaths(new Path[] { srcDir });
-    LOG.info("splitting " + logfiles.length + " log(s) in " +
-      srcDir.toString());
-    Map<Text, SequenceFile.Writer> logWriters =
-      new HashMap<Text, SequenceFile.Writer>();
-    try {
-      for (int i = 0; i < logfiles.length; i++) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Splitting " + i + " of " + logfiles.length + ": " +
-            logfiles[i]);
-        }
-        // Check for empty file.
-        if (fs.getFileStatus(logfiles[i]).getLen() <= 0) {
-          LOG.info("Skipping " + logfiles[i].toString() +
-            " because zero length");
-          continue;
-        }
-        HLogKey key = new HLogKey();
-        HLogEdit val = new HLogEdit();
-        SequenceFile.Reader in = new SequenceFile.Reader(fs, logfiles[i], conf);
-        try {
-          int count = 0;
-          for (; in.next(key, val); count++) {
-            Text regionName = key.getRegionName();
-            SequenceFile.Writer w = logWriters.get(regionName);
-            if (w == null) {
-              Path logfile = new Path(HRegion.getRegionDir(rootDir,
-                HRegionInfo.encodeRegionName(regionName)),
-                HREGION_OLDLOGFILE_NAME);
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Creating new log file writer for path " + logfile +
-                  "; map content " + logWriters.toString());
-              }
-              w = SequenceFile.createWriter(fs, conf, logfile, HLogKey.class,
-                HLogEdit.class, getCompressionType(conf));
-              // Use copy of regionName; regionName object is reused inside in
-              // HStoreKey.getRegionName so its content changes as we iterate.
-              logWriters.put(new Text(regionName), w);
-            }
-            if (count % 10000 == 0 && count > 0 && LOG.isDebugEnabled()) {
-              LOG.debug("Applied " + count + " edits");
-            }
-            w.append(key, val);
-          }
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Applied " + count + " total edits");
-          }
-        } finally {
-          in.close();
-        }
-      }
-    } finally {
-      for (SequenceFile.Writer w : logWriters.values()) {
-        w.close();
-      }
-    }
-
-    if (fs.exists(srcDir)) {
-      if (!fs.delete(srcDir)) {
-        LOG.error("Cannot delete: " + srcDir);
-        if (!FileUtil.fullyDelete(new File(srcDir.toString()))) {
-          throw new IOException("Cannot delete: " + srcDir);
-        }
-      }
-    }
-    LOG.info("log file splitting completed for " + srcDir.toString());
-  }
-
   /**
   /**
    * Create an edit log at the given <code>dir</code> location.
    * Create an edit log at the given <code>dir</code> location.
    *
    *
@@ -222,9 +140,10 @@ public class HLog implements HConstants {
    * @param fs
    * @param fs
    * @param dir
    * @param dir
    * @param conf
    * @param conf
+   * @param listener
    * @throws IOException
    * @throws IOException
    */
    */
-  HLog(final FileSystem fs, final Path dir, final Configuration conf,
+  public HLog(final FileSystem fs, final Path dir, final Configuration conf,
       final LogRollListener listener) throws IOException {
       final LogRollListener listener) throws IOException {
     this.fs = fs;
     this.fs = fs;
     this.dir = dir;
     this.dir = dir;
@@ -381,7 +300,7 @@ public class HLog implements HConstants {
    *
    *
    * @throws IOException
    * @throws IOException
    */
    */
-  void closeAndDelete() throws IOException {
+  public void closeAndDelete() throws IOException {
     close();
     close();
     fs.delete(dir);
     fs.delete(dir);
   }
   }
@@ -559,6 +478,92 @@ public class HLog implements HConstants {
     this.cacheFlushLock.unlock();
     this.cacheFlushLock.unlock();
   }
   }
 
 
+  /**
+   * Split up a bunch of log files, that are no longer being written to, into
+   * new files, one per region. Delete the old log files when finished.
+   *
+   * @param rootDir qualified root directory of the HBase instance
+   * @param srcDir Directory of log files to split: e.g.
+   *                <code>${ROOTDIR}/log_HOST_PORT</code>
+   * @param fs FileSystem
+   * @param conf HBaseConfiguration
+   * @throws IOException
+   */
+  static void splitLog(Path rootDir, Path srcDir, FileSystem fs,
+    Configuration conf) throws IOException {
+    Path logfiles[] = fs.listPaths(new Path[] { srcDir });
+    LOG.info("splitting " + logfiles.length + " log(s) in " +
+      srcDir.toString());
+    Map<Text, SequenceFile.Writer> logWriters =
+      new HashMap<Text, SequenceFile.Writer>();
+    try {
+      for (int i = 0; i < logfiles.length; i++) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Splitting " + i + " of " + logfiles.length + ": " +
+            logfiles[i]);
+        }
+        // Check for empty file.
+        if (fs.getFileStatus(logfiles[i]).getLen() <= 0) {
+          LOG.info("Skipping " + logfiles[i].toString() +
+            " because zero length");
+          continue;
+        }
+        HLogKey key = new HLogKey();
+        HLogEdit val = new HLogEdit();
+        SequenceFile.Reader in = new SequenceFile.Reader(fs, logfiles[i], conf);
+        try {
+          int count = 0;
+          for (; in.next(key, val); count++) {
+            Text tableName = key.getTablename();
+            Text regionName = key.getRegionName();
+            SequenceFile.Writer w = logWriters.get(regionName);
+            if (w == null) {
+              Path logfile = new Path(
+                  HRegion.getRegionDir(
+                      HTableDescriptor.getTableDir(rootDir, tableName),
+                      HRegionInfo.encodeRegionName(regionName)
+                  ),
+                  HREGION_OLDLOGFILE_NAME
+              );
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Creating new log file writer for path " + logfile +
+                  "; map content " + logWriters.toString());
+              }
+              w = SequenceFile.createWriter(fs, conf, logfile, HLogKey.class,
+                HLogEdit.class, getCompressionType(conf));
+              // Use copy of regionName; regionName object is reused inside in
+              // HStoreKey.getRegionName so its content changes as we iterate.
+              logWriters.put(new Text(regionName), w);
+            }
+            if (count % 10000 == 0 && count > 0 && LOG.isDebugEnabled()) {
+              LOG.debug("Applied " + count + " edits");
+            }
+            w.append(key, val);
+          }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Applied " + count + " total edits");
+          }
+        } finally {
+          in.close();
+        }
+      }
+    } finally {
+      for (SequenceFile.Writer w : logWriters.values()) {
+        w.close();
+      }
+    }
+
+    if (fs.exists(srcDir)) {
+      if (!fs.delete(srcDir)) {
+        LOG.error("Cannot delete: " + srcDir);
+        if (!FileUtil.fullyDelete(new File(srcDir.toString()))) {
+          throw new IOException("Cannot delete: " + srcDir);
+        }
+      }
+    }
+    LOG.info("log file splitting completed for " + srcDir.toString());
+  }
+
   private static void usage() {
   private static void usage() {
     System.err.println("Usage: java org.apache.hbase.HLog" +
     System.err.println("Usage: java org.apache.hbase.HLog" +
         " {--dump <logfile>... | --split <logdir>...}");
         " {--dump <logfile>... | --split <logdir>...}");

+ 30 - 30
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java

@@ -95,7 +95,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
   volatile boolean shutdownRequested = false;
   volatile boolean shutdownRequested = false;
   volatile AtomicInteger quiescedMetaServers = new AtomicInteger(0);
   volatile AtomicInteger quiescedMetaServers = new AtomicInteger(0);
   volatile boolean fsOk = true;
   volatile boolean fsOk = true;
-  final Path dir;
+  final Path rootdir;
   final HBaseConfiguration conf;
   final HBaseConfiguration conf;
   final FileSystem fs;
   final FileSystem fs;
   final Random rand;
   final Random rand;
@@ -333,8 +333,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
       if (!hasReferencesA && !hasReferencesB) {
       if (!hasReferencesA && !hasReferencesB) {
         LOG.info("Deleting region " + parent.getRegionName() +
         LOG.info("Deleting region " + parent.getRegionName() +
         " because daughter splits no longer hold references");
         " because daughter splits no longer hold references");
-        if (!HRegion.deleteRegion(fs, dir,
-            HRegionInfo.encodeRegionName(parent.getRegionName()))) {
+        if (!HRegion.deleteRegion(fs, rootdir, parent)) {
           LOG.warn("Deletion of " + parent.getRegionName() + " failed");
           LOG.warn("Deletion of " + parent.getRegionName() + " failed");
         }
         }
         
         
@@ -378,10 +377,11 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
       if (split == null) {
       if (split == null) {
         return result;
         return result;
       }
       }
-      for (Text family: split.getTableDesc().families().keySet()) {
-        Path p = HStoreFile.getMapDir(fs.makeQualified(dir),
-            HRegionInfo.encodeRegionName(split.getRegionName()),
-            HStoreKey.extractFamily(family));
+      Path tabledir =
+        HTableDescriptor.getTableDir(rootdir, split.getTableDesc().getName());
+      for (HColumnDescriptor family: split.getTableDesc().families().values()) {
+        Path p = HStoreFile.getMapDir(tabledir, split.getEncodedName(),
+            family.getFamilyName());
 
 
         // Look for reference files.  Call listPaths with an anonymous
         // Look for reference files.  Call listPaths with an anonymous
         // instance of PathFilter.
         // instance of PathFilter.
@@ -389,7 +389,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
         Path [] ps = fs.listPaths(p,
         Path [] ps = fs.listPaths(p,
             new PathFilter () {
             new PathFilter () {
               public boolean accept(Path path) {
               public boolean accept(Path path) {
-                return HStoreFile.isReference(path);
+                return HStore.isReference(path);
               }
               }
             }
             }
         );
         );
@@ -481,12 +481,12 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
         if (!initialMetaScanComplete && serverName.length() != 0) {
         if (!initialMetaScanComplete && serverName.length() != 0) {
           StringBuilder dirName = new StringBuilder("log_");
           StringBuilder dirName = new StringBuilder("log_");
           dirName.append(serverName.replace(":", "_"));
           dirName.append(serverName.replace(":", "_"));
-          Path logDir = new Path(dir, dirName.toString());
+          Path logDir = new Path(rootdir, dirName.toString());
           try {
           try {
             if (fs.exists(logDir)) {
             if (fs.exists(logDir)) {
               splitLogLock.lock();
               splitLogLock.lock();
               try {
               try {
-                HLog.splitLog(dir, logDir, fs, conf);
+                HLog.splitLog(rootdir, logDir, fs, conf);
               } finally {
               } finally {
                 splitLogLock.unlock();
                 splitLogLock.unlock();
               }
               }
@@ -866,37 +866,38 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
 
 
   /** 
   /** 
    * Build the HMaster
    * Build the HMaster
-   * @param dir base directory
+   * @param rootdir base directory of this HBase instance
    * @param address server address and port number
    * @param address server address and port number
    * @param conf configuration
    * @param conf configuration
    * 
    * 
    * @throws IOException
    * @throws IOException
    */
    */
-  public HMaster(Path dir, HServerAddress address, HBaseConfiguration conf)
+  public HMaster(Path rootdir, HServerAddress address, HBaseConfiguration conf)
     throws IOException {
     throws IOException {
     
     
-    this.dir = dir;
     this.conf = conf;
     this.conf = conf;
     this.fs = FileSystem.get(conf);
     this.fs = FileSystem.get(conf);
+    this.rootdir = fs.makeQualified(rootdir);
+    this.conf.set(HConstants.HBASE_DIR, this.rootdir.toString());
     this.rand = new Random();
     this.rand = new Random();
     
     
-    Path rootRegionDir = HRegion.getRegionDir(dir,
-        HRegionInfo.encodeRegionName(HRegionInfo.rootRegionInfo.getRegionName()));
+    Path rootRegionDir =
+      HRegion.getRegionDir(rootdir, HRegionInfo.rootRegionInfo);
     LOG.info("Root region dir: " + rootRegionDir.toString());
     LOG.info("Root region dir: " + rootRegionDir.toString());
 
 
     try {
     try {
       // Make sure the root directory exists!
       // Make sure the root directory exists!
-      if(! fs.exists(dir)) {
-        fs.mkdirs(dir);
+      if(! fs.exists(rootdir)) {
+        fs.mkdirs(rootdir);
       }
       }
 
 
       if (!fs.exists(rootRegionDir)) {
       if (!fs.exists(rootRegionDir)) {
         LOG.info("bootstrap: creating ROOT and first META regions");
         LOG.info("bootstrap: creating ROOT and first META regions");
         try {
         try {
           HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo,
           HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo,
-              this.dir, this.conf);
+              this.rootdir, this.conf);
           HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo,
           HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo,
-            this.dir, this.conf);
+            this.rootdir, this.conf);
 
 
           // Add first region from the META table to the ROOT region.
           // Add first region from the META table to the ROOT region.
           HRegion.addRegionToMETA(root, meta);
           HRegion.addRegionToMETA(root, meta);
@@ -990,7 +991,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
    * @return Hbase root dir.
    * @return Hbase root dir.
    */
    */
   public Path getRootDir() {
   public Path getRootDir() {
-    return this.dir;
+    return this.rootdir;
   }
   }
 
 
   /**
   /**
@@ -1973,7 +1974,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
       dirName.append(serverInfo.getStartCode());
       dirName.append(serverInfo.getStartCode());
       dirName.append("_");
       dirName.append("_");
       dirName.append(deadServer.getPort());
       dirName.append(deadServer.getPort());
-      this.oldLogDir = new Path(dir, dirName.toString());
+      this.oldLogDir = new Path(rootdir, dirName.toString());
     }
     }
 
 
     /** {@inheritDoc} */
     /** {@inheritDoc} */
@@ -2148,7 +2149,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
             return false;
             return false;
           }
           }
           try {
           try {
-            HLog.splitLog(dir, oldLogDir, fs, conf);
+            HLog.splitLog(rootdir, oldLogDir, fs, conf);
           } finally {
           } finally {
             splitLogLock.unlock();
             splitLogLock.unlock();
           }
           }
@@ -2415,8 +2416,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
 
 
       } else if (deleteRegion) {
       } else if (deleteRegion) {
         try {
         try {
-          HRegion.deleteRegion(fs, dir,
-              HRegionInfo.encodeRegionName(regionInfo.getRegionName()));
+          HRegion.deleteRegion(fs, rootdir, regionInfo);
         } catch (IOException e) {
         } catch (IOException e) {
           e = RemoteExceptionHandler.checkIOException(e);
           e = RemoteExceptionHandler.checkIOException(e);
           LOG.error("failed delete region " + regionInfo.getRegionName(), e);
           LOG.error("failed delete region " + regionInfo.getRegionName(), e);
@@ -2613,7 +2613,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
       // 2. Create the HRegion
       // 2. Create the HRegion
           
           
       HRegion region =
       HRegion region =
-        HRegion.createHRegion(newRegion, this.dir, this.conf);
+        HRegion.createHRegion(newRegion, this.rootdir, this.conf);
 
 
       // 3. Insert into meta
       // 3. Insert into meta
           
           
@@ -2999,8 +2999,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
         // Delete the region
         // Delete the region
       
       
         try {
         try {
-          HRegion.deleteRegion(fs, dir,
-              HRegionInfo.encodeRegionName(i.getRegionName()));
+          HRegion.deleteRegion(fs, rootdir, i);
         
         
         } catch (IOException e) {
         } catch (IOException e) {
           LOG.error("failed to delete region " + i.getRegionName(),
           LOG.error("failed to delete region " + i.getRegionName(),
@@ -3061,15 +3060,16 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
     protected void postProcessMeta(MetaRegion m, HRegionInterface server)
     protected void postProcessMeta(MetaRegion m, HRegionInterface server)
       throws IOException {
       throws IOException {
 
 
+      Path tabledir = new Path(rootdir, tableName.toString());
       for (HRegionInfo i: unservedRegions) {
       for (HRegionInfo i: unservedRegions) {
         i.getTableDesc().families().remove(columnName);
         i.getTableDesc().families().remove(columnName);
         updateRegionInfo(server, m.getRegionName(), i);
         updateRegionInfo(server, m.getRegionName(), i);
 
 
         // Delete the directories used by the column
         // Delete the directories used by the column
 
 
-        String encodedName = HRegionInfo.encodeRegionName(i.getRegionName());
-        fs.delete(HStoreFile.getMapDir(dir, encodedName, columnName));
-        fs.delete(HStoreFile.getInfoDir(dir, encodedName, columnName));
+        String encodedName = i.getEncodedName();
+        fs.delete(HStoreFile.getMapDir(tabledir, encodedName, columnName));
+        fs.delete(HStoreFile.getInfoDir(tabledir, encodedName, columnName));
       }
       }
     }
     }
   }
   }

+ 92 - 94
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMerge.java

@@ -19,20 +19,18 @@
  */
  */
 package org.apache.hadoop.hbase;
 package org.apache.hadoop.hbase;
 
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.Random;
 import java.util.TreeMap;
 import java.util.TreeMap;
-import java.util.TreeSet;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.io.BatchUpdate;
@@ -43,7 +41,6 @@ import org.apache.hadoop.hbase.io.BatchUpdate;
  */
  */
 class HMerge implements HConstants {
 class HMerge implements HConstants {
   static final Log LOG = LogFactory.getLog(HMerge.class);
   static final Log LOG = LogFactory.getLog(HMerge.class);
-  static final Text[] META_COLS = {COL_REGIONINFO};
   static final Random rand = new Random();
   static final Random rand = new Random();
   
   
   private HMerge() {
   private HMerge() {
@@ -74,7 +71,7 @@ class HMerge implements HConstants {
         throw new IllegalStateException(
         throw new IllegalStateException(
             "Can not compact META table if instance is on-line");
             "Can not compact META table if instance is on-line");
       }
       }
-      new OfflineMerger(conf, fs, META_TABLE_NAME).process();
+      new OfflineMerger(conf, fs).process();
 
 
     } else {
     } else {
       if(!masterIsRunning) {
       if(!masterIsRunning) {
@@ -86,42 +83,40 @@ class HMerge implements HConstants {
   }
   }
 
 
   private static abstract class Merger {
   private static abstract class Merger {
-    protected HBaseConfiguration conf;
-    protected FileSystem fs;
-    protected Text tableName;
-    protected Path dir;
-    protected Path basedir;
-    protected HLog hlog;
-    protected DataInputBuffer in;
-    protected boolean more;
-    protected HStoreKey key;
-    protected HRegionInfo info;
+    protected final HBaseConfiguration conf;
+    protected final FileSystem fs;
+    protected final Path tabledir;
+    protected final HLog hlog;
+    private final long maxFilesize;
+
     
     
     protected Merger(HBaseConfiguration conf, FileSystem fs, Text tableName)
     protected Merger(HBaseConfiguration conf, FileSystem fs, Text tableName)
         throws IOException {
         throws IOException {
       
       
       this.conf = conf;
       this.conf = conf;
       this.fs = fs;
       this.fs = fs;
-      this.tableName = tableName;
-      this.in = new DataInputBuffer();
-      this.more = true;
-      this.key = new HStoreKey();
-      this.info = new HRegionInfo();
-      this.dir = new Path(conf.get(HBASE_DIR, DEFAULT_HBASE_DIR));
-      this.basedir = new Path(dir, "merge_" + System.currentTimeMillis());
-      fs.mkdirs(basedir);
+      this.maxFilesize =
+        conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE);
+
+      this.tabledir = new Path(
+          fs.makeQualified(new Path(conf.get(HBASE_DIR, DEFAULT_HBASE_DIR))),
+          tableName.toString()
+      );
+      Path logdir = new Path(tabledir, "merge_" + System.currentTimeMillis() +
+          HREGION_LOGDIR_NAME);
       this.hlog =
       this.hlog =
-        new HLog(fs, new Path(basedir, HREGION_LOGDIR_NAME), conf, null);
+        new HLog(fs, logdir, conf, null);
     }
     }
     
     
     void process() throws IOException {
     void process() throws IOException {
       try {
       try {
-        while(more) {
-          TreeSet<HRegionInfo> regionsToMerge = next();
-          if(regionsToMerge == null) {
-            break;
+        for(HRegionInfo[] regionsToMerge = next();
+        regionsToMerge != null;
+        regionsToMerge = next()) {
+
+          if (!merge(regionsToMerge)) {
+            return;
           }
           }
-          merge(regionsToMerge.toArray(new HRegionInfo[regionsToMerge.size()]));
         }
         }
       } finally {
       } finally {
         try {
         try {
@@ -130,19 +125,13 @@ class HMerge implements HConstants {
         } catch(IOException e) {
         } catch(IOException e) {
           LOG.error(e);
           LOG.error(e);
         }
         }
-        try {
-          fs.delete(basedir);
-          
-        } catch(IOException e) {
-          LOG.error(e);
-        }
       }
       }
     }
     }
     
     
-    private void merge(HRegionInfo[] regions) throws IOException {
-      if(regions.length < 2) {
+    private boolean merge(HRegionInfo[] info) throws IOException {
+      if(info.length < 2) {
         LOG.info("only one region - nothing to merge");
         LOG.info("only one region - nothing to merge");
-        return;
+        return false;
       }
       }
       
       
       HRegion currentRegion = null;
       HRegion currentRegion = null;
@@ -150,20 +139,18 @@ class HMerge implements HConstants {
       HRegion nextRegion = null;
       HRegion nextRegion = null;
       long nextSize = 0;
       long nextSize = 0;
       Text midKey = new Text();
       Text midKey = new Text();
-      for(int i = 0; i < regions.length - 1; i++) {
-        if(currentRegion == null) {
+      for (int i = 0; i < info.length - 1; i++) {
+        if (currentRegion == null) {
           currentRegion =
           currentRegion =
-            new HRegion(dir, hlog, fs, conf, regions[i], null, null);
+            new HRegion(tabledir, hlog, fs, conf, info[i], null, null);
           currentSize = currentRegion.largestHStore(midKey).getAggregate();
           currentSize = currentRegion.largestHStore(midKey).getAggregate();
         }
         }
         nextRegion =
         nextRegion =
-          new HRegion(dir, hlog, fs, conf, regions[i + 1], null, null);
+          new HRegion(tabledir, hlog, fs, conf, info[i + 1], null, null);
 
 
         nextSize = nextRegion.largestHStore(midKey).getAggregate();
         nextSize = nextRegion.largestHStore(midKey).getAggregate();
 
 
-        long maxFilesize =
-          conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE);
-        if((currentSize + nextSize) <= (maxFilesize / 2)) {
+        if ((currentSize + nextSize) <= (maxFilesize / 2)) {
           // We merge two adjacent regions if their total size is less than
           // We merge two adjacent regions if their total size is less than
           // one half of the desired maximum size
           // one half of the desired maximum size
 
 
@@ -175,10 +162,7 @@ class HMerge implements HConstants {
           updateMeta(currentRegion.getRegionName(), nextRegion.getRegionName(),
           updateMeta(currentRegion.getRegionName(), nextRegion.getRegionName(),
               mergedRegion);
               mergedRegion);
 
 
-          currentRegion = null;
-          i++;
-          continue;
-          
+          break;
         }
         }
         LOG.info("not merging regions " + currentRegion.getRegionName()
         LOG.info("not merging regions " + currentRegion.getRegionName()
             + " and " + nextRegion.getRegionName());
             + " and " + nextRegion.getRegionName());
@@ -190,9 +174,10 @@ class HMerge implements HConstants {
       if(currentRegion != null) {
       if(currentRegion != null) {
         currentRegion.close();
         currentRegion.close();
       }
       }
+      return true;
     }
     }
     
     
-    protected abstract TreeSet<HRegionInfo> next() throws IOException;
+    protected abstract HRegionInfo[] next() throws IOException;
     
     
     protected abstract void updateMeta(Text oldRegion1, Text oldRegion2,
     protected abstract void updateMeta(Text oldRegion1, Text oldRegion2,
         HRegion newRegion) throws IOException;
         HRegion newRegion) throws IOException;
@@ -201,55 +186,63 @@ class HMerge implements HConstants {
 
 
   /** Instantiated to compact a normal user table */
   /** Instantiated to compact a normal user table */
   private static class OnlineMerger extends Merger {
   private static class OnlineMerger extends Merger {
-    private HTable table;
-    private HScannerInterface metaScanner;
+    private final Text tableName;
+    private final HTable table;
+    private final HScannerInterface metaScanner;
     private HRegionInfo latestRegion;
     private HRegionInfo latestRegion;
     
     
     OnlineMerger(HBaseConfiguration conf, FileSystem fs, Text tableName)
     OnlineMerger(HBaseConfiguration conf, FileSystem fs, Text tableName)
     throws IOException {
     throws IOException {
       
       
       super(conf, fs, tableName);
       super(conf, fs, tableName);
+      this.tableName = tableName;
       this.table = new HTable(conf, META_TABLE_NAME);
       this.table = new HTable(conf, META_TABLE_NAME);
-      this.metaScanner = table.obtainScanner(META_COLS, new Text());
+      this.metaScanner = table.obtainScanner(COL_REGIONINFO_ARRAY, tableName);
       this.latestRegion = null;
       this.latestRegion = null;
     }
     }
     
     
     private HRegionInfo nextRegion() throws IOException {
     private HRegionInfo nextRegion() throws IOException {
       try {
       try {
+        HStoreKey key = new HStoreKey();
         TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
         TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
-        if(! metaScanner.next(key, results)) {
-          more = false;
+        if (! metaScanner.next(key, results)) {
           return null;
           return null;
         }
         }
         byte[] bytes = results.get(COL_REGIONINFO);
         byte[] bytes = results.get(COL_REGIONINFO);
-        if(bytes == null || bytes.length == 0) {
+        if (bytes == null || bytes.length == 0) {
           throw new NoSuchElementException("meta region entry missing "
           throw new NoSuchElementException("meta region entry missing "
               + COL_REGIONINFO);
               + COL_REGIONINFO);
         }
         }
         HRegionInfo region =
         HRegionInfo region =
           (HRegionInfo) Writables.getWritable(bytes, new HRegionInfo());
           (HRegionInfo) Writables.getWritable(bytes, new HRegionInfo());
 
 
-        if(!region.isOffline()) {
+        if (!region.getTableDesc().getName().equals(tableName)) {
+          return null;
+        }
+        
+        if (!region.isOffline()) {
           throw new TableNotDisabledException("region " + region.getRegionName()
           throw new TableNotDisabledException("region " + region.getRegionName()
               + " is not disabled");
               + " is not disabled");
         }
         }
         return region;
         return region;
         
         
-      } catch(IOException e) {
+      } catch (IOException e) {
+        e = RemoteExceptionHandler.checkIOException(e);
+        LOG.error("meta scanner error", e);
         try {
         try {
           metaScanner.close();
           metaScanner.close();
           
           
-        } catch(IOException ex) {
-          LOG.error(ex);
+        } catch (IOException ex) {
+          ex = RemoteExceptionHandler.checkIOException(ex);
+          LOG.error("error closing scanner", ex);
         }
         }
-        more = false;
         throw e;
         throw e;
       }
       }
     }
     }
 
 
     @Override
     @Override
-    protected TreeSet<HRegionInfo> next() throws IOException {
-      TreeSet<HRegionInfo> regions = new TreeSet<HRegionInfo>();
+    protected HRegionInfo[] next() throws IOException {
+      List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
       if(latestRegion == null) {
       if(latestRegion == null) {
         latestRegion = nextRegion();
         latestRegion = nextRegion();
       }
       }
@@ -260,7 +253,7 @@ class HMerge implements HConstants {
       if(latestRegion != null) {
       if(latestRegion != null) {
         regions.add(latestRegion);
         regions.add(latestRegion);
       }
       }
-      return regions;
+      return regions.toArray(new HRegionInfo[regions.size()]);
     }
     }
 
 
     @Override
     @Override
@@ -280,6 +273,8 @@ class HMerge implements HConstants {
           table.delete(lockid, COL_REGIONINFO);
           table.delete(lockid, COL_REGIONINFO);
           table.delete(lockid, COL_SERVER);
           table.delete(lockid, COL_SERVER);
           table.delete(lockid, COL_STARTCODE);
           table.delete(lockid, COL_STARTCODE);
+          table.delete(lockid, COL_SPLITA);
+          table.delete(lockid, COL_SPLITB);
           table.commit(lockid);
           table.commit(lockid);
           lockid = -1L;
           lockid = -1L;
 
 
@@ -292,14 +287,12 @@ class HMerge implements HConstants {
           }
           }
         }
         }
       }
       }
-      ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
-      DataOutputStream s = new DataOutputStream(byteValue);
       newRegion.getRegionInfo().setOffline(true);
       newRegion.getRegionInfo().setOffline(true);
-      newRegion.getRegionInfo().write(s);
       long lockid = -1L;
       long lockid = -1L;
       try {
       try {
         lockid = table.startUpdate(newRegion.getRegionName());
         lockid = table.startUpdate(newRegion.getRegionName());
-        table.put(lockid, COL_REGIONINFO, byteValue.toByteArray());
+        table.put(lockid, COL_REGIONINFO,
+            Writables.getBytes(newRegion.getRegionInfo()));
         table.commit(lockid);
         table.commit(lockid);
         lockid = -1L;
         lockid = -1L;
 
 
@@ -317,31 +310,35 @@ class HMerge implements HConstants {
 
 
   /** Instantiated to compact the meta region */
   /** Instantiated to compact the meta region */
   private static class OfflineMerger extends Merger {
   private static class OfflineMerger extends Merger {
-    private TreeSet<HRegionInfo> metaRegions;
-    private TreeMap<Text, byte []> results;
+    private final List<HRegionInfo> metaRegions = new ArrayList<HRegionInfo>();
+    private final HRegion root;
     
     
-    OfflineMerger(HBaseConfiguration conf, FileSystem fs, Text tableName)
+    OfflineMerger(HBaseConfiguration conf, FileSystem fs)
         throws IOException {
         throws IOException {
       
       
-      super(conf, fs, tableName);
-      this.metaRegions = new TreeSet<HRegionInfo>();
-      this.results = new TreeMap<Text, byte []>();
+      super(conf, fs, META_TABLE_NAME);
+
+      Path rootTableDir = HTableDescriptor.getTableDir(
+          fs.makeQualified(new Path(conf.get(HBASE_DIR, DEFAULT_HBASE_DIR))),
+          ROOT_TABLE_NAME);
 
 
       // Scan root region to find all the meta regions
       // Scan root region to find all the meta regions
       
       
-      HRegion root =
-        new HRegion(dir, hlog,fs, conf, HRegionInfo.rootRegionInfo, null, null);
+      root = new HRegion(rootTableDir, hlog, fs, conf,
+          HRegionInfo.rootRegionInfo, null, null);
 
 
-      HScannerInterface rootScanner =
-        root.getScanner(META_COLS, new Text(), System.currentTimeMillis(), null);
+      HScannerInterface rootScanner = root.getScanner(COL_REGIONINFO_ARRAY,
+          new Text(), System.currentTimeMillis(), null);
       
       
       try {
       try {
+        HStoreKey key = new HStoreKey();
+        TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
         while(rootScanner.next(key, results)) {
         while(rootScanner.next(key, results)) {
           for(byte [] b: results.values()) {
           for(byte [] b: results.values()) {
-            in.reset(b, b.length);
-            info.readFields(in);
-            metaRegions.add(info);
-            results.clear();
+            HRegionInfo info = Writables.getHRegionInfoOrNull(b);
+            if (info != null) {
+              metaRegions.add(info);
+            }
           }
           }
         }
         }
       } finally {
       } finally {
@@ -356,18 +353,19 @@ class HMerge implements HConstants {
     }
     }
 
 
     @Override
     @Override
-    protected TreeSet<HRegionInfo> next() {
-      more = false;
-      return metaRegions;
+    protected HRegionInfo[] next() {
+      HRegionInfo[] results = null;
+      if (metaRegions.size() > 0) {
+        results = metaRegions.toArray(new HRegionInfo[metaRegions.size()]);
+        metaRegions.clear();
+      }
+      return results;
     }
     }
 
 
     @Override
     @Override
     protected void updateMeta(Text oldRegion1, Text oldRegion2,
     protected void updateMeta(Text oldRegion1, Text oldRegion2,
         HRegion newRegion) throws IOException {
         HRegion newRegion) throws IOException {
       
       
-      HRegion root =
-        new HRegion(dir, hlog, fs, conf, HRegionInfo.rootRegionInfo, null, null);
-
       Text[] regionsToDelete = {
       Text[] regionsToDelete = {
           oldRegion1,
           oldRegion1,
           oldRegion2
           oldRegion2
@@ -379,6 +377,8 @@ class HMerge implements HConstants {
         b.delete(lockid, COL_REGIONINFO);
         b.delete(lockid, COL_REGIONINFO);
         b.delete(lockid, COL_SERVER);
         b.delete(lockid, COL_SERVER);
         b.delete(lockid, COL_STARTCODE);
         b.delete(lockid, COL_STARTCODE);
+        b.delete(lockid, COL_SPLITA);
+        b.delete(lockid, COL_SPLITB);
         root.batchUpdate(System.currentTimeMillis(), b);
         root.batchUpdate(System.currentTimeMillis(), b);
         lockid = -1L;
         lockid = -1L;
 
 
@@ -386,14 +386,12 @@ class HMerge implements HConstants {
           LOG.debug("updated columns in row: " + regionsToDelete[r]);
           LOG.debug("updated columns in row: " + regionsToDelete[r]);
         }
         }
       }
       }
-      ByteArrayOutputStream byteValue = new ByteArrayOutputStream();
-      DataOutputStream s = new DataOutputStream(byteValue);
-      newRegion.getRegionInfo().setOffline(true);
-      newRegion.getRegionInfo().write(s);
+      HRegionInfo newInfo = newRegion.getRegionInfo();
+      newInfo.setOffline(true);
       long lockid = Math.abs(rand.nextLong());
       long lockid = Math.abs(rand.nextLong());
       BatchUpdate b = new BatchUpdate(lockid);
       BatchUpdate b = new BatchUpdate(lockid);
       lockid = b.startUpdate(newRegion.getRegionName());
       lockid = b.startUpdate(newRegion.getRegionName());
-      b.put(lockid, COL_REGIONINFO, byteValue.toByteArray());
+      b.put(lockid, COL_REGIONINFO, Writables.getBytes(newInfo));
       root.batchUpdate(System.currentTimeMillis(), b);
       root.batchUpdate(System.currentTimeMillis(), b);
       if(LOG.isDebugEnabled()) {
       if(LOG.isDebugEnabled()) {
         LOG.debug("updated columns in row: " + newRegion.getRegionName());
         LOG.debug("updated columns in row: " + newRegion.getRegionName());

+ 71 - 81
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java

@@ -122,7 +122,7 @@ public class HRegion implements HConstants {
     HBaseConfiguration conf = a.getConf();
     HBaseConfiguration conf = a.getConf();
     HTableDescriptor tabledesc = a.getTableDesc();
     HTableDescriptor tabledesc = a.getTableDesc();
     HLog log = a.getLog();
     HLog log = a.getLog();
-    Path rootDir = a.getRootDir();
+    Path basedir = a.getBaseDir();
     Text startKey = a.getStartKey();
     Text startKey = a.getStartKey();
     Text endKey = b.getEndKey();
     Text endKey = b.getEndKey();
     Path merges = new Path(a.getRegionDir(), MERGEDIR);
     Path merges = new Path(a.getRegionDir(), MERGEDIR);
@@ -131,8 +131,8 @@ public class HRegion implements HConstants {
     }
     }
 
 
     HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
     HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
-    Path newRegionDir = HRegion.getRegionDir(merges,
-        HRegionInfo.encodeRegionName(newRegionInfo.getRegionName()));
+    Path newRegionDir =
+      HRegion.getRegionDir(merges, newRegionInfo.getEncodedName());
     if(fs.exists(newRegionDir)) {
     if(fs.exists(newRegionDir)) {
       throw new IOException("Cannot merge; target file collision at " +
       throw new IOException("Cannot merge; target file collision at " +
           newRegionDir);
           newRegionDir);
@@ -148,15 +148,14 @@ public class HRegion implements HConstants {
     for (Map.Entry<Text, List<HStoreFile>> es : byFamily.entrySet()) {
     for (Map.Entry<Text, List<HStoreFile>> es : byFamily.entrySet()) {
       Text colFamily = es.getKey();
       Text colFamily = es.getKey();
       List<HStoreFile> srcFiles = es.getValue();
       List<HStoreFile> srcFiles = es.getValue();
-      HStoreFile dst = new HStoreFile(conf, merges,
-          HRegionInfo.encodeRegionName(newRegionInfo.getRegionName()),
-          colFamily, Math.abs(rand.nextLong()));
+      HStoreFile dst = new HStoreFile(conf, fs, merges,
+          newRegionInfo.getEncodedName(), colFamily, -1, null);
       dst.mergeStoreFiles(srcFiles, fs, conf);
       dst.mergeStoreFiles(srcFiles, fs, conf);
     }
     }
 
 
     // Done
     // Done
     // Construction moves the merge files into place under region.
     // Construction moves the merge files into place under region.
-    HRegion dstRegion = new HRegion(rootDir, log, fs, conf, newRegionInfo,
+    HRegion dstRegion = new HRegion(basedir, log, fs, conf, newRegionInfo,
         newRegionDir, null);
         newRegionDir, null);
 
 
     // Get rid of merges directory
     // Get rid of merges directory
@@ -199,7 +198,7 @@ public class HRegion implements HConstants {
 
 
   final AtomicLong memcacheSize = new AtomicLong(0);
   final AtomicLong memcacheSize = new AtomicLong(0);
 
 
-  final Path rootDir;
+  final Path basedir;
   final HLog log;
   final HLog log;
   final FileSystem fs;
   final FileSystem fs;
   final HBaseConfiguration conf;
   final HBaseConfiguration conf;
@@ -228,7 +227,6 @@ public class HRegion implements HConstants {
   private final Integer splitLock = new Integer(0);
   private final Integer splitLock = new Integer(0);
   private final long desiredMaxFileSize;
   private final long desiredMaxFileSize;
   private final long minSequenceId;
   private final long minSequenceId;
-  private final String encodedRegionName;
   final AtomicInteger activeScannerCount = new AtomicInteger(0);
   final AtomicInteger activeScannerCount = new AtomicInteger(0);
 
 
   //////////////////////////////////////////////////////////////////////////////
   //////////////////////////////////////////////////////////////////////////////
@@ -245,7 +243,8 @@ public class HRegion implements HConstants {
    * appropriate log info for this HRegion. If there is a previous log file
    * appropriate log info for this HRegion. If there is a previous log file
    * (implying that the HRegion has been written-to before), then read it from
    * (implying that the HRegion has been written-to before), then read it from
    * the supplied path.
    * the supplied path.
-   * @param rootDir root directory for HBase instance
+   * @param basedir qualified path of directory where region should be located,
+   * usually the table directory.
    * @param fs is the filesystem.  
    * @param fs is the filesystem.  
    * @param conf is global configuration settings.
    * @param conf is global configuration settings.
    * @param regionInfo - HRegionInfo that describes the region
    * @param regionInfo - HRegionInfo that describes the region
@@ -255,22 +254,17 @@ public class HRegion implements HConstants {
    * 
    * 
    * @throws IOException
    * @throws IOException
    */
    */
-  public HRegion(Path rootDir, HLog log, FileSystem fs, HBaseConfiguration conf, 
+  public HRegion(Path basedir, HLog log, FileSystem fs, HBaseConfiguration conf, 
       HRegionInfo regionInfo, Path initialFiles, CacheFlushListener listener)
       HRegionInfo regionInfo, Path initialFiles, CacheFlushListener listener)
     throws IOException {
     throws IOException {
     
     
-    this.rootDir = rootDir;
+    this.basedir = basedir;
     this.log = log;
     this.log = log;
     this.fs = fs;
     this.fs = fs;
     this.conf = conf;
     this.conf = conf;
     this.regionInfo = regionInfo;
     this.regionInfo = regionInfo;
-    this.encodedRegionName =
-      HRegionInfo.encodeRegionName(this.regionInfo.getRegionName());
     this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
     this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
-
-    // Declare the regionName.  This is a unique string for the region, used to 
-    // build a unique filename.
-    this.regiondir = HRegion.getRegionDir(rootDir, this.encodedRegionName);
+    this.regiondir = new Path(basedir, this.regionInfo.getEncodedName());
     Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
     Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
 
 
     // Move prefab HStore files into place (if any).  This picks up split files
     // Move prefab HStore files into place (if any).  This picks up split files
@@ -281,14 +275,13 @@ public class HRegion implements HConstants {
 
 
     // Load in all the HStores.
     // Load in all the HStores.
     long maxSeqId = -1;
     long maxSeqId = -1;
-    for(Map.Entry<Text, HColumnDescriptor> e :
-      this.regionInfo.getTableDesc().families().entrySet()) {
-      Text colFamily = HStoreKey.extractFamily(e.getKey()).toText();
+    for(HColumnDescriptor c :
+      this.regionInfo.getTableDesc().families().values()) {
 
 
-      HStore store = new HStore(rootDir, this.regionInfo.getRegionName(),
-          this.encodedRegionName, e.getValue(), fs, oldLogFile, conf); 
+      HStore store = new HStore(this.basedir, this.regionInfo, c, this.fs,
+          oldLogFile, this.conf);
 
 
-      stores.put(colFamily, store);
+      stores.put(c.getFamilyName(), store);
 
 
       long storeSeqId = store.getMaxSequenceId();
       long storeSeqId = store.getMaxSequenceId();
       if (storeSeqId > maxSeqId) {
       if (storeSeqId > maxSeqId) {
@@ -450,7 +443,7 @@ public class HRegion implements HConstants {
   }
   }
 
 
   /** @return region id */
   /** @return region id */
-  public long getRegionId() {
+  long getRegionId() {
     return this.regionInfo.getRegionId();
     return this.regionInfo.getRegionId();
   }
   }
 
 
@@ -459,13 +452,8 @@ public class HRegion implements HConstants {
     return this.regionInfo.getRegionName();
     return this.regionInfo.getRegionName();
   }
   }
 
 
-  /** @return root directory path */
-  public Path getRootDir() {
-    return rootDir;
-  }
-
   /** @return HTableDescriptor for this region */
   /** @return HTableDescriptor for this region */
-  public HTableDescriptor getTableDesc() {
+  HTableDescriptor getTableDesc() {
     return this.regionInfo.getTableDesc();
     return this.regionInfo.getTableDesc();
   }
   }
 
 
@@ -475,19 +463,17 @@ public class HRegion implements HConstants {
   }
   }
 
 
   /** @return Configuration object */
   /** @return Configuration object */
-  public HBaseConfiguration getConf() {
+  HBaseConfiguration getConf() {
     return this.conf;
     return this.conf;
   }
   }
 
 
-  /** @return region directory Path
-   * @see HRegion#getRegionDir(Path, String)
-   */
-  public Path getRegionDir() {
+  /** @return region directory Path */
+  Path getRegionDir() {
     return this.regiondir;
     return this.regiondir;
   }
   }
 
 
   /** @return FileSystem being used by this region */
   /** @return FileSystem being used by this region */
-  public FileSystem getFilesystem() {
+  FileSystem getFilesystem() {
     return this.fs;
     return this.fs;
   }
   }
 
 
@@ -550,18 +536,19 @@ public class HRegion implements HConstants {
         return null;
         return null;
       }
       }
       long startTime = System.currentTimeMillis();
       long startTime = System.currentTimeMillis();
-      Path splits = getSplitsDir();
+      Path splits = new Path(this.regiondir, SPLITDIR);
+      if(!this.fs.exists(splits)) {
+        this.fs.mkdirs(splits);
+      }
       HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
       HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
           this.regionInfo.getStartKey(), midKey);
           this.regionInfo.getStartKey(), midKey);
-      Path dirA = getSplitRegionDir(splits,
-          HRegionInfo.encodeRegionName(regionAInfo.getRegionName()));
+      Path dirA = new Path(splits, regionAInfo.getEncodedName());
       if(fs.exists(dirA)) {
       if(fs.exists(dirA)) {
         throw new IOException("Cannot split; target file collision at " + dirA);
         throw new IOException("Cannot split; target file collision at " + dirA);
       }
       }
       HRegionInfo regionBInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
       HRegionInfo regionBInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
           midKey, this.regionInfo.getEndKey());
           midKey, this.regionInfo.getEndKey());
-      Path dirB = getSplitRegionDir(splits,
-          HRegionInfo.encodeRegionName(regionBInfo.getRegionName()));
+      Path dirB = new Path(splits, regionBInfo.getEncodedName());
       if(this.fs.exists(dirB)) {
       if(this.fs.exists(dirB)) {
         throw new IOException("Cannot split; target file collision at " + dirB);
         throw new IOException("Cannot split; target file collision at " + dirB);
       }
       }
@@ -592,18 +579,16 @@ public class HRegion implements HConstants {
       for(HStoreFile h: hstoreFilesToSplit) {
       for(HStoreFile h: hstoreFilesToSplit) {
         // A reference to the bottom half of the hsf store file.
         // A reference to the bottom half of the hsf store file.
         HStoreFile.Reference aReference = new HStoreFile.Reference(
         HStoreFile.Reference aReference = new HStoreFile.Reference(
-            this.encodedRegionName, h.getFileId(), new HStoreKey(midKey),
-            HStoreFile.Range.bottom);
-        HStoreFile a = new HStoreFile(this.conf, splits,
-            HRegionInfo.encodeRegionName(regionAInfo.getRegionName()),
-            h.getColFamily(), Math.abs(rand.nextLong()), aReference);
+            this.regionInfo.getEncodedName(), h.getFileId(),
+            new HStoreKey(midKey), HStoreFile.Range.bottom);
+        HStoreFile a = new HStoreFile(this.conf, fs, splits,
+            regionAInfo.getEncodedName(), h.getColFamily(), -1, aReference);
         // Reference to top half of the hsf store file.
         // Reference to top half of the hsf store file.
         HStoreFile.Reference bReference = new HStoreFile.Reference(
         HStoreFile.Reference bReference = new HStoreFile.Reference(
-            this.encodedRegionName, h.getFileId(), new HStoreKey(midKey),
-            HStoreFile.Range.top);
-        HStoreFile b = new HStoreFile(this.conf, splits,
-            HRegionInfo.encodeRegionName(regionBInfo.getRegionName()),
-            h.getColFamily(), Math.abs(rand.nextLong()), bReference);
+            this.regionInfo.getEncodedName(), h.getFileId(),
+            new HStoreKey(midKey), HStoreFile.Range.top);
+        HStoreFile b = new HStoreFile(this.conf, fs, splits,
+            regionBInfo.getEncodedName(), h.getColFamily(), -1, bReference);
         h.splitStoreFile(a, b, this.fs);
         h.splitStoreFile(a, b, this.fs);
       }
       }
 
 
@@ -611,10 +596,10 @@ public class HRegion implements HConstants {
       // Opening the region copies the splits files from the splits directory
       // Opening the region copies the splits files from the splits directory
       // under each region.
       // under each region.
       HRegion regionA =
       HRegion regionA =
-        new HRegion(rootDir, log, fs, conf, regionAInfo, dirA, null);
+        new HRegion(basedir, log, fs, conf, regionAInfo, dirA, null);
       regionA.close();
       regionA.close();
       HRegion regionB =
       HRegion regionB =
-        new HRegion(rootDir, log, fs, conf, regionBInfo, dirB, null);
+        new HRegion(basedir, log, fs, conf, regionBInfo, dirB, null);
       regionB.close();
       regionB.close();
 
 
       // Cleanup
       // Cleanup
@@ -673,18 +658,6 @@ public class HRegion implements HConstants {
     return split;
     return split;
   }
   }
   
   
-  private Path getSplitRegionDir(final Path splits, final String region) {
-    return HRegion.getRegionDir(splits, region);
-  }
-  
-  private Path getSplitsDir() throws IOException {
-    Path splits = new Path(this.regiondir, SPLITDIR);
-    if(!this.fs.exists(splits)) {
-      this.fs.mkdirs(splits);
-    }
-    return splits;
-  }
-
   /**
   /**
    * Only do a compaction if it is necessary
    * Only do a compaction if it is necessary
    * 
    * 
@@ -1511,7 +1484,11 @@ public class HRegion implements HConstants {
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
-    return getRegionName().toString();
+    return regionInfo.getRegionName().toString();
+  }
+  
+  private Path getBaseDir() {
+    return this.basedir;
   }
   }
 
 
   /**
   /**
@@ -1707,13 +1684,13 @@ public class HRegion implements HConstants {
    * @throws IOException
    * @throws IOException
    */
    */
   static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
   static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
-      final HBaseConfiguration conf)
-  throws IOException {
-    Path regionDir = HRegion.getRegionDir(rootDir,
-        HRegionInfo.encodeRegionName(info.getRegionName()));
+      final HBaseConfiguration conf) throws IOException {
+    Path tableDir =
+      HTableDescriptor.getTableDir(rootDir, info.getTableDesc().getName());
+    Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
     fs.mkdirs(regionDir);
     fs.mkdirs(regionDir);
-    return new HRegion(rootDir,
+    return new HRegion(tableDir,
       new HLog(fs, new Path(regionDir, HREGION_LOGDIR_NAME), conf, null),
       new HLog(fs, new Path(regionDir, HREGION_LOGDIR_NAME), conf, null),
       fs, conf, info, null, null);
       fs, conf, info, null, null);
   }
   }
@@ -1749,15 +1726,14 @@ public class HRegion implements HConstants {
    * Deletes all the files for a HRegion
    * Deletes all the files for a HRegion
    * 
    * 
    * @param fs the file system object
    * @param fs the file system object
-   * @param baseDirectory base directory for HBase
-   * @param name region file name ENCODED!
+   * @param rootdir qualified path of HBase root directory
+   * @param info HRegionInfo for region to be deleted
    * @throws IOException
    * @throws IOException
    * @return True if deleted.
    * @return True if deleted.
-   * @see HRegionInfo#encodeRegionName(Text)
    */
    */
-  static boolean deleteRegion(FileSystem fs, Path baseDirectory, String name)
-    throws IOException {
-    Path p = HRegion.getRegionDir(fs.makeQualified(baseDirectory), name);
+  static boolean deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
+  throws IOException {
+    Path p = HRegion.getRegionDir(rootdir, info);
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("DELETING region " + p.toString());
       LOG.debug("DELETING region " + p.toString());
     }
     }
@@ -1767,12 +1743,26 @@ public class HRegion implements HConstants {
   /**
   /**
    * Computes the Path of the HRegion
    * Computes the Path of the HRegion
    * 
    * 
-   * @param dir hbase home directory
+   * @param tabledir qualified path for table
    * @param name region file name ENCODED!
    * @param name region file name ENCODED!
    * @return Path of HRegion directory
    * @return Path of HRegion directory
    * @see HRegionInfo#encodeRegionName(Text)
    * @see HRegionInfo#encodeRegionName(Text)
    */
    */
-  public static Path getRegionDir(final Path dir, final String name) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + name));
+  static Path getRegionDir(final Path tabledir, final String name) {
+    return new Path(tabledir, name);
+  }
+  
+  /**
+   * Computes the Path of the HRegion
+   * 
+   * @param rootdir qualified path of HBase root directory
+   * @param info HRegionInfo for the region
+   * @return qualified path of region directory
+   */
+  static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
+    return new Path(
+        HTableDescriptor.getTableDir(rootdir, info.getTableDesc().getName()),
+        info.getEncodedName()
+    );
   }
   }
 }
 }

+ 11 - 4
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java

@@ -79,6 +79,7 @@ public class HRegionInfo implements WritableComparable {
   private Text startKey;
   private Text startKey;
   private HTableDescriptor tableDesc;
   private HTableDescriptor tableDesc;
   private int hashCode;
   private int hashCode;
+  private transient String encodedName = null;
   
   
   private void setHashCode() {
   private void setHashCode() {
     int result = this.regionName.hashCode();
     int result = this.regionName.hashCode();
@@ -182,6 +183,14 @@ public class HRegionInfo implements WritableComparable {
   public Text getRegionName(){
   public Text getRegionName(){
     return regionName;
     return regionName;
   }
   }
+  
+  /** @return the encoded region name */
+  public synchronized String getEncodedName() {
+    if (encodedName == null) {
+      encodedName = encodeRegionName(regionName);
+    }
+    return encodedName;
+  }
 
 
   /** @return the startKey */
   /** @return the startKey */
   public Text getStartKey(){
   public Text getStartKey(){
@@ -242,10 +251,8 @@ public class HRegionInfo implements WritableComparable {
   @Override
   @Override
   public String toString() {
   public String toString() {
     return "regionname: " + this.regionName.toString() + ", startKey: <" +
     return "regionname: " + this.regionName.toString() + ", startKey: <" +
-      this.startKey.toString() + ">, encodedName(" +
-      encodeRegionName(this.regionName) + ")" +
-      (isOffline()? " offline: true,": "") +
-      (isSplit()? " split: true,": "") +
+      this.startKey.toString() + ">, encodedName(" + getEncodedName() + ")" +
+      (isOffline()? " offline: true,": "") + (isSplit()? " split: true,": "") +
       " tableDesc: {" + this.tableDesc.toString() + "}";
       " tableDesc: {" + this.tableDesc.toString() + "}";
   }
   }
     
     

+ 17 - 12
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java

@@ -92,6 +92,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
   
   
   protected final HServerInfo serverInfo;
   protected final HServerInfo serverInfo;
   protected final HBaseConfiguration conf;
   protected final HBaseConfiguration conf;
+  private FileSystem fs;
+  private Path rootDir;
   private final Random rand = new Random();
   private final Random rand = new Random();
   
   
   // region name -> HRegion
   // region name -> HRegion
@@ -138,15 +140,18 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
    * is registered as a shutdown hook in the HRegionServer constructor and is
    * is registered as a shutdown hook in the HRegionServer constructor and is
    * only called when the HRegionServer receives a kill signal.
    * only called when the HRegionServer receives a kill signal.
    */
    */
-  class ShutdownThread 
-    extends Thread {
-    
+  class ShutdownThread extends Thread {
     private final HRegionServer instance;
     private final HRegionServer instance;
     
     
+    /**
+     * @param instance
+     */
     public ShutdownThread(HRegionServer instance) {
     public ShutdownThread(HRegionServer instance) {
       this.instance = instance;
       this.instance = instance;
     }
     }
 
 
+    /** {@inheritDoc} */
+    @Override
     public synchronized void start() {
     public synchronized void start() {
       LOG.info("Starting shutdown thread.");
       LOG.info("Starting shutdown thread.");
       
       
@@ -914,6 +919,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
         }
         }
         this.conf.set(key, value);
         this.conf.set(key, value);
       }
       }
+      this.fs = FileSystem.get(this.conf);
+      this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
       this.log = setupHLog();
       this.log = setupHLog();
       startServiceThreads();
       startServiceThreads();
     } catch (IOException e) {
     } catch (IOException e) {
@@ -929,15 +936,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
   private HLog setupHLog() throws RegionServerRunningException,
   private HLog setupHLog() throws RegionServerRunningException,
     IOException {
     IOException {
     
     
-    String rootDir = this.conf.get(HConstants.HBASE_DIR);
-    LOG.info("Root dir: " + rootDir);
-    Path logdir = new Path(new Path(rootDir), "log" + "_" + getThisIP() + "_" +
+    Path logdir = new Path(rootDir, "log" + "_" + getThisIP() + "_" +
         this.serverInfo.getStartCode() + "_" + 
         this.serverInfo.getStartCode() + "_" + 
         this.serverInfo.getServerAddress().getPort());
         this.serverInfo.getServerAddress().getPort());
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Log dir " + logdir);
       LOG.debug("Log dir " + logdir);
     }
     }
-    FileSystem fs = FileSystem.get(this.conf);
     if (fs.exists(logdir)) {
     if (fs.exists(logdir)) {
       throw new RegionServerRunningException("region server already " +
       throw new RegionServerRunningException("region server already " +
         "running at " + this.serverInfo.getServerAddress().toString() +
         "running at " + this.serverInfo.getServerAddress().toString() +
@@ -1209,9 +1213,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
     HRegion region = onlineRegions.get(regionInfo.getRegionName());
     HRegion region = onlineRegions.get(regionInfo.getRegionName());
     if(region == null) {
     if(region == null) {
       try {
       try {
-        region = new HRegion(new Path(this.conf.get(HConstants.HBASE_DIR)),
-            this.log, FileSystem.get(conf), conf, regionInfo, null,
-            this.cacheFlusher);
+        region = new HRegion(
+            HTableDescriptor.getTableDir(rootDir,
+                regionInfo.getTableDesc().getName()
+            ),
+            this.log, this.fs, conf, regionInfo, null, this.cacheFlusher
+        );
         
         
       } catch (IOException e) {
       } catch (IOException e) {
         LOG.error("error opening region " + regionInfo.getRegionName(), e);
         LOG.error("error opening region " + regionInfo.getRegionName(), e);
@@ -1651,9 +1658,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
    */
    */
   protected boolean checkFileSystem() {
   protected boolean checkFileSystem() {
     if (this.fsOk) {
     if (this.fsOk) {
-      FileSystem fs = null;
       try {
       try {
-        fs = FileSystem.get(this.conf);
         if (fs != null && !FSUtils.isFileSystemAvailable(fs)) {
         if (fs != null && !FSUtils.isFileSystemAvailable(fs)) {
           LOG.fatal("Shutting down HRegionServer: file system not available");
           LOG.fatal("Shutting down HRegionServer: file system not available");
           this.abortRequested = true;
           this.abortRequested = true;

+ 248 - 197
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java

@@ -19,7 +19,6 @@
  */
  */
 package org.apache.hadoop.hbase;
 package org.apache.hadoop.hbase;
 
 
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.io.UnsupportedEncodingException;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -28,13 +27,13 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
-import java.util.Random;
 import java.util.SortedMap;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeMap;
-import java.util.Vector;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -64,7 +63,7 @@ import org.onelab.filter.RetouchedBloomFilter;
  * Locking and transactions are handled at a higher level.  This API should not 
  * Locking and transactions are handled at a higher level.  This API should not 
  * be called directly by any writer, but rather by an HRegion manager.
  * be called directly by any writer, but rather by an HRegion manager.
  */
  */
-class HStore implements HConstants {
+public class HStore implements HConstants {
   static final Log LOG = LogFactory.getLog(HStore.class);
   static final Log LOG = LogFactory.getLog(HStore.class);
 
 
   /**
   /**
@@ -474,50 +473,51 @@ class HStore implements HConstants {
     }
     }
   }
   }
   
   
-  static final String COMPACTION_TO_REPLACE = "toreplace";    
-  static final String COMPACTION_DONE = "done";
+  /*
+   * Regex that will work for straight filenames and for reference names.
+   * If reference, then the regex has more than just one group.  Group 1 is
+   * this files id.  Group 2 the referenced region name, etc.
+   */
+  private static Pattern REF_NAME_PARSER =
+    Pattern.compile("^(\\d+)(?:\\.(.+))?$");
   
   
   private static final String BLOOMFILTER_FILE_NAME = "filter";
   private static final String BLOOMFILTER_FILE_NAME = "filter";
 
 
   final Memcache memcache = new Memcache();
   final Memcache memcache = new Memcache();
-  Path dir;
-  Text regionName;
-  String encodedRegionName;
-  HColumnDescriptor family;
-  Text familyName;
-  SequenceFile.CompressionType compression;
-  FileSystem fs;
-  HBaseConfiguration conf;
-  Path mapdir;
-  Path loginfodir;
-  Path filterDir;
-  Filter bloomFilter;
-  private String storeName;
+  private final Path basedir;
+  private final HRegionInfo info;
+  private final HColumnDescriptor family;
+  private final SequenceFile.CompressionType compression;
+  final FileSystem fs;
+  private final HBaseConfiguration conf;
+  private final Path filterDir;
+  final Filter bloomFilter;
   private final Path compactionDir;
   private final Path compactionDir;
 
 
-  Integer compactLock = new Integer(0);
-  Integer flushLock = new Integer(0);
+  private final Integer compactLock = new Integer(0);
+  private final Integer flushLock = new Integer(0);
 
 
-  final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
   final AtomicInteger activeScanners = new AtomicInteger(0);
   final AtomicInteger activeScanners = new AtomicInteger(0);
 
 
-  /* Sorted Map of readers keyed by sequence id (Most recent should be last in
+  final String storeName;
+
+  /*
+   * Sorted Map of readers keyed by sequence id (Most recent should be last in
    * in list).
    * in list).
    */
    */
-  SortedMap<Long, HStoreFile> storefiles =
+  final SortedMap<Long, HStoreFile> storefiles =
     Collections.synchronizedSortedMap(new TreeMap<Long, HStoreFile>());
     Collections.synchronizedSortedMap(new TreeMap<Long, HStoreFile>());
   
   
-  /* Sorted Map of readers keyed by sequence id (Most recent should be last in
+  /*
+   * Sorted Map of readers keyed by sequence id (Most recent should be last in
    * in list).
    * in list).
    */
    */
-  TreeMap<Long, MapFile.Reader> readers = new TreeMap<Long, MapFile.Reader>();
+  private final SortedMap<Long, MapFile.Reader> readers =
+    new TreeMap<Long, MapFile.Reader>();
 
 
-  Random rand = new Random();
-  
   private volatile long maxSeqId;
   private volatile long maxSeqId;
-  
   private final int compactionThreshold;
   private final int compactionThreshold;
-  
   private final ReentrantReadWriteLock newScannerLock =
   private final ReentrantReadWriteLock newScannerLock =
     new ReentrantReadWriteLock();
     new ReentrantReadWriteLock();
 
 
@@ -545,70 +545,72 @@ class HStore implements HConstants {
    * <p>It's assumed that after this constructor returns, the reconstructionLog
    * <p>It's assumed that after this constructor returns, the reconstructionLog
    * file will be deleted (by whoever has instantiated the HStore).
    * file will be deleted (by whoever has instantiated the HStore).
    *
    *
-   * @param dir log file directory
-   * @param regionName
-   * @param encodedName
-   * @param family name of column family
+   * @param basedir qualified path under which the region directory lives
+   * @param info HRegionInfo for this region
+   * @param family HColumnDescriptor for this column
    * @param fs file system object
    * @param fs file system object
    * @param reconstructionLog existing log file to apply if any
    * @param reconstructionLog existing log file to apply if any
    * @param conf configuration object
    * @param conf configuration object
    * @throws IOException
    * @throws IOException
    */
    */
-  HStore(Path dir, Text regionName, String encodedName,
-      HColumnDescriptor family, FileSystem fs, Path reconstructionLog,
-      HBaseConfiguration conf) throws IOException {  
+  HStore(Path basedir, HRegionInfo info, HColumnDescriptor family,
+      FileSystem fs, Path reconstructionLog, HBaseConfiguration conf)
+      throws IOException {  
     
     
-    this.dir = dir;
-    this.compactionDir = new Path(HRegion.getRegionDir(dir, encodedName),
-      "compaction.dir");
-    this.regionName = regionName;
-    this.encodedRegionName = encodedName;
+    this.basedir = basedir;
+    this.info = info;
     this.family = family;
     this.family = family;
-    this.familyName = HStoreKey.extractFamily(this.family.getName()).toText();
-    this.compression = SequenceFile.CompressionType.NONE;
-    this.storeName = this.encodedRegionName + "/" + this.familyName.toString();
+    this.fs = fs;
+    this.conf = conf;
     
     
-    if(family.getCompression() != HColumnDescriptor.CompressionType.NONE) {
-      if(family.getCompression() == HColumnDescriptor.CompressionType.BLOCK) {
-        this.compression = SequenceFile.CompressionType.BLOCK;
-      } else if(family.getCompression() ==
-          HColumnDescriptor.CompressionType.RECORD) {
-        this.compression = SequenceFile.CompressionType.RECORD;
-      } else {
-        assert(false);
-      }
+    this.compactionDir = new Path(basedir, "compaction.dir");
+    this.storeName =
+      this.info.getEncodedName() + "/" + this.family.getFamilyName();
+    
+    if (family.getCompression() == HColumnDescriptor.CompressionType.BLOCK) {
+      this.compression = SequenceFile.CompressionType.BLOCK;
+    } else if (family.getCompression() ==
+      HColumnDescriptor.CompressionType.RECORD) {
+      this.compression = SequenceFile.CompressionType.RECORD;
+    } else {
+      this.compression = SequenceFile.CompressionType.NONE;
+    }
+    
+    Path mapdir = HStoreFile.getMapDir(basedir, info.getEncodedName(),
+        family.getFamilyName());
+    if (!fs.exists(mapdir)) {
+      fs.mkdirs(mapdir);
+    }
+    Path infodir = HStoreFile.getInfoDir(basedir, info.getEncodedName(),
+        family.getFamilyName());
+    if (!fs.exists(infodir)) {
+      fs.mkdirs(infodir);
     }
     }
     
     
-    this.fs = fs;
-    this.conf = conf;
-    this.mapdir = HStoreFile.getMapDir(dir, encodedRegionName, familyName);
-    fs.mkdirs(mapdir);
-    this.loginfodir = HStoreFile.getInfoDir(dir, encodedRegionName, familyName);
-    fs.mkdirs(loginfodir);
     if(family.getBloomFilter() == null) {
     if(family.getBloomFilter() == null) {
       this.filterDir = null;
       this.filterDir = null;
       this.bloomFilter = null;
       this.bloomFilter = null;
     } else {
     } else {
-      this.filterDir =
-        HStoreFile.getFilterDir(dir, encodedRegionName, familyName);
-      fs.mkdirs(filterDir);
-      loadOrCreateBloomFilter();
+      this.filterDir = HStoreFile.getFilterDir(basedir, info.getEncodedName(),
+          family.getFamilyName());
+      if (!fs.exists(filterDir)) {
+        fs.mkdirs(filterDir);
+      }
+      this.bloomFilter = loadOrCreateBloomFilter();
     }
     }
 
 
     if(LOG.isDebugEnabled()) {
     if(LOG.isDebugEnabled()) {
-      LOG.debug("starting " + this.regionName + "/" + this.familyName + " ("
-          + this.storeName +
+      LOG.debug("starting " + storeName +
           ((reconstructionLog == null || !fs.exists(reconstructionLog)) ?
           ((reconstructionLog == null || !fs.exists(reconstructionLog)) ?
-          ") (no reconstruction log)": " with reconstruction log: (" +
-          reconstructionLog.toString()));
+          " (no reconstruction log)" :
+            " with reconstruction log: " + reconstructionLog.toString()));
     }
     }
 
 
-    // Go through the 'mapdir' and 'loginfodir' together, make sure that all 
+    // Go through the 'mapdir' and 'infodir' together, make sure that all 
     // MapFiles are in a reliable state.  Every entry in 'mapdir' must have a 
     // MapFiles are in a reliable state.  Every entry in 'mapdir' must have a 
     // corresponding one in 'loginfodir'. Without a corresponding log info
     // corresponding one in 'loginfodir'. Without a corresponding log info
     // file, the entry in 'mapdir' must be deleted.
     // file, the entry in 'mapdir' must be deleted.
-    List<HStoreFile> hstoreFiles = HStoreFile.loadHStoreFiles(conf, dir,
-        encodedRegionName, familyName, fs);
+    List<HStoreFile> hstoreFiles = loadHStoreFiles(infodir, mapdir);
     for(HStoreFile hsf: hstoreFiles) {
     for(HStoreFile hsf: hstoreFiles) {
       this.storefiles.put(Long.valueOf(hsf.loadInfo(fs)), hsf);
       this.storefiles.put(Long.valueOf(hsf.loadInfo(fs)), hsf);
     }
     }
@@ -624,8 +626,8 @@ class HStore implements HConstants {
     
     
     this.maxSeqId = getMaxSequenceId(hstoreFiles);
     this.maxSeqId = getMaxSequenceId(hstoreFiles);
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("maximum sequence id for hstore " + regionName + "/" +
-          familyName + " (" + storeName + ") is " + this.maxSeqId);
+      LOG.debug("maximum sequence id for hstore " + storeName + " is " +
+          this.maxSeqId);
     }
     }
     
     
     doReconstructionLog(reconstructionLog, maxSeqId);
     doReconstructionLog(reconstructionLog, maxSeqId);
@@ -693,14 +695,14 @@ class HStore implements HConstants {
     TreeMap<HStoreKey, byte []> reconstructedCache =
     TreeMap<HStoreKey, byte []> reconstructedCache =
       new TreeMap<HStoreKey, byte []>();
       new TreeMap<HStoreKey, byte []>();
       
       
-    SequenceFile.Reader login = new SequenceFile.Reader(this.fs,
+    SequenceFile.Reader logReader = new SequenceFile.Reader(this.fs,
         reconstructionLog, this.conf);
         reconstructionLog, this.conf);
     
     
     try {
     try {
       HLogKey key = new HLogKey();
       HLogKey key = new HLogKey();
       HLogEdit val = new HLogEdit();
       HLogEdit val = new HLogEdit();
       long skippedEdits = 0;
       long skippedEdits = 0;
-      while (login.next(key, val)) {
+      while (logReader.next(key, val)) {
         maxSeqIdInLog = Math.max(maxSeqIdInLog, key.getLogSeqNum());
         maxSeqIdInLog = Math.max(maxSeqIdInLog, key.getLogSeqNum());
         if (key.getLogSeqNum() <= maxSeqID) {
         if (key.getLogSeqNum() <= maxSeqID) {
           skippedEdits++;
           skippedEdits++;
@@ -714,14 +716,14 @@ class HStore implements HConstants {
         // METACOLUMN info such as HBASE::CACHEFLUSH entries
         // METACOLUMN info such as HBASE::CACHEFLUSH entries
         Text column = val.getColumn();
         Text column = val.getColumn();
         if (column.equals(HLog.METACOLUMN)
         if (column.equals(HLog.METACOLUMN)
-            || !key.getRegionName().equals(regionName)
-            || !HStoreKey.extractFamily(column).equals(this.familyName)) {
+            || !key.getRegionName().equals(info.getRegionName())
+            || !HStoreKey.extractFamily(column).equals(family.getFamilyName())) {
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("Passing on edit " + key.getRegionName() + ", " +
             LOG.debug("Passing on edit " + key.getRegionName() + ", " +
                 column.toString() + ": " + 
                 column.toString() + ": " + 
                 new String(val.getVal(), UTF8_ENCODING) +
                 new String(val.getVal(), UTF8_ENCODING) +
-                ", my region: " + regionName + ", my column: " +
-                this.familyName);
+                ", my region: " + info.getRegionName() + ", my column: " +
+                family.getFamilyName());
           }
           }
           continue;
           continue;
         }
         }
@@ -733,7 +735,7 @@ class HStore implements HConstants {
         reconstructedCache.put(k, val.getVal());
         reconstructedCache.put(k, val.getVal());
       }
       }
     } finally {
     } finally {
-      login.close();
+      logReader.close();
     }
     }
     
     
     if (reconstructedCache.size() > 0) {
     if (reconstructedCache.size() > 0) {
@@ -745,6 +747,76 @@ class HStore implements HConstants {
     }
     }
   }
   }
   
   
+  /*
+   * Creates a series of HStoreFiles loaded from the given directory.
+   * There must be a matching 'mapdir' and 'loginfo' pair of files.
+   * If only one exists, we'll delete it.
+   *
+   * @param infodir qualified path for info file directory
+   * @param mapdir qualified path for map file directory
+   * @throws IOException
+   */
+  private List<HStoreFile> loadHStoreFiles(Path infodir, Path mapdir)
+  throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("infodir: " + infodir.toString() + " mapdir: " +
+          mapdir.toString());
+    }
+    // Look first at info files.  If a reference, these contain info we need
+    // to create the HStoreFile.
+    Path infofiles[] = fs.listPaths(new Path[] {infodir});
+    ArrayList<HStoreFile> results = new ArrayList<HStoreFile>(infofiles.length);
+    ArrayList<Path> mapfiles = new ArrayList<Path>(infofiles.length);
+    for (Path p: infofiles) {
+      Matcher m = REF_NAME_PARSER.matcher(p.getName());
+      /*
+       *  *  *  *  *  N O T E  *  *  *  *  *
+       *  
+       *  We call isReference(Path, Matcher) here because it calls
+       *  Matcher.matches() which must be called before Matcher.group(int)
+       *  and we don't want to call Matcher.matches() twice.
+       *  
+       *  *  *  *  *  N O T E  *  *  *  *  *
+       */
+      boolean isReference = isReference(p, m);
+      long fid = Long.parseLong(m.group(1));
+
+      HStoreFile curfile = null;
+      HStoreFile.Reference reference = null;
+      if (isReference) {
+        reference = readSplitInfo(p, fs);
+      }
+      curfile = new HStoreFile(conf, fs, basedir, info.getEncodedName(),
+          family.getFamilyName(), fid, reference);
+      Path mapfile = curfile.getMapFilePath();
+      if (!fs.exists(mapfile)) {
+        fs.delete(curfile.getInfoFilePath());
+        LOG.warn("Mapfile " + mapfile.toString() + " does not exist. " +
+          "Cleaned up info file.  Continuing...");
+        continue;
+      }
+      
+      // TODO: Confirm referent exists.
+      
+      // Found map and sympathetic info file.  Add this hstorefile to result.
+      results.add(curfile);
+      // Keep list of sympathetic data mapfiles for cleaning info dir in next
+      // section.  Make sure path is fully qualified for compare.
+      mapfiles.add(mapfile);
+    }
+    
+    // List paths by experience returns fully qualified names -- at least when
+    // running on a mini hdfs cluster.
+    Path datfiles[] = fs.listPaths(new Path[] {mapdir});
+    for (int i = 0; i < datfiles.length; i++) {
+      // If does not have sympathetic info file, delete.
+      if (!mapfiles.contains(fs.makeQualified(datfiles[i]))) {
+        fs.delete(datfiles[i]);
+      }
+    }
+    return results;
+  }
+  
   //////////////////////////////////////////////////////////////////////////////
   //////////////////////////////////////////////////////////////////////////////
   // Bloom filters
   // Bloom filters
   //////////////////////////////////////////////////////////////////////////////
   //////////////////////////////////////////////////////////////////////////////
@@ -754,12 +826,12 @@ class HStore implements HConstants {
    * If the HStore already exists, it will read in the bloom filter saved
    * If the HStore already exists, it will read in the bloom filter saved
    * previously. Otherwise, it will create a new bloom filter.
    * previously. Otherwise, it will create a new bloom filter.
    */
    */
-  private void loadOrCreateBloomFilter() throws IOException {
+  private Filter loadOrCreateBloomFilter() throws IOException {
     Path filterFile = new Path(filterDir, BLOOMFILTER_FILE_NAME);
     Path filterFile = new Path(filterDir, BLOOMFILTER_FILE_NAME);
+    Filter bloomFilter = null;
     if(fs.exists(filterFile)) {
     if(fs.exists(filterFile)) {
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("loading bloom filter for " + this.regionName + "/" +
-            this.familyName + " (" + this.storeName + ")");
+        LOG.debug("loading bloom filter for " + this.storeName);
       }
       }
       
       
       BloomFilterDescriptor.BloomFilterType type =
       BloomFilterDescriptor.BloomFilterType type =
@@ -777,6 +849,11 @@ class HStore implements HConstants {
         
         
       case RETOUCHED_BLOOMFILTER:
       case RETOUCHED_BLOOMFILTER:
         bloomFilter = new RetouchedBloomFilter();
         bloomFilter = new RetouchedBloomFilter();
+        break;
+      
+      default:
+        throw new IllegalArgumentException("unknown bloom filter type: " +
+            type);
       }
       }
       FSDataInputStream in = fs.open(filterFile);
       FSDataInputStream in = fs.open(filterFile);
       try {
       try {
@@ -786,8 +863,7 @@ class HStore implements HConstants {
       }
       }
     } else {
     } else {
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("creating bloom filter for " + this.regionName + "/" +
-            this.familyName + " (" + this.storeName + ")");
+        LOG.debug("creating bloom filter for " + this.storeName);
       }
       }
 
 
       BloomFilterDescriptor.BloomFilterType type =
       BloomFilterDescriptor.BloomFilterType type =
@@ -812,6 +888,7 @@ class HStore implements HConstants {
             family.getBloomFilter().nbHash);
             family.getBloomFilter().nbHash);
       }
       }
     }
     }
+    return bloomFilter;
   }
   }
 
 
   /**
   /**
@@ -821,8 +898,7 @@ class HStore implements HConstants {
    */
    */
   private void flushBloomFilter() throws IOException {
   private void flushBloomFilter() throws IOException {
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("flushing bloom filter for " + this.regionName + "/" +
-          this.familyName + " (" + this.storeName + ")");
+      LOG.debug("flushing bloom filter for " + this.storeName);
     }
     }
     FSDataOutputStream out =
     FSDataOutputStream out =
       fs.create(new Path(filterDir, BLOOMFILTER_FILE_NAME));
       fs.create(new Path(filterDir, BLOOMFILTER_FILE_NAME));
@@ -832,8 +908,7 @@ class HStore implements HConstants {
       out.close();
       out.close();
     }
     }
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("flushed bloom filter for " + this.regionName + "/" +
-          this.familyName + " (" + this.storeName + ")");
+      LOG.debug("flushed bloom filter for " + this.storeName);
     }
     }
   }
   }
   
   
@@ -875,8 +950,7 @@ class HStore implements HConstants {
       this.readers.clear();
       this.readers.clear();
       result = new ArrayList<HStoreFile>(storefiles.values());
       result = new ArrayList<HStoreFile>(storefiles.values());
       this.storefiles.clear();
       this.storefiles.clear();
-      LOG.debug("closed " + this.regionName + "/" + this.familyName + " ("
-          + this.storeName + ")");
+      LOG.debug("closed " + this.storeName);
       return result;
       return result;
     } finally {
     } finally {
       this.lock.writeLock().unlock();
       this.lock.writeLock().unlock();
@@ -918,8 +992,8 @@ class HStore implements HConstants {
     
     
     synchronized(flushLock) {
     synchronized(flushLock) {
       // A. Write the Maps out to the disk
       // A. Write the Maps out to the disk
-      HStoreFile flushedFile = HStoreFile.obtainNewHStoreFile(conf, dir,
-        encodedRegionName, familyName, fs);
+      HStoreFile flushedFile = new HStoreFile(conf, fs, basedir,
+        info.getEncodedName(), family.getFamilyName(), -1L, null);
       String name = flushedFile.toString();
       String name = flushedFile.toString();
       MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression,
       MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression,
         this.bloomFilter);
         this.bloomFilter);
@@ -940,7 +1014,7 @@ class HStore implements HConstants {
         for (Map.Entry<HStoreKey, byte []> es: cache.entrySet()) {
         for (Map.Entry<HStoreKey, byte []> es: cache.entrySet()) {
           HStoreKey curkey = es.getKey();
           HStoreKey curkey = es.getKey();
           TextSequence f = HStoreKey.extractFamily(curkey.getColumn());
           TextSequence f = HStoreKey.extractFamily(curkey.getColumn());
-          if (f.equals(this.familyName)) {
+          if (f.equals(this.family.getFamilyName())) {
             entries++;
             entries++;
             out.append(curkey, new ImmutableBytesWritable(es.getValue()));
             out.append(curkey, new ImmutableBytesWritable(es.getValue()));
           }
           }
@@ -970,7 +1044,7 @@ class HStore implements HConstants {
           LOG.debug("Added " + name + " with " + entries +
           LOG.debug("Added " + name + " with " + entries +
             " entries, sequence id " + logCacheFlushId + ", and size " +
             " entries, sequence id " + logCacheFlushId + ", and size " +
             StringUtils.humanReadableInt(flushedFile.length()) + " for " +
             StringUtils.humanReadableInt(flushedFile.length()) + " for " +
-            this.regionName + "/" + this.familyName);
+            this.storeName);
         }
         }
       } finally {
       } finally {
         this.lock.writeLock().unlock();
         this.lock.writeLock().unlock();
@@ -991,7 +1065,7 @@ class HStore implements HConstants {
     if (this.storefiles != null) {
     if (this.storefiles != null) {
       compactionNeeded = this.storefiles.size() >= this.compactionThreshold;
       compactionNeeded = this.storefiles.size() >= this.compactionThreshold;
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("compaction for HStore " + regionName + "/" + familyName +
+        LOG.debug("compaction for HStore " + storeName +
             (compactionNeeded ? " " : " not ") + "needed.");
             (compactionNeeded ? " " : " not ") + "needed.");
       }
       }
     }
     }
@@ -1019,17 +1093,16 @@ class HStore implements HConstants {
    */
    */
   boolean compact() throws IOException {
   boolean compact() throws IOException {
     synchronized (compactLock) {
     synchronized (compactLock) {
-      Path curCompactStore = getCompactionDir();
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("started compaction of " + storefiles.size() +
         LOG.debug("started compaction of " + storefiles.size() +
-          " files using " + curCompactStore.toString() + " for " +
-          this.regionName + "/" + this.familyName);
+          " files using " + compactionDir.toString() + " for " +
+          this.storeName);
       }
       }
-      if (this.fs.exists(curCompactStore)) {
+      if (this.fs.exists(compactionDir)) {
         // Clean out its content in prep. for this new compaction.  Has either
         // Clean out its content in prep. for this new compaction.  Has either
         // aborted previous compaction or it has content of a previous
         // aborted previous compaction or it has content of a previous
         // compaction.
         // compaction.
-        Path [] toRemove = this.fs.listPaths(new Path [] {curCompactStore});
+        Path [] toRemove = this.fs.listPaths(new Path [] {compactionDir});
         for (int i = 0; i < toRemove.length; i++) {
         for (int i = 0; i < toRemove.length; i++) {
           this.fs.delete(toRemove[i]);
           this.fs.delete(toRemove[i]);
         }
         }
@@ -1042,22 +1115,21 @@ class HStore implements HConstants {
       if (filesToCompact.size() < 1 ||
       if (filesToCompact.size() < 1 ||
         (filesToCompact.size() == 1 && !filesToCompact.get(0).isReference())) {
         (filesToCompact.size() == 1 && !filesToCompact.get(0).isReference())) {
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("nothing to compact for " + this.regionName + "/" +
-              this.familyName + " (" + this.storeName + ")");
+          LOG.debug("nothing to compact for " + this.storeName);
         }
         }
         return false;
         return false;
       }
       }
 
 
-      if (!fs.exists(curCompactStore) && !fs.mkdirs(curCompactStore)) {
-        LOG.warn("Mkdir on " + curCompactStore.toString() + " for " +
-            this.regionName + "/" +
-            this.familyName + " failed");
+      if (!fs.exists(compactionDir) && !fs.mkdirs(compactionDir)) {
+        LOG.warn("Mkdir on " + compactionDir.toString() + " for " +
+            this.storeName + " failed");
         return false;
         return false;
       }
       }
 
 
-      // Step through them, writing to the brand-new TreeMap
-      HStoreFile compactedOutputFile = new HStoreFile(conf, this.compactionDir,
-        encodedRegionName, familyName, -1);
+      // Step through them, writing to the brand-new MapFile
+      HStoreFile compactedOutputFile = new HStoreFile(conf, fs, 
+          this.compactionDir, info.getEncodedName(), family.getFamilyName(),
+          -1L, null);
       MapFile.Writer compactedOut = compactedOutputFile.getWriter(this.fs,
       MapFile.Writer compactedOut = compactedOutputFile.getWriter(this.fs,
         this.compression, this.bloomFilter);
         this.compression, this.bloomFilter);
       try {
       try {
@@ -1071,24 +1143,8 @@ class HStore implements HConstants {
       long maxId = getMaxSequenceId(filesToCompact);
       long maxId = getMaxSequenceId(filesToCompact);
       compactedOutputFile.writeInfo(fs, maxId);
       compactedOutputFile.writeInfo(fs, maxId);
 
 
-      // Write out a list of data files that we're replacing
-      Path filesToReplace = new Path(curCompactStore, COMPACTION_TO_REPLACE);
-      FSDataOutputStream out = fs.create(filesToReplace);
-      try {
-        out.writeInt(filesToCompact.size());
-        for (HStoreFile hsf : filesToCompact) {
-          hsf.write(out);
-        }
-      } finally {
-        out.close();
-      }
-
-      // Indicate that we're done.
-      Path doneFile = new Path(curCompactStore, COMPACTION_DONE);
-      fs.create(doneFile).close();
-
       // Move the compaction into place.
       // Move the compaction into place.
-      completeCompaction(curCompactStore);
+      completeCompaction(filesToCompact, compactedOutputFile);
       return true;
       return true;
     }
     }
   }
   }
@@ -1121,7 +1177,7 @@ class HStore implements HConstants {
         // culprit.
         // culprit.
         LOG.warn("Failed with " + e.toString() + ": " + hsf.toString() +
         LOG.warn("Failed with " + e.toString() + ": " + hsf.toString() +
           (hsf.isReference() ? " " + hsf.getReference().toString() : "") +
           (hsf.isReference() ? " " + hsf.getReference().toString() : "") +
-          " for " + this.regionName + "/" + this.familyName);
+          " for " + this.storeName);
         closeCompactionReaders(rdrs);
         closeCompactionReaders(rdrs);
         throw e;
         throw e;
       }
       }
@@ -1221,8 +1277,7 @@ class HStore implements HConstants {
         try {
         try {
           rdrs[i].close();
           rdrs[i].close();
         } catch (IOException e) {
         } catch (IOException e) {
-          LOG.warn("Exception closing reader for " + this.regionName + "/" +
-              this.familyName, e);
+          LOG.warn("Exception closing reader for " + this.storeName, e);
         }
         }
       }
       }
     }
     }
@@ -1348,11 +1403,15 @@ class HStore implements HConstants {
    * 9) Allow new scanners to proceed.
    * 9) Allow new scanners to proceed.
    * </pre>
    * </pre>
    * 
    * 
-   * @param curCompactStore Compaction to complete.
+   * @param compactedFiles list of files that were compacted
+   * @param compactedFile HStoreFile that is the result of the compaction
+   * @throws IOException
    */
    */
-  private void completeCompaction(final Path curCompactStore)
-  throws IOException {
+  private void completeCompaction(List<HStoreFile> compactedFiles,
+      HStoreFile compactedFile) throws IOException {
+    
     // 1. Wait for active scanners to exit
     // 1. Wait for active scanners to exit
+    
     newScannerLock.writeLock().lock();                  // prevent new scanners
     newScannerLock.writeLock().lock();                  // prevent new scanners
     try {
     try {
       synchronized (activeScanners) {
       synchronized (activeScanners) {
@@ -1369,54 +1428,27 @@ class HStore implements HConstants {
       }
       }
 
 
       try {
       try {
-        Path doneFile = new Path(curCompactStore, COMPACTION_DONE);
-        if (!fs.exists(doneFile)) {
-          // The last execution didn't finish the compaction, so there's nothing 
-          // we can do.  We'll just have to redo it. Abandon it and return.
-          LOG.warn("Redo failed compaction (missing 'done' file) for " +
-              this.regionName + "/" + this.familyName);
-          return;
-        }
-
-        // 3. Load in the files to be deleted.
-        Vector<HStoreFile> toCompactFiles = new Vector<HStoreFile>();
-        Path filesToReplace = new Path(curCompactStore, COMPACTION_TO_REPLACE);
-        DataInputStream in = new DataInputStream(fs.open(filesToReplace));
-        try {
-          int numfiles = in.readInt();
-          for(int i = 0; i < numfiles; i++) {
-            HStoreFile hsf = new HStoreFile(conf);
-            hsf.readFields(in);
-            toCompactFiles.add(hsf);
-          }
-        } finally {
-          in.close();
-        }
-
-        // 4. Moving the new MapFile into place.
-        HStoreFile compactedFile = new HStoreFile(conf, this.compactionDir,
-            encodedRegionName, familyName, -1);
-        // obtainNewHStoreFile does its best to generate a filename that does not
-        // currently exist.
-        HStoreFile finalCompactedFile = HStoreFile.obtainNewHStoreFile(conf, dir,
-            encodedRegionName, familyName, fs);
+        // 3. Moving the new MapFile into place.
+        
+        HStoreFile finalCompactedFile = new HStoreFile(conf, fs, basedir,
+            info.getEncodedName(), family.getFamilyName(), -1, null);
         if(LOG.isDebugEnabled()) {
         if(LOG.isDebugEnabled()) {
           LOG.debug("moving " + compactedFile.toString() + " in " +
           LOG.debug("moving " + compactedFile.toString() + " in " +
               this.compactionDir.toString() + " to " +
               this.compactionDir.toString() + " to " +
-              finalCompactedFile.toString() + " in " + dir.toString() +
-              " for " + this.regionName + "/" + this.familyName);
+              finalCompactedFile.toString() + " in " + basedir.toString() +
+              " for " + this.storeName);
         }
         }
         if (!compactedFile.rename(this.fs, finalCompactedFile)) {
         if (!compactedFile.rename(this.fs, finalCompactedFile)) {
           LOG.error("Failed move of compacted file " +
           LOG.error("Failed move of compacted file " +
-              finalCompactedFile.toString() + " for " + this.regionName + "/" +
-              this.familyName);
+              finalCompactedFile.toString() + " for " + this.storeName);
           return;
           return;
         }
         }
 
 
-        // 5. and 6. Unload all the replaced MapFiles, close and delete.
-        Vector<Long> toDelete = new Vector<Long>(toCompactFiles.size());
+        // 4. and 5. Unload all the replaced MapFiles, close and delete.
+        
+        List<Long> toDelete = new ArrayList<Long>();
         for (Map.Entry<Long, HStoreFile> e: this.storefiles.entrySet()) {
         for (Map.Entry<Long, HStoreFile> e: this.storefiles.entrySet()) {
-          if (!toCompactFiles.contains(e.getValue())) {
+          if (!compactedFiles.contains(e.getValue())) {
             continue;
             continue;
           }
           }
           Long key = e.getKey();
           Long key = e.getKey();
@@ -1433,24 +1465,24 @@ class HStore implements HConstants {
             hsf.delete();
             hsf.delete();
           }
           }
 
 
-          // 7. Loading the new TreeMap.
+          // 6. Loading the new TreeMap.
           Long orderVal = Long.valueOf(finalCompactedFile.loadInfo(fs));
           Long orderVal = Long.valueOf(finalCompactedFile.loadInfo(fs));
           this.readers.put(orderVal,
           this.readers.put(orderVal,
             finalCompactedFile.getReader(this.fs, this.bloomFilter));
             finalCompactedFile.getReader(this.fs, this.bloomFilter));
           this.storefiles.put(orderVal, finalCompactedFile);
           this.storefiles.put(orderVal, finalCompactedFile);
         } catch (IOException e) {
         } catch (IOException e) {
-          LOG.error("Failed replacing compacted files for " +
-              this.regionName + "/" + this.familyName + ". Compacted file is " +
-              finalCompactedFile.toString() + ".  Files replaced are " +
-              toCompactFiles.toString() +
+          e = RemoteExceptionHandler.checkIOException(e);
+          LOG.error("Failed replacing compacted files for " + this.storeName +
+              ". Compacted file is " + finalCompactedFile.toString() +
+              ".  Files replaced are " + compactedFiles.toString() +
               " some of which may have been already removed", e);
               " some of which may have been already removed", e);
         }
         }
       } finally {
       } finally {
-        // 8. Releasing the write-lock
+        // 7. Releasing the write-lock
         this.lock.writeLock().unlock();
         this.lock.writeLock().unlock();
       }
       }
     } finally {
     } finally {
-      // 9. Allow new scanners to proceed.
+      // 8. Allow new scanners to proceed.
       newScannerLock.writeLock().unlock();
       newScannerLock.writeLock().unlock();
     }
     }
   }
   }
@@ -1505,18 +1537,7 @@ class HStore implements HConstants {
     }
     }
   }
   }
   
   
-  /*
-   * @return Path to the compaction directory for this column family.
-   * Compaction dir is a subdirectory of the region.  Needs to have the
-   * same regiondir/storefamily path prefix; HStoreFile constructor presumes
-   * it (TODO: Fix).
-   */
-  private Path getCompactionDir() {
-    return HStoreFile.getHStoreDir(this.compactionDir,
-      this.encodedRegionName, this.familyName);
-  }
-  
-  private MapFile.Reader [] getReaders() {
+  MapFile.Reader [] getReaders() {
     return this.readers.values().
     return this.readers.values().
       toArray(new MapFile.Reader[this.readers.size()]);
       toArray(new MapFile.Reader[this.readers.size()]);
   }
   }
@@ -1796,8 +1817,7 @@ class HStore implements HConstants {
         midKey.set(((HStoreKey)midkey).getRow());
         midKey.set(((HStoreKey)midkey).getRow());
       }
       }
     } catch(IOException e) {
     } catch(IOException e) {
-      LOG.warn("Failed getting store size for " + this.regionName + "/" +
-          this.familyName, e);
+      LOG.warn("Failed getting store size for " + this.storeName, e);
     } finally {
     } finally {
       this.lock.readLock().unlock();
       this.lock.readLock().unlock();
     }
     }
@@ -1835,6 +1855,38 @@ class HStore implements HConstants {
     return this.storeName;
     return this.storeName;
   }
   }
 
 
+  /*
+   * @see writeSplitInfo(Path p, HStoreFile hsf, FileSystem fs)
+   */
+  static HStoreFile.Reference readSplitInfo(final Path p, final FileSystem fs)
+  throws IOException {
+    FSDataInputStream in = fs.open(p);
+    try {
+      HStoreFile.Reference r = new HStoreFile.Reference();
+      r.readFields(in);
+      return r;
+    } finally {
+      in.close();
+    }
+  }
+
+  /**
+   * @param p Path to check.
+   * @return True if the path has format of a HStoreFile reference.
+   */
+  public static boolean isReference(final Path p) {
+    return isReference(p, REF_NAME_PARSER.matcher(p.getName()));
+  }
+ 
+  private static boolean isReference(final Path p, final Matcher m) {
+    if (m == null || !m.matches()) {
+      LOG.warn("Failed match of store file name " + p.toString());
+      throw new RuntimeException("Failed match of store file name " +
+          p.toString());
+    }
+    return m.groupCount() > 1 && m.group(2) != null;
+  }
+
   /**
   /**
    * A scanner that iterates through the HStore files
    * A scanner that iterates through the HStore files
    */
    */
@@ -1939,7 +1991,7 @@ class HStore implements HConstants {
           try {
           try {
             readers[i].close();
             readers[i].close();
           } catch(IOException e) {
           } catch(IOException e) {
-            LOG.error(regionName + "/" + familyName + " closing sub-scanner", e);
+            LOG.error(storeName + " closing sub-scanner", e);
           }
           }
         }
         }
         
         
@@ -1959,7 +2011,7 @@ class HStore implements HConstants {
               try {
               try {
                 readers[i].close();
                 readers[i].close();
               } catch(IOException e) {
               } catch(IOException e) {
-                LOG.error(regionName + "/" + familyName + " closing scanner", e);
+                LOG.error(storeName + " closing scanner", e);
               }
               }
             }
             }
           }
           }
@@ -2195,8 +2247,7 @@ class HStore implements HConstants {
         try {
         try {
           scanners[i].close();
           scanners[i].close();
         } catch (IOException e) {
         } catch (IOException e) {
-          LOG.warn(regionName + "/" + familyName + " failed closing scanner "
-              + i, e);
+          LOG.warn(storeName + " failed closing scanner " + i, e);
         }
         }
       } finally {
       } finally {
         scanners[i] = null;
         scanners[i] = null;
@@ -2217,7 +2268,7 @@ class HStore implements HConstants {
         synchronized (activeScanners) {
         synchronized (activeScanners) {
           int numberOfScanners = activeScanners.decrementAndGet();
           int numberOfScanners = activeScanners.decrementAndGet();
           if (numberOfScanners < 0) {
           if (numberOfScanners < 0) {
-            LOG.error(regionName + "/" + familyName +
+            LOG.error(storeName +
                 " number of active scanners less than zero: " +
                 " number of active scanners less than zero: " +
                 numberOfScanners + " resetting to zero");
                 numberOfScanners + " resetting to zero");
             activeScanners.set(0);
             activeScanners.set(0);

+ 291 - 482
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java

@@ -25,16 +25,12 @@ import java.io.DataOutput;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 import java.util.Random;
 import java.util.Random;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -89,7 +85,7 @@ import org.onelab.filter.Key;
  * <p>When merging or splitting HRegions, we might want to modify one of the 
  * <p>When merging or splitting HRegions, we might want to modify one of the 
  * params for an HStoreFile (effectively moving it elsewhere).
  * params for an HStoreFile (effectively moving it elsewhere).
  */
  */
-public class HStoreFile implements HConstants, WritableComparable {
+public class HStoreFile implements HConstants {
   static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
   static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
   static final byte INFO_SEQ_NUM = 0;
   static final byte INFO_SEQ_NUM = 0;
   static final String HSTORE_DATFILE_DIR = "mapfiles";
   static final String HSTORE_DATFILE_DIR = "mapfiles";
@@ -107,344 +103,115 @@ public class HStoreFile implements HConstants, WritableComparable {
     bottom
     bottom
   }
   }
   
   
-  /*
-   * Regex that will work for straight filenames and for reference names.
-   * If reference, then the regex has more than just one group.  Group 1 is
-   * this files id.  Group 2 the referenced region name, etc.
-   */
-  private static Pattern REF_NAME_PARSER =
-    Pattern.compile("^(\\d+)(?:\\.(.+))?$");
-  
-  private static Random rand = new Random();
+  private final static Random rand = new Random();
 
 
-  private Path dir;
-  private String encodedRegionName;
-  private Text colFamily;
-  private long fileId;
+  private final Path basedir;
+  private final String encodedRegionName;
+  private final Text colFamily;
+  private final long fileId;
   private final HBaseConfiguration conf;
   private final HBaseConfiguration conf;
-  private Reference reference;
-
-  /** Shutdown constructor used by Writable */
-  HStoreFile(HBaseConfiguration conf) {
-    this(conf, new Path(Path.CUR_DIR), "", new Text(), 0);
-  }
-  
-  /**
-   * Constructor that fully initializes the object
-   * @param conf Configuration object
-   * @param dir directory path
-   * @param encodedRegionName name of the region
-   * @param colFamily name of the column family
-   * @param fileId file identifier
-   */
-  HStoreFile(final HBaseConfiguration conf, final Path dir, 
-      final String encodedRegionName, final Text colFamily, final long fileId) {
-    this(conf, dir, encodedRegionName, colFamily, fileId, null);
-  }
+  private final FileSystem fs;
+  private final Reference reference;
 
 
   /**
   /**
    * Constructor that fully initializes the object
    * Constructor that fully initializes the object
    * @param conf Configuration object
    * @param conf Configuration object
-   * @param dir directory path
+   * @param basedir qualified path that is parent of region directory
    * @param encodedRegionName file name friendly name of the region
    * @param encodedRegionName file name friendly name of the region
    * @param colFamily name of the column family
    * @param colFamily name of the column family
    * @param fileId file identifier
    * @param fileId file identifier
    * @param ref Reference to another HStoreFile.
    * @param ref Reference to another HStoreFile.
+   * @throws IOException
    */
    */
-  HStoreFile(HBaseConfiguration conf, Path dir, String encodedRegionName, 
-      Text colFamily, long fileId, final Reference ref) {
+  HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir,
+      String encodedRegionName, Text colFamily, long fileId,
+      final Reference ref) throws IOException {
     this.conf = conf;
     this.conf = conf;
-    this.dir = dir;
+    this.fs = fs;
+    this.basedir = basedir;
     this.encodedRegionName = encodedRegionName;
     this.encodedRegionName = encodedRegionName;
     this.colFamily = new Text(colFamily);
     this.colFamily = new Text(colFamily);
-    this.fileId = fileId;
+    
+    long id = fileId;
+    if (id == -1) {
+      Path mapdir = HStoreFile.getMapDir(basedir, encodedRegionName, colFamily);
+      Path testpath = null;
+      do {
+        id = Math.abs(rand.nextLong());
+        testpath = new Path(mapdir, createHStoreFilename(id, null));
+      } while(fs.exists(testpath));
+    }
+    this.fileId = id;
+    
     // If a reference, construction does not write the pointer files.  Thats
     // If a reference, construction does not write the pointer files.  Thats
     // done by invocations of writeReferenceFiles(hsf, fs).  Happens at fast
     // done by invocations of writeReferenceFiles(hsf, fs).  Happens at fast
     // split time.
     // split time.
     this.reference = ref;
     this.reference = ref;
   }
   }
 
 
-  /*
-   * Data structure to hold reference to a store file over in another region.
-   */
-  static class Reference implements Writable {
-    private String encodedRegionName;
-    private long fileid;
-    private Range region;
-    private HStoreKey midkey;
-    
-    Reference(final String ern, final long fid, final HStoreKey m,
-        final Range fr) {
-      this.encodedRegionName = ern;
-      this.fileid = fid;
-      this.region = fr;
-      this.midkey = m;
-    }
-    
-    Reference() {
-      this(null, -1, null, Range.bottom);
-    }
-
-    long getFileId() {
-      return this.fileid;
-    }
-
-    Range getFileRegion() {
-      return this.region;
-    }
-    
-    HStoreKey getMidkey() {
-      return this.midkey;
-    }
-    
-    String getEncodedRegionName() {
-      return this.encodedRegionName;
-    }
-   
-    /** {@inheritDoc} */
-    @Override
-    public String toString() {
-      return this.encodedRegionName + "/" + this.fileid + "/" + this.region;
-    }
-
-    // Make it serializable.
-
-    /** {@inheritDoc} */
-    public void write(DataOutput out) throws IOException {
-      out.writeUTF(this.encodedRegionName);
-      out.writeLong(this.fileid);
-      // Write true if we're doing top of the file.
-      out.writeBoolean(isTopFileRegion(this.region));
-      this.midkey.write(out);
-    }
-
-    /** {@inheritDoc} */
-    public void readFields(DataInput in) throws IOException {
-      this.encodedRegionName = in.readUTF();
-      this.fileid = in.readLong();
-      boolean tmp = in.readBoolean();
-      // If true, set region to top.
-      this.region = tmp? Range.top: Range.bottom;
-      this.midkey = new HStoreKey();
-      this.midkey.readFields(in);
-    }
-  }
-
-  static boolean isTopFileRegion(final Range r) {
-    return r.equals(Range.top);
-  }
-
   /** @return the region name */
   /** @return the region name */
   boolean isReference() {
   boolean isReference() {
-    return this.reference != null;
+    return reference != null;
   }
   }
   
   
   Reference getReference() {
   Reference getReference() {
-    return this.reference;
+    return reference;
   }
   }
 
 
   String getEncodedRegionName() {
   String getEncodedRegionName() {
-    return this.encodedRegionName;
+    return encodedRegionName;
   }
   }
 
 
   /** @return the column family */
   /** @return the column family */
   Text getColFamily() {
   Text getColFamily() {
-    return this.colFamily;
+    return colFamily;
   }
   }
 
 
   /** @return the file identifier */
   /** @return the file identifier */
   long getFileId() {
   long getFileId() {
-    return this.fileId;
+    return fileId;
   }
   }
 
 
   // Build full filenames from those components
   // Build full filenames from those components
+  
   /** @return path for MapFile */
   /** @return path for MapFile */
   Path getMapFilePath() {
   Path getMapFilePath() {
-    return isReference()?
-      getMapFilePath(this.encodedRegionName, this.fileId,
-        this.reference.getEncodedRegionName()):
-      getMapFilePath(this.encodedRegionName, this.fileId);
+    if (isReference()) {
+      return getMapFilePath(encodedRegionName, fileId,
+          reference.getEncodedRegionName());
+    }
+    return getMapFilePath(encodedRegionName, fileId, null);
   }
   }
 
 
   private Path getMapFilePath(final Reference r) {
   private Path getMapFilePath(final Reference r) {
-    return r == null?
-      getMapFilePath():
-      getMapFilePath(r.getEncodedRegionName(), r.getFileId());
+    if (r == null) {
+      return getMapFilePath();
+    }
+    return getMapFilePath(r.getEncodedRegionName(), r.getFileId(), null);
   }
   }
 
 
-  private Path getMapFilePath(final String encodedName, final long fid) {
-    return new Path(HStoreFile.getMapDir(dir, encodedName, colFamily), 
-      createHStoreFilename(fid, null));
-  }
-  
   private Path getMapFilePath(final String encodedName, final long fid,
   private Path getMapFilePath(final String encodedName, final long fid,
       final String ern) {
       final String ern) {
-    return new Path(HStoreFile.getMapDir(dir, encodedName, colFamily), 
+    return new Path(HStoreFile.getMapDir(basedir, encodedName, colFamily), 
       createHStoreFilename(fid, ern));
       createHStoreFilename(fid, ern));
   }
   }
 
 
   /** @return path for info file */
   /** @return path for info file */
   Path getInfoFilePath() {
   Path getInfoFilePath() {
-    return isReference()?
-      getInfoFilePath(this.encodedRegionName, this.fileId,
-        this.reference.getEncodedRegionName()):
-      getInfoFilePath(this.encodedRegionName, this.fileId);
-  }
-  
-  private Path getInfoFilePath(final String encodedName, final long fid) {
-    return new Path(HStoreFile.getInfoDir(dir, encodedName, colFamily), 
-      createHStoreFilename(fid, null));
+    if (isReference()) {
+      return getInfoFilePath(encodedRegionName, fileId,
+          reference.getEncodedRegionName());
+ 
+    }
+    return getInfoFilePath(encodedRegionName, fileId, null);
   }
   }
   
   
   private Path getInfoFilePath(final String encodedName, final long fid,
   private Path getInfoFilePath(final String encodedName, final long fid,
       final String ern) {
       final String ern) {
-    return new Path(HStoreFile.getInfoDir(dir, encodedName, colFamily), 
+    return new Path(HStoreFile.getInfoDir(basedir, encodedName, colFamily), 
       createHStoreFilename(fid, ern));
       createHStoreFilename(fid, ern));
   }
   }
 
 
-  // Static methods to build partial paths to internal directories.  Useful for 
-  // HStore construction and log-rebuilding.
-  private static String createHStoreFilename(final long fid) {
-    return createHStoreFilename(fid, null);
-  }
-  
-  private static String createHStoreFilename(final long fid,
-      final String encodedRegionName) {
-    return Long.toString(fid) +
-      ((encodedRegionName != null) ? "." + encodedRegionName : "");
-  }
-  
-  private static String createHStoreInfoFilename(final long fid) {
-    return createHStoreFilename(fid, null);
-  }
-  
-  static Path getMapDir(Path dir, String encodedRegionName, Text colFamily) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName, 
-        new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
-  }
-
-  /** @return the info directory path */
-  static Path getInfoDir(Path dir, String encodedRegionName, Text colFamily) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName, 
-        new Path(colFamily.toString(), HSTORE_INFO_DIR)));
-  }
-
-  /** @return the bloom filter directory path */
-  static Path getFilterDir(Path dir, String encodedRegionName, Text colFamily) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName,
-        new Path(colFamily.toString(), HSTORE_FILTER_DIR)));
-  }
-
-  /** @return the HStore directory path */
-  static Path getHStoreDir(Path dir, String encodedRegionName, Text colFamily) {
-    return new Path(dir, new Path(HREGIONDIR_PREFIX + encodedRegionName, 
-        colFamily.toString()));
-  }
-
-  /**
-   * @return a brand-new randomly-named HStoreFile.
-   * 
-   * Checks the filesystem to determine if the file already exists. If so, it
-   * will keep generating names until it generates a name that does not exist.
-   */
-  static HStoreFile obtainNewHStoreFile(HBaseConfiguration conf, Path dir, 
-      String encodedRegionName, Text colFamily, FileSystem fs)
-      throws IOException {
-    Path mapdir = HStoreFile.getMapDir(dir, encodedRegionName, colFamily);
-    Path testpath1 = null;
-    Path testpath2 = null;
-    long fileId = -1;
-    do {
-      fileId = Math.abs(rand.nextLong());
-      testpath1 = new Path(mapdir, createHStoreFilename(fileId));
-      testpath2 = new Path(mapdir, createHStoreInfoFilename(fileId));
-    } while(fs.exists(testpath1) || fs.exists(testpath2));
-    return new HStoreFile(conf, dir, encodedRegionName, colFamily, fileId);
-  }
-
-  /*
-   * Creates a series of HStoreFiles loaded from the given directory.
-   * There must be a matching 'mapdir' and 'loginfo' pair of files.
-   * If only one exists, we'll delete it.
-   *
-   * @param conf Configuration object
-   * @param dir directory path
-   * @param regionName region name
-   * @param colFamily column family
-   * @param fs file system
-   * @return List of store file instances loaded from passed dir.
-   * @throws IOException
-   */
-  static List<HStoreFile> loadHStoreFiles(HBaseConfiguration conf, Path dir, 
-      String encodedRegionName, Text colFamily, FileSystem fs)
-  throws IOException {
-    // Look first at info files.  If a reference, these contain info we need
-    // to create the HStoreFile.
-    Path infodir = HStoreFile.getInfoDir(dir, encodedRegionName, colFamily);
-    Path infofiles[] = fs.listPaths(new Path[] {infodir});
-    ArrayList<HStoreFile> results = new ArrayList<HStoreFile>(infofiles.length);
-    ArrayList<Path> mapfiles = new ArrayList<Path>(infofiles.length);
-    for (int i = 0; i < infofiles.length; i++) {
-      Path p = infofiles[i];
-      Matcher m = REF_NAME_PARSER.matcher(p.getName());
-      boolean isReference =  isReference(p, m);
-      long fid = Long.parseLong(m.group(1));
-      HStoreFile curfile = null;
-      if (isReference) {
-        Reference reference = readSplitInfo(infofiles[i], fs);
-        curfile = new HStoreFile(conf, dir, encodedRegionName, colFamily, fid,
-          reference);
-      } else {
-        curfile = new HStoreFile(conf, dir, encodedRegionName, colFamily, fid);
-      }
-      Path mapfile = curfile.getMapFilePath();
-      if (!fs.exists(mapfile)) {
-        fs.delete(curfile.getInfoFilePath());
-        LOG.warn("Mapfile " + mapfile.toString() + " does not exist. " +
-          "Cleaned up info file.  Continuing...");
-        continue;
-      }
-      
-      // TODO: Confirm referent exists.
-      
-      // Found map and sympathetic info file.  Add this hstorefile to result.
-      results.add(curfile);
-      // Keep list of sympathetic data mapfiles for cleaning info dir in next
-      // section.  Make sure path is fully qualified for compare.
-      Path qualified = fs.makeQualified(mapfile);
-      mapfiles.add(qualified);
-    }
-    
-    Path mapdir = HStoreFile.getMapDir(dir, encodedRegionName, colFamily);
-    // List paths by experience returns fully qualified names -- at least when
-    // running on a mini hdfs cluster.
-    Path datfiles[] = fs.listPaths(new Path[] {mapdir});
-    for (int i = 0; i < datfiles.length; i++) {
-      // If does not have sympathetic info file, delete.
-      if (!mapfiles.contains(fs.makeQualified(datfiles[i]))) {
-        fs.delete(datfiles[i]);
-      }
-    }
-    return results;
-  }
-  
-  /**
-   * @param p Path to check.
-   * @return True if the path has format of a HStoreFile reference.
-   */
-  static boolean isReference(final Path p) {
-    return isReference(p, REF_NAME_PARSER.matcher(p.getName()));
-  }
- 
-  private static boolean isReference(final Path p, final Matcher m) {
-    if (m == null || !m.matches()) {
-      LOG.warn("Failed match of store file name " + p.toString());
-      throw new RuntimeException("Failed match of store file name " +
-          p.toString());
-    }
-    return m.groupCount() > 1 && m.group(2) != null;
-  }
-
   // File handling
   // File handling
 
 
   /*
   /*
@@ -499,21 +266,6 @@ public class HStoreFile implements HConstants, WritableComparable {
    }
    }
   }
   }
   
   
-  /*
-   * @see writeSplitInfo(Path p, HStoreFile hsf, FileSystem fs)
-   */
-  static Reference readSplitInfo(final Path p, final FileSystem fs)
-  throws IOException {
-    FSDataInputStream in = fs.open(p);
-    try {
-      Reference r = new Reference();
-      r.readFields(in);
-      return r;
-    } finally {
-      in.close();
-    }
-  }
-
   private void createOrFail(final FileSystem fs, final Path p)
   private void createOrFail(final FileSystem fs, final Path p)
   throws IOException {
   throws IOException {
     if (fs.exists(p)) {
     if (fs.exists(p)) {
@@ -577,10 +329,13 @@ public class HStoreFile implements HConstants, WritableComparable {
    * @throws IOException
    * @throws IOException
    */
    */
   long loadInfo(FileSystem fs) throws IOException {
   long loadInfo(FileSystem fs) throws IOException {
-    Path p = isReference() ?
-        getInfoFilePath(this.reference.getEncodedRegionName(),
-            this.reference.getFileId()) :
-              getInfoFilePath();
+    Path p = null;
+    if (isReference()) {
+      p = getInfoFilePath(reference.getEncodedRegionName(),
+          reference.getFileId(), null);
+    } else {
+      p = getInfoFilePath();
+    }
     DataInputStream in = new DataInputStream(fs.open(p));
     DataInputStream in = new DataInputStream(fs.open(p));
     try {
     try {
       byte flag = in.readByte();
       byte flag = in.readByte();
@@ -616,12 +371,8 @@ public class HStoreFile implements HConstants, WritableComparable {
    * @throws IOException 
    * @throws IOException 
    */
    */
   public void delete() throws IOException {
   public void delete() throws IOException {
-    delete(getMapFilePath());
-    delete(getInfoFilePath());
-  }
-  
-  private void delete(final Path p) throws IOException {
-    p.getFileSystem(this.conf).delete(p);
+    fs.delete(getMapFilePath());
+    fs.delete(getInfoFilePath());
   }
   }
   
   
   /**
   /**
@@ -654,12 +405,191 @@ public class HStoreFile implements HConstants, WritableComparable {
     return success;
     return success;
   }
   }
   
   
+  /**
+   * Get reader for the store file map file.
+   * Client is responsible for closing file when done.
+   * @param fs
+   * @param bloomFilter If null, no filtering is done.
+   * @return MapFile.Reader
+   * @throws IOException
+   */
+  public synchronized MapFile.Reader getReader(final FileSystem fs,
+      final Filter bloomFilter) throws IOException {
+    
+    if (isReference()) {
+      return new HStoreFile.HalfMapFileReader(fs,
+          getMapFilePath(reference).toString(), conf, 
+          reference.getFileRegion(), reference.getMidkey(), bloomFilter);
+    }
+    return new BloomFilterMapFile.Reader(fs, getMapFilePath().toString(),
+        conf, bloomFilter);
+  }
+
+  /**
+   * Get a store file writer.
+   * Client is responsible for closing file when done.
+   * @param fs
+   * @param compression Pass <code>SequenceFile.CompressionType.NONE</code>
+   * for none.
+   * @param bloomFilter If null, no filtering is done.
+   * @return MapFile.Writer
+   * @throws IOException
+   */
+  public MapFile.Writer getWriter(final FileSystem fs,
+      final SequenceFile.CompressionType compression,
+      final Filter bloomFilter)
+  throws IOException {
+    if (isReference()) {
+      throw new IOException("Illegal Access: Cannot get a writer on a" +
+        "HStoreFile reference");
+    }
+    return new BloomFilterMapFile.Writer(conf, fs,
+      getMapFilePath().toString(), HStoreKey.class,
+      ImmutableBytesWritable.class, compression, bloomFilter);
+  }
+
+  /**
+   * @return Length of the store map file.  If a reference, size is
+   * approximation.
+   * @throws IOException
+   */
+  public long length() throws IOException {
+    Path p = new Path(getMapFilePath(reference), MapFile.DATA_FILE_NAME);
+    long l = p.getFileSystem(conf).getFileStatus(p).getLen();
+    return (isReference())? l / 2: l;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public String toString() {
+    return encodedRegionName + "/" + colFamily + "/" + fileId +
+      (isReference()? "/" + reference.toString(): "");
+  }
+  
+  /**
+   * Custom bloom filter key maker.
+   * @param key
+   * @return Key made of bytes of row and column only.
+   * @throws IOException
+   */
+  static Key getBloomFilterKey(WritableComparable key)
+  throws IOException {
+    HStoreKey hsk = (HStoreKey)key;
+    byte [] bytes = null;
+    try {
+      bytes = (hsk.getRow().toString() + hsk.getColumn().toString()).
+        getBytes(UTF8_ENCODING);
+    } catch (UnsupportedEncodingException e) {
+      throw new IOException(e.toString());
+    }
+    return new Key(bytes);
+  }
+
+  static boolean isTopFileRegion(final Range r) {
+    return r.equals(Range.top);
+  }
+
+  private static String createHStoreFilename(final long fid,
+      final String encodedRegionName) {
+    return Long.toString(fid) +
+      ((encodedRegionName != null) ? "." + encodedRegionName : "");
+  }
+  
+  static Path getMapDir(Path dir, String encodedRegionName, Text colFamily) {
+    return new Path(dir, new Path(encodedRegionName, 
+        new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
+  }
+
+  /** @return the info directory path */
+  static Path getInfoDir(Path dir, String encodedRegionName, Text colFamily) {
+    return new Path(dir, new Path(encodedRegionName, 
+        new Path(colFamily.toString(), HSTORE_INFO_DIR)));
+  }
+
+  /** @return the bloom filter directory path */
+  static Path getFilterDir(Path dir, String encodedRegionName, Text colFamily) {
+    return new Path(dir, new Path(encodedRegionName,
+        new Path(colFamily.toString(), HSTORE_FILTER_DIR)));
+  }
+
+  /*
+   * Data structure to hold reference to a store file over in another region.
+   */
+  static class Reference implements Writable {
+    private String encodedRegionName;
+    private long fileid;
+    private Range region;
+    private HStoreKey midkey;
+    
+    Reference(final String ern, final long fid, final HStoreKey m,
+        final Range fr) {
+      this.encodedRegionName = ern;
+      this.fileid = fid;
+      this.region = fr;
+      this.midkey = m;
+    }
+    
+    Reference() {
+      this(null, -1, null, Range.bottom);
+    }
+
+    long getFileId() {
+      return fileid;
+    }
+
+    Range getFileRegion() {
+      return region;
+    }
+    
+    HStoreKey getMidkey() {
+      return midkey;
+    }
+    
+    String getEncodedRegionName() {
+      return encodedRegionName;
+    }
+   
+    /** {@inheritDoc} */
+    @Override
+    public String toString() {
+      return encodedRegionName + "/" + fileid + "/" + region;
+    }
+
+    // Make it serializable.
+
+    /** {@inheritDoc} */
+    public void write(DataOutput out) throws IOException {
+      out.writeUTF(encodedRegionName);
+      out.writeLong(fileid);
+      // Write true if we're doing top of the file.
+      out.writeBoolean(isTopFileRegion(region));
+      midkey.write(out);
+    }
+
+    /** {@inheritDoc} */
+    public void readFields(DataInput in) throws IOException {
+      encodedRegionName = in.readUTF();
+      fileid = in.readLong();
+      boolean tmp = in.readBoolean();
+      // If true, set region to top.
+      region = tmp? Range.top: Range.bottom;
+      midkey = new HStoreKey();
+      midkey.readFields(in);
+    }
+  }
+
   /**
   /**
    * Hbase customizations of MapFile.
    * Hbase customizations of MapFile.
    */
    */
   static class HbaseMapFile extends MapFile {
   static class HbaseMapFile extends MapFile {
 
 
     static class HbaseReader extends MapFile.Reader {
     static class HbaseReader extends MapFile.Reader {
+      /**
+       * @param fs
+       * @param dirName
+       * @param conf
+       * @throws IOException
+       */
       public HbaseReader(FileSystem fs, String dirName, Configuration conf)
       public HbaseReader(FileSystem fs, String dirName, Configuration conf)
       throws IOException {
       throws IOException {
         super(fs, dirName, conf);
         super(fs, dirName, conf);
@@ -676,6 +606,15 @@ public class HStoreFile implements HConstants, WritableComparable {
     }
     }
     
     
     static class HbaseWriter extends MapFile.Writer {
     static class HbaseWriter extends MapFile.Writer {
+      /**
+       * @param conf
+       * @param fs
+       * @param dirName
+       * @param keyClass
+       * @param valClass
+       * @param compression
+       * @throws IOException
+       */
       public HbaseWriter(Configuration conf, FileSystem fs, String dirName,
       public HbaseWriter(Configuration conf, FileSystem fs, String dirName,
           Class<Writable> keyClass, Class<Writable> valClass,
           Class<Writable> keyClass, Class<Writable> valClass,
           SequenceFile.CompressionType compression)
           SequenceFile.CompressionType compression)
@@ -698,20 +637,28 @@ public class HStoreFile implements HConstants, WritableComparable {
     static class Reader extends HbaseReader {
     static class Reader extends HbaseReader {
       private final Filter bloomFilter;
       private final Filter bloomFilter;
 
 
+      /**
+       * @param fs
+       * @param dirName
+       * @param conf
+       * @param filter
+       * @throws IOException
+       */
       public Reader(FileSystem fs, String dirName, Configuration conf,
       public Reader(FileSystem fs, String dirName, Configuration conf,
           final Filter filter)
           final Filter filter)
       throws IOException {
       throws IOException {
         super(fs, dirName, conf);
         super(fs, dirName, conf);
-        this.bloomFilter = filter;
+        bloomFilter = filter;
       }
       }
       
       
+      /** {@inheritDoc} */
       @Override
       @Override
       public Writable get(WritableComparable key, Writable val)
       public Writable get(WritableComparable key, Writable val)
       throws IOException {
       throws IOException {
-        if (this.bloomFilter == null) {
+        if (bloomFilter == null) {
           return super.get(key, val);
           return super.get(key, val);
         }
         }
-        if(this.bloomFilter.membershipTest(getBloomFilterKey(key))) {
+        if(bloomFilter.membershipTest(getBloomFilterKey(key))) {
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("bloom filter reported that key exists");
             LOG.debug("bloom filter reported that key exists");
           }
           }
@@ -723,15 +670,15 @@ public class HStoreFile implements HConstants, WritableComparable {
         return null;
         return null;
       }
       }
 
 
+      /** {@inheritDoc} */
       @Override
       @Override
       public WritableComparable getClosest(WritableComparable key,
       public WritableComparable getClosest(WritableComparable key,
-          Writable val)
-      throws IOException {
-        if (this.bloomFilter == null) {
+          Writable val) throws IOException {
+        if (bloomFilter == null) {
           return super.getClosest(key, val);
           return super.getClosest(key, val);
         }
         }
         // Note - the key being passed to us is always a HStoreKey
         // Note - the key being passed to us is always a HStoreKey
-        if(this.bloomFilter.membershipTest(getBloomFilterKey(key))) {
+        if(bloomFilter.membershipTest(getBloomFilterKey(key))) {
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("bloom filter reported that key exists");
             LOG.debug("bloom filter reported that key exists");
           }
           }
@@ -747,89 +694,37 @@ public class HStoreFile implements HConstants, WritableComparable {
     static class Writer extends HbaseWriter {
     static class Writer extends HbaseWriter {
       private final Filter bloomFilter;
       private final Filter bloomFilter;
       
       
+      /**
+       * @param conf
+       * @param fs
+       * @param dirName
+       * @param keyClass
+       * @param valClass
+       * @param compression
+       * @param filter
+       * @throws IOException
+       */
       @SuppressWarnings("unchecked")
       @SuppressWarnings("unchecked")
       public Writer(Configuration conf, FileSystem fs, String dirName,
       public Writer(Configuration conf, FileSystem fs, String dirName,
           Class keyClass, Class valClass,
           Class keyClass, Class valClass,
           SequenceFile.CompressionType compression, final Filter filter)
           SequenceFile.CompressionType compression, final Filter filter)
       throws IOException {
       throws IOException {
         super(conf, fs, dirName, keyClass, valClass, compression);
         super(conf, fs, dirName, keyClass, valClass, compression);
-        this.bloomFilter = filter;
+        bloomFilter = filter;
       }
       }
       
       
+      /** {@inheritDoc} */
       @Override
       @Override
       public void append(WritableComparable key, Writable val)
       public void append(WritableComparable key, Writable val)
       throws IOException {
       throws IOException {
-        if (this.bloomFilter != null) {
-          this.bloomFilter.add(getBloomFilterKey(key));
+        if (bloomFilter != null) {
+          bloomFilter.add(getBloomFilterKey(key));
         }
         }
         super.append(key, val);
         super.append(key, val);
       }
       }
     }
     }
   }
   }
   
   
-  /**
-   * Custom bloom filter key maker.
-   * @param key
-   * @return Key made of bytes of row and column only.
-   * @throws IOException
-   */
-  static Key getBloomFilterKey(WritableComparable key)
-  throws IOException {
-    HStoreKey hsk = (HStoreKey)key;
-    byte [] bytes = null;
-    try {
-      bytes = (hsk.getRow().toString() + hsk.getColumn().toString()).
-        getBytes(UTF8_ENCODING);
-    } catch (UnsupportedEncodingException e) {
-      throw new IOException(e.toString());
-    }
-    return new Key(bytes);
-  }
-
-  /**
-   * Get reader for the store file map file.
-   * Client is responsible for closing file when done.
-   * @param fs
-   * @param bloomFilter If null, no filtering is done.
-   * @return MapFile.Reader
-   * @throws IOException
-   */
-  public synchronized MapFile.Reader getReader(final FileSystem fs,
-      final Filter bloomFilter)
-  throws IOException {
-    return isReference()?
-      new HStoreFile.HalfMapFileReader(fs,
-        getMapFilePath(getReference().getEncodedRegionName(),
-          getReference().getFileId()).toString(),
-        this.conf, getReference().getFileRegion(), getReference().getMidkey(),
-        bloomFilter):
-      new BloomFilterMapFile.Reader(fs, getMapFilePath().toString(),
-        this.conf, bloomFilter);
-  }
-
-  /**
-   * Get a store file writer.
-   * Client is responsible for closing file when done.
-   * @param fs
-   * @param compression Pass <code>SequenceFile.CompressionType.NONE</code>
-   * for none.
-   * @param bloomFilter If null, no filtering is done.
-   * @return MapFile.Writer
-   * @throws IOException
-   */
-  public MapFile.Writer getWriter(final FileSystem fs,
-      final SequenceFile.CompressionType compression,
-      final Filter bloomFilter)
-  throws IOException {
-    if (isReference()) {
-      throw new IOException("Illegal Access: Cannot get a writer on a" +
-        "HStoreFile reference");
-    }
-    return new BloomFilterMapFile.Writer(conf, fs,
-      getMapFilePath().toString(), HStoreKey.class,
-      ImmutableBytesWritable.class, compression, bloomFilter);
-  }
-
   /**
   /**
    * A facade for a {@link MapFile.Reader} that serves up either the top or
    * A facade for a {@link MapFile.Reader} that serves up either the top or
    * bottom half of a MapFile (where 'bottom' is the first half of the file
    * bottom half of a MapFile (where 'bottom' is the first half of the file
@@ -856,19 +751,19 @@ public class HStoreFile implements HConstants, WritableComparable {
         final WritableComparable midKey, final Filter filter)
         final WritableComparable midKey, final Filter filter)
     throws IOException {
     throws IOException {
       super(fs, dirName, conf, filter);
       super(fs, dirName, conf, filter);
-      this.top = isTopFileRegion(r);
-      this.midkey = midKey;
+      top = isTopFileRegion(r);
+      midkey = midKey;
     }
     }
     
     
     @SuppressWarnings("unchecked")
     @SuppressWarnings("unchecked")
     private void checkKey(final WritableComparable key)
     private void checkKey(final WritableComparable key)
     throws IOException {
     throws IOException {
-      if (this.top) {
-        if (key.compareTo(this.midkey) < 0) {
+      if (top) {
+        if (key.compareTo(midkey) < 0) {
           throw new IOException("Illegal Access: Key is less than midKey of " +
           throw new IOException("Illegal Access: Key is less than midKey of " +
           "backing mapfile");
           "backing mapfile");
         }
         }
-      } else if (key.compareTo(this.midkey) >= 0) {
+      } else if (key.compareTo(midkey) >= 0) {
         throw new IOException("Illegal Access: Key is greater than or equal " +
         throw new IOException("Illegal Access: Key is greater than or equal " +
         "to midKey of backing mapfile");
         "to midKey of backing mapfile");
       }
       }
@@ -896,11 +791,11 @@ public class HStoreFile implements HConstants, WritableComparable {
     public synchronized WritableComparable getClosest(WritableComparable key,
     public synchronized WritableComparable getClosest(WritableComparable key,
         Writable val)
         Writable val)
     throws IOException {
     throws IOException {
-      if (this.top) {
-        if (key.compareTo(this.midkey) < 0) {
-          return this.midkey;
+      if (top) {
+        if (key.compareTo(midkey) < 0) {
+          return midkey;
         }
         }
-      } else if (key.compareTo(this.midkey) >= 0) {
+      } else if (key.compareTo(midkey) >= 0) {
         // Contract says return null if EOF.
         // Contract says return null if EOF.
         return null;
         return null;
       }
       }
@@ -920,12 +815,12 @@ public class HStoreFile implements HConstants, WritableComparable {
     @Override
     @Override
     public synchronized boolean next(WritableComparable key, Writable val)
     public synchronized boolean next(WritableComparable key, Writable val)
     throws IOException {
     throws IOException {
-      if (this.top && this.topFirstNextCall) {
-        this.topFirstNextCall = false;
+      if (top && topFirstNextCall) {
+        topFirstNextCall = false;
         return doFirstNextProcessing(key, val);
         return doFirstNextProcessing(key, val);
       }
       }
       boolean result = super.next(key, val);
       boolean result = super.next(key, val);
-      if (!top && key.compareTo(this.midkey) >= 0) {
+      if (!top && key.compareTo(midkey) >= 0) {
         result = false;
         result = false;
       }
       }
       return result;
       return result;
@@ -935,7 +830,7 @@ public class HStoreFile implements HConstants, WritableComparable {
     throws IOException {
     throws IOException {
       // Seek to midkey.  Midkey may not exist in this file.  That should be
       // Seek to midkey.  Midkey may not exist in this file.  That should be
       // fine.  Then we'll either be positioned at end or start of file.
       // fine.  Then we'll either be positioned at end or start of file.
-      WritableComparable nearest = getClosest(this.midkey, val);
+      WritableComparable nearest = getClosest(midkey, val);
       // Now copy the mid key into the passed key.
       // Now copy the mid key into the passed key.
       if (nearest != null) {
       if (nearest != null) {
         Writables.copyWritable(nearest, key);
         Writables.copyWritable(nearest, key);
@@ -948,8 +843,8 @@ public class HStoreFile implements HConstants, WritableComparable {
     @Override
     @Override
     public synchronized void reset() throws IOException {
     public synchronized void reset() throws IOException {
       if (top) {
       if (top) {
-        this.topFirstNextCall = true;
-        seek(this.midkey);
+        topFirstNextCall = true;
+        seek(midkey);
         return;
         return;
       }
       }
       super.reset();
       super.reset();
@@ -963,90 +858,4 @@ public class HStoreFile implements HConstants, WritableComparable {
       return super.seek(key);
       return super.seek(key);
     }
     }
   }
   }
-
-  /**
-   * @return Length of the store map file.  If a reference, size is
-   * approximation.
-   * @throws IOException
-   */
-  public long length() throws IOException {
-    Path p = new Path(getMapFilePath(getReference()), MapFile.DATA_FILE_NAME);
-    long l = p.getFileSystem(this.conf).getFileStatus(p).getLen();
-    return (isReference())? l / 2: l;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public String toString() {
-    return this.encodedRegionName.toString() + "/" + this.colFamily.toString() +
-      "/" + this.fileId +
-      (isReference()? "/" + this.reference.toString(): "");
-  }
-  
-  /** {@inheritDoc} */
-  @Override
-  public boolean equals(Object o) {
-    return this.compareTo(o) == 0;
-  }
-  
-  /** {@inheritDoc} */
-  @Override
-  public int hashCode() {
-    int result = this.dir.hashCode();
-    result ^= this.encodedRegionName.hashCode();
-    result ^= this.colFamily.hashCode();
-    result ^= this.fileId;
-    return result;
-  }
-
-  // Writable
-
-  /** {@inheritDoc} */
-  public void write(DataOutput out) throws IOException {
-    out.writeUTF(dir.toString());
-    out.writeUTF(this.encodedRegionName);
-    this.colFamily.write(out);
-    out.writeLong(fileId);
-    out.writeBoolean(isReference());
-    if (isReference()) {
-      this.reference.write(out);
-    }
-  }
-  
-  /** {@inheritDoc} */
-  public void readFields(DataInput in) throws IOException {
-    this.dir = new Path(in.readUTF());
-    this.encodedRegionName = in.readUTF();
-    this.colFamily.readFields(in);
-    this.fileId = in.readLong();
-    this.reference = null;
-    boolean isReferent = in.readBoolean();
-    this.reference = new HStoreFile.Reference();
-    if (isReferent) {
-      this.reference.readFields(in);
-    }
-  }
-
-  // Comparable
-
-  /** {@inheritDoc} */
-  public int compareTo(Object o) {
-    HStoreFile other = (HStoreFile) o;
-    int result = this.dir.compareTo(other.dir);    
-    if(result == 0) {
-      this.encodedRegionName.compareTo(other.encodedRegionName);
-    }
-    if(result == 0) {
-      result = this.colFamily.compareTo(other.colFamily);
-    }    
-    if(result == 0) {
-      if(this.fileId < other.fileId) {
-        result = -1;
-        
-      } else if(this.fileId > other.fileId) {
-        result = 1;
-      }
-    }
-    return result;
-  }
 }
 }

+ 10 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java

@@ -30,6 +30,7 @@ import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparable;
 
 
@@ -243,4 +244,13 @@ public class HTableDescriptor implements WritableComparable {
   public SortedMap<Text, HColumnDescriptor> getFamilies() {
   public SortedMap<Text, HColumnDescriptor> getFamilies() {
     return Collections.unmodifiableSortedMap(this.families);
     return Collections.unmodifiableSortedMap(this.families);
   }
   }
+
+  /**
+   * @param rootdir qualified path of HBase root directory
+   * @param tableName name of table
+   * @return path for table
+   */
+  public static Path getTableDir(Path rootdir, Text tableName) {
+    return new Path(rootdir, tableName.toString());
+  }
 }
 }

+ 437 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/Migrate.java

@@ -0,0 +1,437 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.io.IOException;
+
+import java.util.Map;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hadoop.io.Text;
+
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HLog;
+import org.apache.hadoop.hbase.HRegion;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HScannerInterface;
+import org.apache.hadoop.hbase.HStore;
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+
+/**
+ * Perform a file system upgrade to convert older file layouts to that
+ * supported by HADOOP-2478
+ */
+public class Migrate extends Configured implements Tool {
+  static final Log LOG = LogFactory.getLog(Migrate.class);
+
+  private static final String OLD_PREFIX = "hregion_";
+
+  private final HBaseConfiguration conf;
+
+  /** Action to take when an extra file is found */
+  private static enum EXTRA_FILES  {
+    /** Stop conversion */
+    ABORT,
+    /** print a warning message, but otherwise ignore */
+    IGNORE,
+    /** delete extra files */
+    DELETE,
+    /** prompt for disposition of extra files */
+    PROMPT
+  }
+  
+  private static final Map<String, EXTRA_FILES> options =
+    new HashMap<String, EXTRA_FILES>();
+  
+  static {
+   options.put("abort", EXTRA_FILES.ABORT);
+   options.put("ignore", EXTRA_FILES.IGNORE);
+   options.put("delete", EXTRA_FILES.DELETE);
+   options.put("prompt", EXTRA_FILES.PROMPT);
+  }
+  
+  private EXTRA_FILES logFiles = EXTRA_FILES.ABORT;
+  private EXTRA_FILES otherFiles = EXTRA_FILES.IGNORE;
+
+  private BufferedReader reader = null;
+  
+  private final Set<String> references = new HashSet<String>();
+  
+  /** default constructor */
+  public Migrate() {
+    this(new HBaseConfiguration());
+  }
+  
+  /**
+   * @param conf
+   */
+  public Migrate(HBaseConfiguration conf) {
+    super(conf);
+    this.conf = conf;
+    conf.setInt("hbase.client.retries.number", 1);
+  }
+  
+  /** {@inheritDoc} */
+  public int run(String[] args) throws Exception {
+    parseArgs(args);
+    
+    try {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      if (admin.isMasterRunning()) {
+        throw new IllegalStateException(
+        "HBase cluster must be off-line while being upgraded");
+      }
+    } catch (MasterNotRunningException e) {
+      // ignore
+    }
+    FileSystem fs = FileSystem.get(conf);               // get DFS handle
+    Path rootdir = fs.makeQualified(new Path(          // get path for instance
+        conf.get(HConstants.HBASE_DIR, HConstants.DEFAULT_HBASE_DIR)));
+
+    // check for "extra" files
+
+    extraFiles(fs, rootdir);
+
+    // find root region
+
+    Path rootRegion = new Path(rootdir, 
+        OLD_PREFIX + HRegionInfo.rootRegionInfo.getEncodedName());
+
+    if (!fs.exists(rootRegion)) {
+      throw new IOException("cannot find root region " + rootRegion.toString());
+    }
+
+    processRegionDir(fs, rootdir, HConstants.ROOT_TABLE_NAME, rootRegion);
+    scanRootRegion(fs, rootdir);
+
+    // scan for left over regions
+
+    extraRegions(fs, rootdir);
+
+    return 0;
+  }
+
+  private void extraFiles(FileSystem fs, Path rootdir) throws IOException {
+    FileStatus[] stats = fs.listStatus(rootdir);
+    if (stats == null || stats.length == 0) {
+      throw new IOException("No files found under root directory " +
+          rootdir.toString());
+    }
+    for (int i = 0; i < stats.length; i++) {
+      String name = stats[i].getPath().getName();
+      if (!name.startsWith(OLD_PREFIX)) {
+        if (name.startsWith("log_")) {
+          String message = "unrecovered region server log file " + name; 
+          extraFile(logFiles, message, fs, stats[i].getPath());
+        } else {
+          String message = "unrecognized file " + name;
+          extraFile(otherFiles, message, fs, stats[i].getPath());
+        }
+      }
+    }
+  }
+
+  private void extraFile(EXTRA_FILES action, String message, FileSystem fs,
+      Path p) throws IOException {
+    
+    if (action == EXTRA_FILES.ABORT) {
+      throw new IOException(message + " aborting");
+
+    } else if (action == EXTRA_FILES.IGNORE) {
+      LOG.info(message + " ignoring");
+
+    } else if (action == EXTRA_FILES.DELETE) {
+      LOG.info(message + " deleting");
+      fs.delete(p);
+
+    } else {
+      // logFiles == EXTRA_FILES.PROMPT
+      String response = prompt(message + " delete? [y/n]");
+
+      if (response.startsWith("Y") || response.startsWith("y")) {
+        LOG.info(message + " deleting");
+        fs.delete(p);
+      }
+    }
+  }
+  
+  private void processRegionDir(FileSystem fs, Path rootdir, Text tableName,
+      Path oldPath) throws IOException {
+
+    // Create directory where table will live
+
+    Path tableDir = new Path(rootdir, tableName.toString());
+    fs.mkdirs(tableDir);
+
+    // Move the old region directory under the table directory
+
+    Path newPath =
+      new Path(tableDir, oldPath.getName().substring(OLD_PREFIX.length()));
+    fs.rename(oldPath, newPath);
+
+    processRegionSubDirs(fs, newPath);
+  }
+  
+  private void processRegionSubDirs(FileSystem fs, Path newPath)
+  throws IOException {
+    String newName = newPath.getName();
+    FileStatus[] children = fs.listStatus(newPath);
+    for (int i = 0; i < children.length; i++) {
+      String child = children[i].getPath().getName();
+      if (children[i].isDir()) {
+        processRegionSubDirs(fs, children[i].getPath());
+
+        // Rename old compaction directories
+
+        if (child.startsWith(OLD_PREFIX)) {
+          fs.rename(children[i].getPath(),
+              new Path(newPath, child.substring(OLD_PREFIX.length())));
+        }
+      } else {
+        if (newName.compareTo("mapfiles") == 0) {
+          // Check to see if this mapfile is a reference
+
+          if (HStore.isReference(children[i].getPath())) {
+            // Keep track of references in case we come across a region
+            // that we can't otherwise account for.
+            references.add(child.substring(child.indexOf(".") + 1));
+          }
+        }
+      }
+    }
+  }
+  
+  private void scanRootRegion(FileSystem fs, Path rootdir) throws IOException {
+    HLog log = new HLog(fs, new Path(rootdir, HConstants.HREGION_LOGDIR_NAME),
+        conf, null);
+
+    try {
+      // Open root region so we can scan it
+
+      HRegion rootRegion = new HRegion(
+          new Path(rootdir, HConstants.ROOT_TABLE_NAME.toString()), log, fs, conf,
+          HRegionInfo.rootRegionInfo, null, null);
+
+      try {
+        HScannerInterface rootScanner = rootRegion.getScanner(
+            HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW,
+            HConstants.LATEST_TIMESTAMP, null);
+
+        try {
+          HStoreKey key = new HStoreKey();
+          SortedMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
+          while (rootScanner.next(key, results)) {
+            HRegionInfo info = Writables.getHRegionInfoOrNull(
+                results.get(HConstants.COL_REGIONINFO));
+            if (info == null) {
+              LOG.warn("region info is null for row " + key.getRow() +
+                  " in table " + HConstants.ROOT_TABLE_NAME);
+              continue;
+            }
+
+            // First move the meta region to where it should be and rename
+            // subdirectories as necessary
+
+            processRegionDir(fs, rootdir, HConstants.META_TABLE_NAME,
+                new Path(rootdir, OLD_PREFIX + info.getEncodedName()));
+
+            // Now scan and process the meta table
+
+            scanMetaRegion(fs, rootdir, log, info);
+          }
+
+        } finally {
+          rootScanner.close();
+        }
+
+      } finally {
+        rootRegion.close();
+      }
+
+    } finally {
+      log.closeAndDelete();
+    }
+  }
+  
+  private void scanMetaRegion(FileSystem fs, Path rootdir, HLog log,
+      HRegionInfo info) throws IOException {
+
+    HRegion metaRegion = new HRegion(
+        new Path(rootdir, info.getTableDesc().getName().toString()), log, fs,
+        conf, info, null, null);
+
+    try {
+      HScannerInterface metaScanner = metaRegion.getScanner(
+          HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW,
+          HConstants.LATEST_TIMESTAMP, null);
+
+      try {
+        HStoreKey key = new HStoreKey();
+        SortedMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
+        while (metaScanner.next(key, results)) {
+          HRegionInfo region = Writables.getHRegionInfoOrNull(
+              results.get(HConstants.COL_REGIONINFO));
+          if (region == null) {
+            LOG.warn("region info is null for row " + key.getRow() +
+                " in table " + HConstants.META_TABLE_NAME);
+            continue;
+          }
+
+          // Move the region to where it should be and rename
+          // subdirectories as necessary
+
+          processRegionDir(fs, rootdir, region.getTableDesc().getName(),
+              new Path(rootdir, OLD_PREFIX + region.getEncodedName()));
+
+          results.clear();
+        }
+
+      } finally {
+        metaScanner.close();
+      }
+
+    } finally {
+      metaRegion.close();
+    }
+  }
+  
+  private void extraRegions(FileSystem fs, Path rootdir) throws IOException {
+    FileStatus[] stats = fs.listStatus(rootdir);
+    if (stats == null || stats.length == 0) {
+      throw new IOException("No files found under root directory " +
+          rootdir.toString());
+    }
+    for (int i = 0; i < stats.length; i++) {
+      String name = stats[i].getPath().getName();
+      if (name.startsWith(OLD_PREFIX)) {
+        String encodedName = name.substring(OLD_PREFIX.length());
+        String message;
+        if (references.contains(encodedName)) {
+          message =
+            "region not in meta table but other regions reference it " + name;
+
+        } else {
+          message = 
+            "region not in meta table and no other regions reference it " + name;
+        }
+        extraFile(otherFiles, message, fs, stats[i].getPath());
+      }
+    }
+  }
+  
+  @SuppressWarnings("static-access")
+  private void parseArgs(String[] args) {
+    Options opts = new Options();
+    Option logFiles = OptionBuilder.withArgName("abort|ignore|delete|prompt")
+    .hasArg()
+    .withDescription(
+        "disposition of unrecovered region server logs: {abort|ignore|delete|prompt}")
+    .create("logfiles");
+
+    Option extraFiles = OptionBuilder.withArgName("abort|ignore|delete|prompt")
+    .hasArg()
+    .withDescription("disposition of 'extra' files: {abort|ignore|delete|prompt}")
+    .create("extrafiles");
+    
+    opts.addOption(logFiles);
+    opts.addOption(extraFiles);
+    
+    CommandLineParser parser = new GnuParser();
+    try {
+      CommandLine commandLine = parser.parse(opts, args, true);
+      if (commandLine.hasOption("log-files")) {
+        this.logFiles = options.get(commandLine.getOptionValue("log-files"));
+      }
+      if (commandLine.hasOption("extra-files")) {
+        this.otherFiles = options.get(commandLine.getOptionValue("extra-files"));
+      }
+    } catch (ParseException e) {
+      LOG.error("options parsing failed", e);
+      
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp("options are: ", opts);
+    }
+  }
+  
+  private synchronized String prompt(String prompt) {
+    System.out.print(prompt + " > ");
+    System.out.flush();
+    if (reader == null) {
+      reader = new BufferedReader(new InputStreamReader(System.in));
+    }
+    try {
+      return reader.readLine();
+      
+    } catch (IOException e) {
+      return null;
+    }
+  }
+  
+  /**
+   * Main program
+   * 
+   * @param args command line arguments
+   */
+  public static void main(String[] args) {
+    Tool t = new Migrate();
+    GenericOptionsParser hadoopOpts =
+      new GenericOptionsParser(t.getConf(), args);
+
+    int status = 0;
+    try {
+      status = ToolRunner.run(t, hadoopOpts.getRemainingArgs());
+    } catch (Exception e) {
+      LOG.error("exiting due to error", e);
+      status = -1;
+    }
+    System.exit(status);
+  }
+}

+ 7 - 0
src/contrib/hbase/src/test/hbase-site.xml

@@ -22,6 +22,13 @@
  */
  */
 -->
 -->
 <configuration>
 <configuration>
+  <property>
+    <name>fs.default.name</name>
+    <value></value>
+    <description>Use hdfs as file system by default. Modify this to run on
+    local file system.
+    </description>
+  </property>
   <property>
   <property>
     <name>hbase.regionserver.msginterval</name>
     <name>hbase.regionserver.msginterval</name>
     <value>1000</value>
     <value>1000</value>

+ 32 - 26
src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java

@@ -20,11 +20,10 @@
 package org.apache.hadoop.hbase;
 package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.io.UnsupportedEncodingException;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
@@ -38,18 +37,9 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
   protected HTableDescriptor desc;
   protected HTableDescriptor desc;
   protected ImmutableBytesWritable value;
   protected ImmutableBytesWritable value;
 
 
-  protected MiniDFSCluster dfsCluster = null;
-  protected FileSystem fs;
-  protected Path dir;
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    desc = new HTableDescriptor("test");
-    desc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString()));
+  /** constructor */
+  public AbstractMergeTestBase() {
+    super();
     
     
     // We will use the same value for the rows as that is not really important here
     // We will use the same value for the rows as that is not really important here
     
     
@@ -58,11 +48,31 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
     while(val.length() < 1024) {
     while(val.length() < 1024) {
       val.append(partialValue);
       val.append(partialValue);
     }
     }
-    
-    value = new ImmutableBytesWritable(
-        val.toString().getBytes(HConstants.UTF8_ENCODING));
+ 
+    try {
+      value = new ImmutableBytesWritable(
+          val.toString().getBytes(HConstants.UTF8_ENCODING));
+    } catch (UnsupportedEncodingException e) {
+      fail();
+    }
+    desc = new HTableDescriptor("test");
+    desc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString()));
+  }
+
+  protected MiniDFSCluster dfsCluster = null;
 
 
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void setUp() throws Exception {
+    conf.setLong("hbase.hregion.max.filesize", 64L * 1024L * 1024L);
     dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
     dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
+    
+    // Note: we must call super.setUp after starting the mini cluster or
+    // we will end up with a local file system
+    
+    super.setUp();
       
       
     // We create three data regions: The first is too large to merge since it 
     // We create three data regions: The first is too large to merge since it 
     // will be > 64 MB in size. The second two will be smaller and will be 
     // will be > 64 MB in size. The second two will be smaller and will be 
@@ -72,10 +82,6 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
     // least 65536 rows. We will make certain by writing 70000
     // least 65536 rows. We will make certain by writing 70000
 
 
     try {
     try {
-      fs = dfsCluster.getFileSystem();
-      dir = new Path("/hbase");
-      fs.mkdirs(dir);
-
       Text row_70001 = new Text("row_70001");
       Text row_70001 = new Text("row_70001");
       Text row_80001 = new Text("row_80001");
       Text row_80001 = new Text("row_80001");
       
       
@@ -88,8 +94,10 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
       // Now create the root and meta regions and insert the data regions
       // Now create the root and meta regions and insert the data regions
       // created above into the meta
       // created above into the meta
       
       
-      HRegion root = createNewHRegion(dir, conf, HRegionInfo.rootRegionInfo);
-      HRegion meta = createNewHRegion(dir, conf, HRegionInfo.firstMetaRegionInfo);
+      HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo,
+          testDir, this.conf);
+      HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo,
+        testDir, this.conf);
       HRegion.addRegionToMETA(root, meta);
       HRegion.addRegionToMETA(root, meta);
       
       
       for(int i = 0; i < regions.length; i++) {
       for(int i = 0; i < regions.length; i++) {
@@ -119,7 +127,7 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
   private HRegion createAregion(Text startKey, Text endKey, int firstRow,
   private HRegion createAregion(Text startKey, Text endKey, int firstRow,
       int nrows) throws IOException {
       int nrows) throws IOException {
     
     
-    HRegion region = createNewHRegion(dir, conf, desc, startKey, endKey);
+    HRegion region = createNewHRegion(desc, startKey, endKey);
     
     
     System.out.println("created region " + region.getRegionName());
     System.out.println("created region " + region.getRegionName());
 
 
@@ -135,8 +143,6 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
         r.flushcache();
         r.flushcache();
       }
       }
     }
     }
-    System.out.println("Rolling log...");
-    region.log.rollWriter();
     region.compactIfNeeded();
     region.compactIfNeeded();
     region.close();
     region.close();
     region.getLog().closeAndDelete();
     region.getLog().closeAndDelete();

+ 1 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java

@@ -115,7 +115,7 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
     threadDumpingJoin(this.cluster.getMaster());
     threadDumpingJoin(this.cluster.getMaster());
   }
   }
 
 
-  public void threadDumpingJoin(final Thread t) {
+  protected void threadDumpingJoin(final Thread t) {
     if (t == null) {
     if (t == null) {
       return;
       return;
     }
     }

+ 185 - 46
src/contrib/hbase/src/test/org/apache/hadoop/hbase/HBaseTestCase.java

@@ -26,7 +26,8 @@ import java.util.Random;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
-import org.apache.hadoop.conf.Configuration;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
 import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
@@ -37,13 +38,16 @@ import org.apache.hadoop.io.Text;
  * Abstract base class for test cases. Performs all static initialization
  * Abstract base class for test cases. Performs all static initialization
  */
  */
 public abstract class HBaseTestCase extends TestCase {
 public abstract class HBaseTestCase extends TestCase {
+  private static final Log LOG = LogFactory.getLog(HBaseTestCase.class);
+
   protected final static String COLFAMILY_NAME1 = "colfamily1:";
   protected final static String COLFAMILY_NAME1 = "colfamily1:";
   protected final static String COLFAMILY_NAME2 = "colfamily2:";
   protected final static String COLFAMILY_NAME2 = "colfamily2:";
   protected final static String COLFAMILY_NAME3 = "colfamily3:";
   protected final static String COLFAMILY_NAME3 = "colfamily3:";
   protected static Text [] COLUMNS = new Text [] {new Text(COLFAMILY_NAME1),
   protected static Text [] COLUMNS = new Text [] {new Text(COLFAMILY_NAME1),
     new Text(COLFAMILY_NAME2), new Text(COLFAMILY_NAME3)};
     new Text(COLFAMILY_NAME2), new Text(COLFAMILY_NAME3)};
+  private boolean localfs = false;
   protected Path testDir = null;
   protected Path testDir = null;
-  protected FileSystem localFs = null;
+  protected FileSystem fs = null;
   protected static final char FIRST_CHAR = 'a';
   protected static final char FIRST_CHAR = 'a';
   protected static final char LAST_CHAR = 'z';
   protected static final char LAST_CHAR = 'z';
   protected static final String PUNCTUATION = "~`@#$%^&*()-_+=:;',.<>/?[]{}|";
   protected static final String PUNCTUATION = "~`@#$%^&*()-_+=:;',.<>/?[]{}|";
@@ -58,9 +62,7 @@ public abstract class HBaseTestCase extends TestCase {
   
   
   protected volatile HBaseConfiguration conf;
   protected volatile HBaseConfiguration conf;
 
 
-  /**
-   * constructor
-   */
+  /** constructor */
   public HBaseTestCase() {
   public HBaseTestCase() {
     super();
     super();
     init();
     init();
@@ -77,51 +79,88 @@ public abstract class HBaseTestCase extends TestCase {
   private void init() {
   private void init() {
     conf = new HBaseConfiguration();
     conf = new HBaseConfiguration();
     try {
     try {
-      START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_ENCODING) + PUNCTUATION;
+      START_KEY =
+        new String(START_KEY_BYTES, HConstants.UTF8_ENCODING) + PUNCTUATION;
     } catch (UnsupportedEncodingException e) {
     } catch (UnsupportedEncodingException e) {
+      LOG.fatal("error during initialization", e);
       fail();
       fail();
     }
     }
   }
   }
   
   
-  /** {@inheritDoc} */
+  /**
+   * {@inheritDoc}
+   * 
+   * Note that this method must be called after the mini hdfs cluster has
+   * started or we end up with a local file system.
+   * 
+   */
   @Override
   @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
-    this.testDir = getUnitTestdir(getName());
-    this.localFs = FileSystem.getLocal(this.conf);
-    if (localFs.exists(testDir)) {
-      localFs.delete(testDir);
+    localfs =
+      (conf.get("fs.default.name", "file:///").compareTo("file::///") == 0);
+
+    try {
+      this.fs = FileSystem.get(conf);
+    } catch (IOException e) {
+      LOG.fatal("error getting file system", e);
+      throw e;
+    }
+    try {
+      if (localfs) {
+        this.testDir = getUnitTestdir(getName());
+        if (fs.exists(testDir)) {
+          fs.delete(testDir);
+        }
+      } else {
+        this.testDir = fs.makeQualified(
+            new Path(conf.get(HConstants.HBASE_DIR, HConstants.DEFAULT_HBASE_DIR))
+        );
+      }
+    } catch (Exception e) {
+      LOG.fatal("error during setup", e);
+      throw e;
     }
     }
   }
   }
   
   
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   protected void tearDown() throws Exception {
   protected void tearDown() throws Exception {
-    if (this.localFs != null && this.testDir != null &&
-        this.localFs.exists(testDir)) {
-      this.localFs.delete(testDir);
+    try {
+      if (localfs) {
+        if (this.fs.exists(testDir)) {
+          this.fs.delete(testDir);
+        }
+      }
+    } catch (Exception e) {
+      LOG.fatal("error during tear down", e);
     }
     }
     super.tearDown();
     super.tearDown();
   }
   }
 
 
   protected Path getUnitTestdir(String testName) {
   protected Path getUnitTestdir(String testName) {
-    return new Path(StaticTestEnvironment.TEST_DIRECTORY_KEY, testName);
+    return new Path(
+        conf.get(StaticTestEnvironment.TEST_DIRECTORY_KEY, "test/build/data"),
+        testName);
   }
   }
 
 
-  protected HRegion createNewHRegion(Path dir, Configuration c,
-      HTableDescriptor desc, Text startKey, Text endKey) throws IOException {
-    return createNewHRegion(dir, c, new HRegionInfo(desc, startKey, endKey));
+  protected HRegion createNewHRegion(HTableDescriptor desc, Text startKey,
+      Text endKey) throws IOException {
+    
+    FileSystem fs = FileSystem.get(conf);
+    Path rootdir = fs.makeQualified(
+        new Path(conf.get(HConstants.HBASE_DIR, HConstants.DEFAULT_HBASE_DIR)));
+    fs.mkdirs(rootdir);
+    
+    return HRegion.createHRegion(new HRegionInfo(desc, startKey, endKey),
+        rootdir, conf);
   }
   }
   
   
-  protected HRegion createNewHRegion(Path dir, Configuration c,
-        HRegionInfo info) throws IOException {
-    Path regionDir = HRegion.getRegionDir(dir
-        , HRegionInfo.encodeRegionName(info.getRegionName()));
-    FileSystem fs = dir.getFileSystem(c);
-    fs.mkdirs(regionDir);
-    return new HRegion(dir,
-      new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME), conf,
-          null), fs, conf, info, null, null);
+  protected HRegion openClosedRegion(final HRegion closedRegion)
+  throws IOException {
+    return new HRegion(closedRegion.basedir, closedRegion.getLog(),
+      closedRegion.getFilesystem(), closedRegion.getConf(),
+      closedRegion.getRegionInfo(), null, null);
   }
   }
   
   
   /**
   /**
@@ -275,6 +314,9 @@ public abstract class HBaseTestCase extends TestCase {
    * Implementors can flushcache.
    * Implementors can flushcache.
    */
    */
   public static interface FlushCache {
   public static interface FlushCache {
+    /**
+     * @throws IOException
+     */
     public void flushcache() throws IOException;
     public void flushcache() throws IOException;
   }
   }
   
   
@@ -285,21 +327,82 @@ public abstract class HBaseTestCase extends TestCase {
    * TOOD: Come up w/ a better name for this interface.
    * TOOD: Come up w/ a better name for this interface.
    */
    */
   public static interface Incommon {
   public static interface Incommon {
+    /**
+     * @param row
+     * @param column
+     * @return value for row/column pair
+     * @throws IOException
+     */
     public byte [] get(Text row, Text column) throws IOException;
     public byte [] get(Text row, Text column) throws IOException;
-    public byte [][] get(Text row, Text column, int versions)
-    throws IOException;
+    /**
+     * @param row
+     * @param column
+     * @param versions
+     * @return value for row/column pair for number of versions requested
+     * @throws IOException
+     */
+    public byte [][] get(Text row, Text column, int versions) throws IOException;
+    /**
+     * @param row
+     * @param column
+     * @param ts
+     * @param versions
+     * @return value for row/column/timestamp tuple for number of versions
+     * @throws IOException
+     */
     public byte [][] get(Text row, Text column, long ts, int versions)
     public byte [][] get(Text row, Text column, long ts, int versions)
     throws IOException;
     throws IOException;
+    /**
+     * @param row
+     * @return batch update identifier
+     * @throws IOException
+     */
     public long startBatchUpdate(final Text row) throws IOException;
     public long startBatchUpdate(final Text row) throws IOException;
+    /**
+     * @param lockid
+     * @param column
+     * @param val
+     * @throws IOException
+     */
     public void put(long lockid, Text column, byte val[]) throws IOException;
     public void put(long lockid, Text column, byte val[]) throws IOException;
+    /**
+     * @param lockid
+     * @param column
+     * @throws IOException
+     */
     public void delete(long lockid, Text column) throws IOException;
     public void delete(long lockid, Text column) throws IOException;
+    /**
+     * @param row
+     * @param column
+     * @param ts
+     * @throws IOException
+     */
     public void deleteAll(Text row, Text column, long ts) throws IOException;
     public void deleteAll(Text row, Text column, long ts) throws IOException;
+    /**
+     * @param lockid
+     * @throws IOException
+     */
     public void commit(long lockid) throws IOException;
     public void commit(long lockid) throws IOException;
+    /**
+     * @param lockid
+     * @param ts
+     * @throws IOException
+     */
     public void commit(long lockid, long ts) throws IOException;
     public void commit(long lockid, long ts) throws IOException;
+    /**
+     * @param lockid
+     * @throws IOException
+     */
     public void abort(long lockid) throws IOException;
     public void abort(long lockid) throws IOException;
+    /**
+     * @param columns
+     * @param firstRow
+     * @param ts
+     * @return scanner for specified columns, first row and timestamp
+     * @throws IOException
+     */
     public HScannerInterface getScanner(Text [] columns, Text firstRow,
     public HScannerInterface getScanner(Text [] columns, Text firstRow,
-      long ts)
-    throws IOException;
+      long ts) throws IOException;
   }
   }
   
   
   /**
   /**
@@ -316,17 +419,24 @@ public abstract class HBaseTestCase extends TestCase {
       }
       }
     }
     }
     
     
+    /**
+     * @param HRegion
+     */
     public HRegionIncommon(final HRegion HRegion) {
     public HRegionIncommon(final HRegion HRegion) {
       this.region = HRegion;
       this.region = HRegion;
       this.batch = null;
       this.batch = null;
     }
     }
-    public void abort(long lockid) throws IOException {
+    /** {@inheritDoc} */
+    public void abort(@SuppressWarnings("unused") long lockid) {
       this.batch = null;
       this.batch = null;
     }
     }
+    /** {@inheritDoc} */
     public void commit(long lockid) throws IOException {
     public void commit(long lockid) throws IOException {
       commit(lockid, HConstants.LATEST_TIMESTAMP);
       commit(lockid, HConstants.LATEST_TIMESTAMP);
     }
     }
-    public void commit(long lockid, final long ts) throws IOException {
+    /** {@inheritDoc} */
+    public void commit(@SuppressWarnings("unused") long lockid, final long ts)
+    throws IOException {
       checkBatch();
       checkBatch();
       try {
       try {
         this.region.batchUpdate(ts, batch);
         this.region.batchUpdate(ts, batch);
@@ -334,21 +444,29 @@ public abstract class HBaseTestCase extends TestCase {
         this.batch = null;
         this.batch = null;
       }
       }
     }
     }
-    public void put(long lockid, Text column, byte[] val) throws IOException {
+    /** {@inheritDoc} */
+    public void put(long lockid, Text column, byte[] val) {
       checkBatch();
       checkBatch();
       this.batch.put(lockid, column, val);
       this.batch.put(lockid, column, val);
     }
     }
-    public void delete(long lockid, Text column) throws IOException {
+    /** {@inheritDoc} */
+    public void delete(long lockid, Text column) {
       checkBatch();
       checkBatch();
       this.batch.delete(lockid, column);
       this.batch.delete(lockid, column);
     }
     }
+    /** {@inheritDoc} */
     public void deleteAll(Text row, Text column, long ts) throws IOException {
     public void deleteAll(Text row, Text column, long ts) throws IOException {
       this.region.deleteAll(row, column, ts);
       this.region.deleteAll(row, column, ts);
     }
     }
-    public long startBatchUpdate(Text row) throws IOException {
+    /** {@inheritDoc} */
+    public long startBatchUpdate(Text row) {
       return startUpdate(row);
       return startUpdate(row);
     }
     }
-    public long startUpdate(Text row) throws IOException {
+    /**
+     * @param row
+     * @return update id
+     */
+    public long startUpdate(Text row) {
       if (this.batch != null) {
       if (this.batch != null) {
         throw new IllegalStateException("Update already in progress");
         throw new IllegalStateException("Update already in progress");
       }
       }
@@ -356,24 +474,33 @@ public abstract class HBaseTestCase extends TestCase {
       this.batch = new BatchUpdate(lockid);
       this.batch = new BatchUpdate(lockid);
       return batch.startUpdate(row);
       return batch.startUpdate(row);
     }
     }
+    /** {@inheritDoc} */
     public HScannerInterface getScanner(Text [] columns, Text firstRow,
     public HScannerInterface getScanner(Text [] columns, Text firstRow,
-        long ts)
-    throws IOException {
+        long ts) throws IOException {
       return this.region.getScanner(columns, firstRow, ts, null);
       return this.region.getScanner(columns, firstRow, ts, null);
     }
     }
+    /** {@inheritDoc} */
     public byte[] get(Text row, Text column) throws IOException {
     public byte[] get(Text row, Text column) throws IOException {
       return this.region.get(row, column);
       return this.region.get(row, column);
     }
     }
+    /** {@inheritDoc} */
     public byte[][] get(Text row, Text column, int versions) throws IOException {
     public byte[][] get(Text row, Text column, int versions) throws IOException {
       return this.region.get(row, column, versions);
       return this.region.get(row, column, versions);
     }
     }
+    /** {@inheritDoc} */
     public byte[][] get(Text row, Text column, long ts, int versions)
     public byte[][] get(Text row, Text column, long ts, int versions)
         throws IOException {
         throws IOException {
       return this.region.get(row, column, ts, versions);
       return this.region.get(row, column, ts, versions);
     }
     }
+    /**
+     * @param row
+     * @return values for each column in the specified row
+     * @throws IOException
+     */
     public Map<Text, byte []> getFull(Text row) throws IOException {
     public Map<Text, byte []> getFull(Text row) throws IOException {
       return region.getFull(row);
       return region.getFull(row);
     }
     }
+    /** {@inheritDoc} */
     public void flushcache() throws IOException {
     public void flushcache() throws IOException {
       this.region.flushcache();
       this.region.flushcache();
     }
     }
@@ -384,43 +511,55 @@ public abstract class HBaseTestCase extends TestCase {
    */
    */
   public static class HTableIncommon implements Incommon {
   public static class HTableIncommon implements Incommon {
     final HTable table;
     final HTable table;
+    /**
+     * @param table
+     */
     public HTableIncommon(final HTable table) {
     public HTableIncommon(final HTable table) {
       super();
       super();
       this.table = table;
       this.table = table;
     }
     }
-    public void abort(long lockid) throws IOException {
+    /** {@inheritDoc} */
+    public void abort(long lockid) {
       this.table.abort(lockid);
       this.table.abort(lockid);
     }
     }
+    /** {@inheritDoc} */
     public void commit(long lockid) throws IOException {
     public void commit(long lockid) throws IOException {
       this.table.commit(lockid);
       this.table.commit(lockid);
     }
     }
+    /** {@inheritDoc} */
     public void commit(long lockid, final long ts) throws IOException {
     public void commit(long lockid, final long ts) throws IOException {
       this.table.commit(lockid, ts);
       this.table.commit(lockid, ts);
     }
     }
-    public void put(long lockid, Text column, byte[] val) throws IOException {
+    /** {@inheritDoc} */
+    public void put(long lockid, Text column, byte[] val) {
       this.table.put(lockid, column, val);
       this.table.put(lockid, column, val);
     }
     }
-    public void delete(long lockid, Text column) throws IOException {
+    /** {@inheritDoc} */
+    public void delete(long lockid, Text column) {
       this.table.delete(lockid, column);
       this.table.delete(lockid, column);
     }
     }
+    /** {@inheritDoc} */
     public void deleteAll(Text row, Text column, long ts) throws IOException {
     public void deleteAll(Text row, Text column, long ts) throws IOException {
       this.table.deleteAll(row, column, ts);
       this.table.deleteAll(row, column, ts);
     }
     }
+    /** {@inheritDoc} */
     public long startBatchUpdate(Text row) {
     public long startBatchUpdate(Text row) {
       return this.table.startUpdate(row);
       return this.table.startUpdate(row);
     }
     }
+    /** {@inheritDoc} */
     public HScannerInterface getScanner(Text [] columns, Text firstRow,
     public HScannerInterface getScanner(Text [] columns, Text firstRow,
-        long ts)
-    throws IOException {
+        long ts) throws IOException {
       return this.table.obtainScanner(columns, firstRow, ts, null);
       return this.table.obtainScanner(columns, firstRow, ts, null);
     }
     }
+    /** {@inheritDoc} */
     public byte[] get(Text row, Text column) throws IOException {
     public byte[] get(Text row, Text column) throws IOException {
       return this.table.get(row, column);
       return this.table.get(row, column);
     }
     }
-    public byte[][] get(Text row, Text column, int versions)
-    throws IOException {
+    /** {@inheritDoc} */
+    public byte[][] get(Text row, Text column, int versions) throws IOException {
       return this.table.get(row, column, versions);
       return this.table.get(row, column, versions);
     }
     }
+    /** {@inheritDoc} */
     public byte[][] get(Text row, Text column, long ts, int versions)
     public byte[][] get(Text row, Text column, long ts, int versions)
     throws IOException {
     throws IOException {
       return this.table.get(row, column, ts, versions);
       return this.table.get(row, column, ts, versions);

+ 3 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java

@@ -86,15 +86,17 @@ public class MiniHBaseCluster implements HConstants {
    * @param conf
    * @param conf
    * @param nRegionNodes
    * @param nRegionNodes
    * @param dfsCluster
    * @param dfsCluster
+   * @param deleteOnExit
    * @throws IOException
    * @throws IOException
    */
    */
   public MiniHBaseCluster(HBaseConfiguration conf, int nRegionNodes,
   public MiniHBaseCluster(HBaseConfiguration conf, int nRegionNodes,
-      MiniDFSCluster dfsCluster) throws IOException {
+      MiniDFSCluster dfsCluster, boolean deleteOnExit) throws IOException {
 
 
     this.conf = conf;
     this.conf = conf;
     this.fs = dfsCluster.getFileSystem();
     this.fs = dfsCluster.getFileSystem();
     this.cluster = dfsCluster;
     this.cluster = dfsCluster;
     this.shutdownDFS = false;
     this.shutdownDFS = false;
+    this.deleteOnExit = deleteOnExit;
     init(nRegionNodes);
     init(nRegionNodes);
   }
   }
 
 

+ 4 - 6
src/contrib/hbase/src/test/org/apache/hadoop/hbase/MultiRegionTable.java

@@ -45,14 +45,14 @@ public class MultiRegionTable extends HBaseTestCase {
    * daughter splits release all references.
    * daughter splits release all references.
    * @param conf
    * @param conf
    * @param cluster
    * @param cluster
-   * @param localFs
+   * @param fs
    * @param tableName
    * @param tableName
    * @param columnName
    * @param columnName
    * @throws IOException
    * @throws IOException
    */
    */
   @SuppressWarnings("null")
   @SuppressWarnings("null")
   public static void makeMultiRegionTable(HBaseConfiguration conf,
   public static void makeMultiRegionTable(HBaseConfiguration conf,
-      MiniHBaseCluster cluster, FileSystem localFs, String tableName,
+      MiniHBaseCluster cluster, FileSystem fs, String tableName,
       String columnName) throws IOException {  
       String columnName) throws IOException {  
     final int retries = 10; 
     final int retries = 10; 
     final long waitTime = 20L * 1000L;
     final long waitTime = 20L * 1000L;
@@ -63,8 +63,6 @@ public class MultiRegionTable extends HBaseTestCase {
     assertTrue(conf.getLong("hbase.hregion.max.filesize",
     assertTrue(conf.getLong("hbase.hregion.max.filesize",
       HConstants.DEFAULT_MAX_FILE_SIZE) <= 1024 * 1024);
       HConstants.DEFAULT_MAX_FILE_SIZE) <= 1024 * 1024);
 
 
-    FileSystem fs = (cluster.getDFSCluster() == null) ?
-      localFs : cluster.getDFSCluster().getFileSystem();
     assertNotNull(fs);
     assertNotNull(fs);
     Path d = fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR)));
     Path d = fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR)));
 
 
@@ -135,8 +133,8 @@ public class MultiRegionTable extends HBaseTestCase {
       Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITA));
       Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITA));
     HRegionInfo splitB =
     HRegionInfo splitB =
       Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITB));
       Writables.getHRegionInfoOrNull(data.get(HConstants.COL_SPLITB));
-    Path parentDir = HRegion.getRegionDir(d,
-        HRegionInfo.encodeRegionName(parent.getRegionName()));
+    Path parentDir = HRegion.getRegionDir(new Path(d, tableName),
+        parent.getEncodedName());
     assertTrue(fs.exists(parentDir));
     assertTrue(fs.exists(parentDir));
     LOG.info("Split happened. Parent is " + parent.getRegionName() +
     LOG.info("Split happened. Parent is " + parent.getRegionName() +
         " and daughters are " +
         " and daughters are " +

+ 13 - 7
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompaction.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+import org.apache.hadoop.dfs.MiniDFSCluster;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -32,7 +34,6 @@ import org.apache.hadoop.io.Text;
  */
  */
 public class TestCompaction extends HBaseTestCase {
 public class TestCompaction extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestCompaction.class.getName());
   static final Log LOG = LogFactory.getLog(TestCompaction.class.getName());
-  private HLog hlog = null;
   private HRegion r = null;
   private HRegion r = null;
   private static final String COLUMN_FAMILY = COLFAMILY_NAME1;
   private static final String COLUMN_FAMILY = COLFAMILY_NAME1;
   private final Text STARTROW;
   private final Text STARTROW;
@@ -40,6 +41,8 @@ public class TestCompaction extends HBaseTestCase {
   private static final Text COLUMN_FAMILY_TEXT_MINUS_COLON =
   private static final Text COLUMN_FAMILY_TEXT_MINUS_COLON =
     new Text(COLUMN_FAMILY.substring(0, COLUMN_FAMILY.length() - 1));
     new Text(COLUMN_FAMILY.substring(0, COLUMN_FAMILY.length() - 1));
   private static final int COMPACTION_THRESHOLD = MAXVERSIONS;
   private static final int COMPACTION_THRESHOLD = MAXVERSIONS;
+
+  private MiniDFSCluster cluster;
   
   
   /** constructor */
   /** constructor */
   public TestCompaction() {
   public TestCompaction() {
@@ -48,24 +51,27 @@ public class TestCompaction extends HBaseTestCase {
     
     
     // Set cache flush size to 1MB
     // Set cache flush size to 1MB
     conf.setInt("hbase.hregion.memcache.flush.size", 1024*1024);
     conf.setInt("hbase.hregion.memcache.flush.size", 1024*1024);
+    this.cluster = null;
   }
   }
   
   
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public void setUp() throws Exception {
   public void setUp() throws Exception {
+    this.cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
     super.setUp();
     super.setUp();
-    this.hlog = new HLog(this.localFs, this.testDir, this.conf, null);
     HTableDescriptor htd = createTableDescriptor(getName());
     HTableDescriptor htd = createTableDescriptor(getName());
-    HRegionInfo hri = new HRegionInfo(htd, null, null);
-    this.r =
-      new HRegion(testDir, hlog, this.localFs, this.conf, hri, null, null);
+    this.r = createNewHRegion(htd, null, null);
   }
   }
   
   
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public void tearDown() throws Exception {
   public void tearDown() throws Exception {
+    HLog hlog = r.getLog();
     this.r.close();
     this.r.close();
-    this.hlog.closeAndDelete();
+    hlog.closeAndDelete();
+    if (this.cluster != null) {
+      StaticTestEnvironment.shutdownDfs(cluster);
+    }
     super.tearDown();
     super.tearDown();
   }
   }
   
   
@@ -158,7 +164,7 @@ public class TestCompaction extends HBaseTestCase {
     assertNull(bytes);
     assertNull(bytes);
     // Assert the store files do not have the first record 'aaa' keys in them.
     // Assert the store files do not have the first record 'aaa' keys in them.
     for (MapFile.Reader reader:
     for (MapFile.Reader reader:
-        this.r.stores.get(COLUMN_FAMILY_TEXT_MINUS_COLON).readers.values()) {
+        this.r.stores.get(COLUMN_FAMILY_TEXT_MINUS_COLON).getReaders()) {
       reader.reset();
       reader.reset();
       HStoreKey key = new HStoreKey();
       HStoreKey key = new HStoreKey();
       ImmutableBytesWritable val = new ImmutableBytesWritable();
       ImmutableBytesWritable val = new ImmutableBytesWritable();

+ 12 - 16
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteAll.java

@@ -21,27 +21,27 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Map;
 import java.util.Map;
-import java.util.TreeMap;
 
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.hbase.filter.StopRowFilter;
-import org.apache.hadoop.hbase.filter.WhileMatchRowFilter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.commons.logging.*;
 import org.apache.commons.logging.*;
 
 
-import java.util.List;
-
 /**
 /**
  * Test the functionality of deleteAll.
  * Test the functionality of deleteAll.
  */
  */
 public class TestDeleteAll extends HBaseTestCase {
 public class TestDeleteAll extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestDeleteAll.class);
   static final Log LOG = LogFactory.getLog(TestDeleteAll.class);
   private MiniDFSCluster miniHdfs;
   private MiniDFSCluster miniHdfs;
-
+  
   @Override
   @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
-    this.miniHdfs = new MiniDFSCluster(this.conf, 1, true, null);
+    try {
+      this.miniHdfs = new MiniDFSCluster(this.conf, 1, true, null);
+    } catch (Exception e) {
+      LOG.fatal("error starting MiniDFSCluster", e);
+      throw e;
+    }
   }
   }
   
   
   /**
   /**
@@ -51,14 +51,9 @@ public class TestDeleteAll extends HBaseTestCase {
   public void testDeleteAll() throws Exception {
   public void testDeleteAll() throws Exception {
     HRegion region = null;
     HRegion region = null;
     HRegionIncommon region_incommon = null;
     HRegionIncommon region_incommon = null;
-    HLog hlog = new HLog(this.miniHdfs.getFileSystem(), this.testDir,
-      this.conf, null);
-    
-    try{
+    try {
       HTableDescriptor htd = createTableDescriptor(getName());
       HTableDescriptor htd = createTableDescriptor(getName());
-      HRegionInfo hri = new HRegionInfo(htd, null, null);
-      region = new HRegion(this.testDir, hlog, this.miniHdfs.getFileSystem(),
-        this.conf, hri, null, null);
+      region = createNewHRegion(htd, null, null);
       region_incommon = new HRegionIncommon(region);
       region_incommon = new HRegionIncommon(region);
       
       
       // test memcache
       // test memcache
@@ -73,8 +68,8 @@ public class TestDeleteAll extends HBaseTestCase {
         } catch (Exception e) {
         } catch (Exception e) {
           e.printStackTrace();
           e.printStackTrace();
         }
         }
+        region.getLog().closeAndDelete();
       }
       }
-      hlog.closeAndDelete();
     }
     }
   }
   }
     
     
@@ -157,7 +152,8 @@ public class TestDeleteAll extends HBaseTestCase {
   private String cellData(int tsNum, boolean flush){
   private String cellData(int tsNum, boolean flush){
     return "t" + tsNum + " data" + (flush ? " - with flush" : "");
     return "t" + tsNum + " data" + (flush ? " - with flush" : "");
   }
   }
-  
+
+  @Override
   protected void tearDown() throws Exception {
   protected void tearDown() throws Exception {
     if (this.miniHdfs != null) {
     if (this.miniHdfs != null) {
       StaticTestEnvironment.shutdownDfs(this.miniHdfs);
       StaticTestEnvironment.shutdownDfs(this.miniHdfs);

+ 8 - 10
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDeleteFamily.java

@@ -22,14 +22,16 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Map;
 import java.util.Map;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
-import org.apache.commons.logging.*;
 
 
 /**
 /**
  * Test the functionality of deleteFamily.
  * Test the functionality of deleteFamily.
  */
  */
 public class TestDeleteFamily extends HBaseTestCase {
 public class TestDeleteFamily extends HBaseTestCase {
+  static final Log LOG = LogFactory.getLog(TestDeleteFamily.class);
   private MiniDFSCluster miniHdfs;
   private MiniDFSCluster miniHdfs;
 
 
   @Override
   @Override
@@ -45,14 +47,9 @@ public class TestDeleteFamily extends HBaseTestCase {
   public void testDeleteFamily() throws Exception {
   public void testDeleteFamily() throws Exception {
     HRegion region = null;
     HRegion region = null;
     HRegionIncommon region_incommon = null;
     HRegionIncommon region_incommon = null;
-    HLog hlog = new HLog(this.miniHdfs.getFileSystem(), this.testDir,
-      this.conf, null);
-    
-    try{
+    try {
       HTableDescriptor htd = createTableDescriptor(getName());
       HTableDescriptor htd = createTableDescriptor(getName());
-      HRegionInfo hri = new HRegionInfo(htd, null, null);
-      region = new HRegion(this.testDir, hlog, this.miniHdfs.getFileSystem(),
-        this.conf, hri, null, null);
+      region = createNewHRegion(htd, null, null);
       region_incommon = new HRegionIncommon(region);
       region_incommon = new HRegionIncommon(region);
       
       
       // test memcache
       // test memcache
@@ -67,8 +64,8 @@ public class TestDeleteFamily extends HBaseTestCase {
         } catch (Exception e) {
         } catch (Exception e) {
           e.printStackTrace();
           e.printStackTrace();
         }
         }
+        region.getLog().closeAndDelete();
       }
       }
-      hlog.closeAndDelete();
     }
     }
   }
   }
     
     
@@ -156,7 +153,8 @@ public class TestDeleteFamily extends HBaseTestCase {
   private String cellData(int tsNum, boolean flush){
   private String cellData(int tsNum, boolean flush){
     return "t" + tsNum + " data" + (flush ? " - with flush" : "");
     return "t" + tsNum + " data" + (flush ? " - with flush" : "");
   }
   }
-  
+
+  @Override
   protected void tearDown() throws Exception {
   protected void tearDown() throws Exception {
     if (this.miniHdfs != null) {
     if (this.miniHdfs != null) {
       this.miniHdfs.shutdown();
       this.miniHdfs.shutdown();

+ 14 - 42
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet.java

@@ -19,8 +19,6 @@
  */
  */
 package org.apache.hadoop.hbase;
 package org.apache.hadoop.hbase;
 
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map;
@@ -28,8 +26,6 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.util.Writables;
@@ -43,6 +39,7 @@ public class TestGet extends HBaseTestCase {
     new Text(HRegionInfo.rootRegionInfo.getRegionName());
     new Text(HRegionInfo.rootRegionInfo.getRegionName());
   private static final String SERVER_ADDRESS = "foo.bar.com:1234";
   private static final String SERVER_ADDRESS = "foo.bar.com:1234";
 
 
+
   
   
   private void verifyGet(final HRegionIncommon r, final String expectedServer)
   private void verifyGet(final HRegionIncommon r, final String expectedServer)
   throws IOException {
   throws IOException {
@@ -74,56 +71,35 @@ public class TestGet extends HBaseTestCase {
    */
    */
   public void testGet() throws IOException {
   public void testGet() throws IOException {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
+    HRegion region = null;
 
 
     try {
     try {
       
       
       // Initialization
       // Initialization
       
       
       cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
       cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
-      FileSystem fs = cluster.getFileSystem();
-      Path dir = new Path("/hbase");
-      fs.mkdirs(dir);
       
       
       HTableDescriptor desc = new HTableDescriptor("test");
       HTableDescriptor desc = new HTableDescriptor("test");
       desc.addFamily(new HColumnDescriptor(CONTENTS.toString()));
       desc.addFamily(new HColumnDescriptor(CONTENTS.toString()));
       desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
       desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
       
       
-      HRegionInfo info = new HRegionInfo(desc, null, null);
-      Path regionDir = HRegion.getRegionDir(dir,
-          HRegionInfo.encodeRegionName(info.getRegionName()));
-      fs.mkdirs(regionDir);
-      
-      HLog log = new HLog(fs, new Path(regionDir, "log"), conf, null);
-
-      HRegion region = new HRegion(dir, log, fs, conf, info, null, null);
+      region = createNewHRegion(desc, null, null);
       HRegionIncommon r = new HRegionIncommon(region);
       HRegionIncommon r = new HRegionIncommon(region);
       
       
       // Write information to the table
       // Write information to the table
       
       
       long lockid = r.startUpdate(ROW_KEY);
       long lockid = r.startUpdate(ROW_KEY);
-      ByteArrayOutputStream bytes = new ByteArrayOutputStream();
-      DataOutputStream s = new DataOutputStream(bytes);
-      CONTENTS.write(s);
-      r.put(lockid, CONTENTS, bytes.toByteArray());
-
-      bytes.reset();
-      HRegionInfo.rootRegionInfo.write(s);
-      
+      r.put(lockid, CONTENTS, Writables.getBytes(CONTENTS));
       r.put(lockid, HConstants.COL_REGIONINFO, 
       r.put(lockid, HConstants.COL_REGIONINFO, 
           Writables.getBytes(HRegionInfo.rootRegionInfo));
           Writables.getBytes(HRegionInfo.rootRegionInfo));
-      
       r.commit(lockid, System.currentTimeMillis());
       r.commit(lockid, System.currentTimeMillis());
       
       
       lockid = r.startUpdate(ROW_KEY);
       lockid = r.startUpdate(ROW_KEY);
-
       r.put(lockid, HConstants.COL_SERVER, 
       r.put(lockid, HConstants.COL_SERVER, 
         Writables.stringToBytes(new HServerAddress(SERVER_ADDRESS).toString()));
         Writables.stringToBytes(new HServerAddress(SERVER_ADDRESS).toString()));
-      
       r.put(lockid, HConstants.COL_STARTCODE, Writables.longToBytes(lockid));
       r.put(lockid, HConstants.COL_STARTCODE, Writables.longToBytes(lockid));
-      
       r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "region"), 
       r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "region"), 
         "region".getBytes(HConstants.UTF8_ENCODING));
         "region".getBytes(HConstants.UTF8_ENCODING));
-
       r.commit(lockid, System.currentTimeMillis());
       r.commit(lockid, System.currentTimeMillis());
       
       
       // Verify that get works the same from memcache as when reading from disk
       // Verify that get works the same from memcache as when reading from disk
@@ -134,8 +110,7 @@ public class TestGet extends HBaseTestCase {
       // Close and re-open region, forcing updates to disk
       // Close and re-open region, forcing updates to disk
       
       
       region.close();
       region.close();
-      log.rollWriter();
-      region = new HRegion(dir, log, fs, conf, info, null, null);
+      region = openClosedRegion(region);
       r = new HRegionIncommon(region);
       r = new HRegionIncommon(region);
       
       
       // Read it back
       // Read it back
@@ -145,17 +120,13 @@ public class TestGet extends HBaseTestCase {
       // Update one family member and add a new one
       // Update one family member and add a new one
       
       
       lockid = r.startUpdate(ROW_KEY);
       lockid = r.startUpdate(ROW_KEY);
-
       r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "region"),
       r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "region"),
         "region2".getBytes(HConstants.UTF8_ENCODING));
         "region2".getBytes(HConstants.UTF8_ENCODING));
-
       String otherServerName = "bar.foo.com:4321";
       String otherServerName = "bar.foo.com:4321";
       r.put(lockid, HConstants.COL_SERVER, 
       r.put(lockid, HConstants.COL_SERVER, 
         Writables.stringToBytes(new HServerAddress(otherServerName).toString()));
         Writables.stringToBytes(new HServerAddress(otherServerName).toString()));
-      
       r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "junk"),
       r.put(lockid, new Text(HConstants.COLUMN_FAMILY + "junk"),
         "junk".getBytes(HConstants.UTF8_ENCODING));
         "junk".getBytes(HConstants.UTF8_ENCODING));
-      
       r.commit(lockid, System.currentTimeMillis());
       r.commit(lockid, System.currentTimeMillis());
 
 
       verifyGet(r, otherServerName);
       verifyGet(r, otherServerName);
@@ -163,21 +134,22 @@ public class TestGet extends HBaseTestCase {
       // Close region and re-open it
       // Close region and re-open it
       
       
       region.close();
       region.close();
-      log.rollWriter();
-      region = new HRegion(dir, log, fs, conf, info, null, null);
+      region = openClosedRegion(region);
       r = new HRegionIncommon(region);
       r = new HRegionIncommon(region);
 
 
       // Read it back
       // Read it back
       
       
       verifyGet(r, otherServerName);
       verifyGet(r, otherServerName);
 
 
-      // Close region once and for all
-      
-      region.close();
-      log.closeAndDelete();
-      
     } finally {
     } finally {
-      StaticTestEnvironment.shutdownDfs(cluster);
+      if (region != null) {
+        // Close region once and for all
+        region.close();
+        region.getLog().closeAndDelete();
+      }
+      if (cluster != null) {
+        StaticTestEnvironment.shutdownDfs(cluster);
+      }
     }
     }
   }
   }
 }
 }

+ 8 - 14
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestGet2.java

@@ -49,13 +49,9 @@ public class TestGet2 extends HBaseTestCase {
   public void testGetFull() throws Exception {
   public void testGetFull() throws Exception {
     HRegion region = null;
     HRegion region = null;
     HScannerInterface scanner = null;
     HScannerInterface scanner = null;
-    HLog hlog = new HLog(this.miniHdfs.getFileSystem(), this.testDir,
-      this.conf, null);
     try {
     try {
       HTableDescriptor htd = createTableDescriptor(getName());
       HTableDescriptor htd = createTableDescriptor(getName());
-      HRegionInfo hri = new HRegionInfo(htd, null, null);
-      region = new HRegion(this.testDir, hlog, this.miniHdfs.getFileSystem(),
-        this.conf, hri, null, null);
+      region = createNewHRegion(htd, null, null);
       for (int i = 0; i < COLUMNS.length; i++) {
       for (int i = 0; i < COLUMNS.length; i++) {
         addContent(region, COLUMNS[i].toString());
         addContent(region, COLUMNS[i].toString());
       }
       }
@@ -95,22 +91,20 @@ public class TestGet2 extends HBaseTestCase {
         } catch (Exception e) {
         } catch (Exception e) {
           e.printStackTrace();
           e.printStackTrace();
         }
         }
+        region.getLog().closeAndDelete();
       }
       }
-      hlog.closeAndDelete();
     }
     }
   }
   }
   
   
+  /**
+   * @throws IOException
+   */
   public void testGetAtTimestamp() throws IOException{
   public void testGetAtTimestamp() throws IOException{
     HRegion region = null;
     HRegion region = null;
     HRegionIncommon region_incommon = null;
     HRegionIncommon region_incommon = null;
-    HLog hlog = new HLog(this.miniHdfs.getFileSystem(), this.testDir,
-      this.conf, null);
-
     try {
     try {
       HTableDescriptor htd = createTableDescriptor(getName());
       HTableDescriptor htd = createTableDescriptor(getName());
-      HRegionInfo hri = new HRegionInfo(htd, null, null);
-      region = new HRegion(this.testDir, hlog, this.miniHdfs.getFileSystem(),
-        this.conf, hri, null, null);
+      region = createNewHRegion(htd, null, null);
       region_incommon = new HRegionIncommon(region);
       region_incommon = new HRegionIncommon(region);
       
       
       long right_now = System.currentTimeMillis();
       long right_now = System.currentTimeMillis();
@@ -141,10 +135,9 @@ public class TestGet2 extends HBaseTestCase {
         } catch (Exception e) {
         } catch (Exception e) {
           e.printStackTrace();
           e.printStackTrace();
         }
         }
+        region.getLog().closeAndDelete();
       }
       }
-      hlog.closeAndDelete();
     }
     }
-    
   }
   }
   
   
   
   
@@ -176,6 +169,7 @@ public class TestGet2 extends HBaseTestCase {
     assertEquals("count of columns", columnCount, COLUMNS.length);
     assertEquals("count of columns", columnCount, COLUMNS.length);
   }
   }
 
 
+  @Override
   protected void tearDown() throws Exception {
   protected void tearDown() throws Exception {
     if (this.miniHdfs != null) {
     if (this.miniHdfs != null) {
       this.miniHdfs.shutdown();
       this.miniHdfs.shutdown();

+ 3 - 10
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java

@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.util.TreeMap;
 import java.util.TreeMap;
 
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -31,21 +30,15 @@ import org.apache.hadoop.io.SequenceFile.Reader;
 
 
 /** JUnit test case for HLog */
 /** JUnit test case for HLog */
 public class TestHLog extends HBaseTestCase implements HConstants {
 public class TestHLog extends HBaseTestCase implements HConstants {
-  private final Path dir = new Path("/hbase");
-  private FileSystem fs;
+  private Path dir;
   private MiniDFSCluster cluster;
   private MiniDFSCluster cluster;
 
 
-  /** constructor */
-  public TestHLog() {
-    this.cluster = null;
-  }
-  
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public void setUp() throws Exception {
   public void setUp() throws Exception {
-    super.setUp();
     cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
     cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
-    this.fs = cluster.getFileSystem();
+    super.setUp();
+    this.dir = new Path("/hbase", getName());
     if (fs.exists(dir)) {
     if (fs.exists(dir)) {
       fs.delete(dir);
       fs.delete(dir);
     }
     }

+ 14 - 13
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java

@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.TreeMap;
 import java.util.TreeMap;
 
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
@@ -39,7 +38,8 @@ import org.apache.log4j.Logger;
  * A lot of the meta information for an HRegion now lives inside other
  * A lot of the meta information for an HRegion now lives inside other
  * HRegions or in the HBaseMaster, so only basic testing is possible.
  * HRegions or in the HBaseMaster, so only basic testing is possible.
  */
  */
-public class TestHRegion extends HBaseTestCase implements RegionUnavailableListener {
+public class TestHRegion extends HBaseTestCase
+implements RegionUnavailableListener {
   static final Logger LOG =
   static final Logger LOG =
     Logger.getLogger(TestHRegion.class.getName());
     Logger.getLogger(TestHRegion.class.getName());
   
   
@@ -61,6 +61,12 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe
       read();
       read();
       cleanup();
       cleanup();
     } finally {
     } finally {
+      if (r != null) {
+        r.close();
+      }
+      if (log != null) {
+        log.closeAndDelete();
+      }
       StaticTestEnvironment.shutdownDfs(cluster);
       StaticTestEnvironment.shutdownDfs(cluster);
     }
     }
   }
   }
@@ -78,9 +84,6 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe
   private static final Text ANCHOR_SECONDCOL = new Text("anchor:secondcol");
   private static final Text ANCHOR_SECONDCOL = new Text("anchor:secondcol");
   
   
   private MiniDFSCluster cluster = null;
   private MiniDFSCluster cluster = null;
-  private FileSystem fs = null;
-  private Path parentdir = null;
-  private Path newlogdir = null;
   private HLog log = null;
   private HLog log = null;
   private HTableDescriptor desc = null;
   private HTableDescriptor desc = null;
   HRegion r = null;
   HRegion r = null;
@@ -93,17 +96,12 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe
   private void setup() throws IOException {
   private void setup() throws IOException {
 
 
     cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
     cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
-    fs = cluster.getFileSystem();
-    parentdir = new Path("/hbase");
-    fs.mkdirs(parentdir);
-    newlogdir = new Path(parentdir, "log");
 
 
-    log = new HLog(fs, newlogdir, conf, null);
     desc = new HTableDescriptor("test");
     desc = new HTableDescriptor("test");
     desc.addFamily(new HColumnDescriptor("contents:"));
     desc.addFamily(new HColumnDescriptor("contents:"));
     desc.addFamily(new HColumnDescriptor("anchor:"));
     desc.addFamily(new HColumnDescriptor("anchor:"));
-    r = new HRegion(parentdir, log, fs, conf, 
-        new HRegionInfo(desc, null, null), null, null);
+    r = createNewHRegion(desc, null, null);
+    log = r.getLog();
     region = new HRegionIncommon(r);
     region = new HRegionIncommon(r);
   }
   }
 
 
@@ -162,7 +160,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe
         + ((System.currentTimeMillis() - startTime) / 1000.0));
         + ((System.currentTimeMillis() - startTime) / 1000.0));
   }
   }
   
   
-  private void badPuts() throws IOException {
+  private void badPuts() {
     
     
     // Try put with bad lockid.
     // Try put with bad lockid.
     boolean exceptionThrown = false;
     boolean exceptionThrown = false;
@@ -799,7 +797,10 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe
   
   
   private void cleanup() {
   private void cleanup() {
     try {
     try {
+      r.close();
+      r = null;
       log.closeAndDelete();
       log.closeAndDelete();
+      log = null;
     } catch (IOException e) {
     } catch (IOException e) {
       e.printStackTrace();
       e.printStackTrace();
     }
     }

+ 6 - 9
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java

@@ -24,7 +24,6 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.MapFile;
@@ -39,7 +38,6 @@ public class TestHStoreFile extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
   static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
   private static String DIR = "/";
   private static String DIR = "/";
   private MiniDFSCluster cluster;
   private MiniDFSCluster cluster;
-  private FileSystem fs;
   private Path dir = null;
   private Path dir = null;
   
   
   /** {@inheritDoc} */
   /** {@inheritDoc} */
@@ -47,7 +45,6 @@ public class TestHStoreFile extends HBaseTestCase {
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     try {
     try {
       this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
       this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
-      this.fs = cluster.getFileSystem();
       this.dir = new Path(DIR, getName());
       this.dir = new Path(DIR, getName());
     } catch (IOException e) {
     } catch (IOException e) {
       StaticTestEnvironment.shutdownDfs(cluster);
       StaticTestEnvironment.shutdownDfs(cluster);
@@ -121,8 +118,8 @@ public class TestHStoreFile extends HBaseTestCase {
   public void testReference()
   public void testReference()
   throws IOException {
   throws IOException {
     // Make a store file and write data to it.
     // Make a store file and write data to it.
-    HStoreFile hsf = new HStoreFile(this.conf, this.dir, getName(),
-        new Text("colfamily"), 1234567890L);
+    HStoreFile hsf = new HStoreFile(this.conf, this.fs, this.dir, getName(),
+        new Text("colfamily"), 1234567890L, null);
     MapFile.Writer writer =
     MapFile.Writer writer =
       hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, null);
       hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, null);
     writeStoreFile(writer);
     writeStoreFile(writer);
@@ -138,16 +135,16 @@ public class TestHStoreFile extends HBaseTestCase {
     HStoreFile.Reference reference =
     HStoreFile.Reference reference =
       new HStoreFile.Reference(hsf.getEncodedRegionName(), hsf.getFileId(),
       new HStoreFile.Reference(hsf.getEncodedRegionName(), hsf.getFileId(),
           midkey, HStoreFile.Range.top);
           midkey, HStoreFile.Range.top);
-    HStoreFile refHsf = new HStoreFile(this.conf, new Path(DIR, getName()),
-        getName() + "_reference", hsf.getColFamily(), 456,
-        reference);
+    HStoreFile refHsf = new HStoreFile(this.conf, this.fs, 
+        new Path(DIR, getName()), getName() + "_reference", hsf.getColFamily(),
+        456, reference);
     // Assert that reference files are written and that we can write and
     // Assert that reference files are written and that we can write and
     // read the info reference file at least.
     // read the info reference file at least.
     refHsf.writeReferenceFiles(this.fs);
     refHsf.writeReferenceFiles(this.fs);
     assertTrue(this.fs.exists(refHsf.getMapFilePath()));
     assertTrue(this.fs.exists(refHsf.getMapFilePath()));
     assertTrue(this.fs.exists(refHsf.getInfoFilePath()));
     assertTrue(this.fs.exists(refHsf.getInfoFilePath()));
     HStoreFile.Reference otherReference =
     HStoreFile.Reference otherReference =
-      HStoreFile.readSplitInfo(refHsf.getInfoFilePath(), this.fs);
+      HStore.readSplitInfo(refHsf.getInfoFilePath(), this.fs);
     assertEquals(reference.getEncodedRegionName(),
     assertEquals(reference.getEncodedRegionName(),
         otherReference.getEncodedRegionName());
         otherReference.getEncodedRegionName());
     assertEquals(reference.getFileId(),
     assertEquals(reference.getFileId(),

+ 6 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestInfoServers.java

@@ -33,15 +33,20 @@ import org.apache.hadoop.io.Text;
  */
  */
 public class TestInfoServers extends HBaseTestCase {
 public class TestInfoServers extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestInfoServers.class);
   static final Log LOG = LogFactory.getLog(TestInfoServers.class);
-  
+
+  @Override  
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
   }
   }
 
 
+  @Override
   protected void tearDown() throws Exception {
   protected void tearDown() throws Exception {
     super.tearDown();
     super.tearDown();
   }
   }
   
   
+  /**
+   * @throws Exception
+   */
   public void testInfoServersAreUp() throws Exception {
   public void testInfoServersAreUp() throws Exception {
     // Bring up info servers on 'odd' port numbers in case the test is not
     // Bring up info servers on 'odd' port numbers in case the test is not
     // sourcing the src/test/hbase-default.xml.
     // sourcing the src/test/hbase-default.xml.

+ 1 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestLogRolling.java

@@ -118,7 +118,7 @@ public class TestLogRolling extends HBaseTestCase {
   }
   }
   
   
   private void startAndWriteData() throws Exception {
   private void startAndWriteData() throws Exception {
-    cluster = new MiniHBaseCluster(conf, 1, dfs);
+    cluster = new MiniHBaseCluster(conf, 1, dfs, true);
     try {
     try {
       Thread.sleep(10 * 1000);                  // Wait for region server to start
       Thread.sleep(10 * 1000);                  // Wait for region server to start
     } catch (InterruptedException e) {
     } catch (InterruptedException e) {

+ 1 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeMeta.java

@@ -37,6 +37,6 @@ public class TestMergeMeta extends AbstractMergeTestBase {
    */
    */
   public void testMergeMeta() throws IOException {
   public void testMergeMeta() throws IOException {
     assertNotNull(dfsCluster);
     assertNotNull(dfsCluster);
-    HMerge.merge(conf, fs, HConstants.META_TABLE_NAME);
+    HMerge.merge(conf, dfsCluster.getFileSystem(), HConstants.META_TABLE_NAME);
   }
   }
 }  
 }  

+ 2 - 2
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeTable.java

@@ -32,9 +32,9 @@ public class TestMergeTable extends AbstractMergeTestBase {
    */
    */
   public void testMergeTable() throws IOException {
   public void testMergeTable() throws IOException {
     assertNotNull(dfsCluster);
     assertNotNull(dfsCluster);
-    MiniHBaseCluster hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
+    MiniHBaseCluster hCluster = new MiniHBaseCluster(conf, 1, dfsCluster, true);
     try {
     try {
-      HMerge.merge(conf, fs, desc.getName());
+      HMerge.merge(conf, dfsCluster.getFileSystem(), desc.getName());
     } finally {
     } finally {
       hCluster.shutdown();
       hCluster.shutdown();
     }
     }

+ 14 - 28
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java

@@ -25,8 +25,6 @@ import java.io.IOException;
 import java.util.TreeMap;
 import java.util.TreeMap;
 
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
@@ -50,9 +48,18 @@ public class TestScanner extends HBaseTestCase {
   
   
   private static final long START_CODE = Long.MAX_VALUE;
   private static final long START_CODE = Long.MAX_VALUE;
 
 
+  private MiniDFSCluster cluster = null;
   private HRegion r;
   private HRegion r;
   private HRegionIncommon region;
   private HRegionIncommon region;
 
 
+  /** {@inheritDoc} */
+  @Override
+  public void setUp() throws Exception {
+    cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
+    super.setUp();
+    
+  }
+  
   /** Compare the HRegionInfo we read from HBase to what we stored */
   /** Compare the HRegionInfo we read from HBase to what we stored */
   private void validateRegionInfo(byte [] regionBytes) throws IOException {
   private void validateRegionInfo(byte [] regionBytes) throws IOException {
     HRegionInfo info =
     HRegionInfo info =
@@ -127,26 +134,8 @@ public class TestScanner extends HBaseTestCase {
    * @throws IOException
    * @throws IOException
    */
    */
   public void testScanner() throws IOException {
   public void testScanner() throws IOException {
-    MiniDFSCluster cluster = null;
-    FileSystem fs = null;
-    
     try {
     try {
-      
-      // Initialization
-      
-      HBaseConfiguration conf = new HBaseConfiguration();
-      cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
-      fs = cluster.getFileSystem();
-      Path dir = new Path("/hbase");
-      fs.mkdirs(dir);
-      
-      Path regionDir = HRegion.getRegionDir(dir,
-          HRegionInfo.encodeRegionName(REGION_INFO.getRegionName()));
-      fs.mkdirs(regionDir);
-      
-      HLog log = new HLog(fs, new Path(regionDir, "log"), conf, null);
-
-      r = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
+      r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
       region = new HRegionIncommon(r);
       region = new HRegionIncommon(r);
       
       
       // Write information to the meta table
       // Write information to the meta table
@@ -168,8 +157,7 @@ public class TestScanner extends HBaseTestCase {
       // Close and re-open
       // Close and re-open
       
       
       r.close();
       r.close();
-      log.rollWriter();
-      r = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
+      r = openClosedRegion(r);
       region = new HRegionIncommon(r);
       region = new HRegionIncommon(r);
 
 
       // Verify we can get the data back now that it is on disk.
       // Verify we can get the data back now that it is on disk.
@@ -209,8 +197,7 @@ public class TestScanner extends HBaseTestCase {
       // Close and reopen
       // Close and reopen
       
       
       r.close();
       r.close();
-      log.rollWriter();
-      r = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
+      r = openClosedRegion(r);
       region = new HRegionIncommon(r);
       region = new HRegionIncommon(r);
 
 
       // Validate again
       // Validate again
@@ -246,8 +233,7 @@ public class TestScanner extends HBaseTestCase {
       // Close and reopen
       // Close and reopen
       
       
       r.close();
       r.close();
-      log.rollWriter();
-      r = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
+      r = openClosedRegion(r);
       region = new HRegionIncommon(r);
       region = new HRegionIncommon(r);
 
 
       // Validate again
       // Validate again
@@ -258,7 +244,7 @@ public class TestScanner extends HBaseTestCase {
       // clean up
       // clean up
       
       
       r.close();
       r.close();
-      log.closeAndDelete();
+      r.getLog().closeAndDelete();
       
       
     } finally {
     } finally {
       StaticTestEnvironment.shutdownDfs(cluster);
       StaticTestEnvironment.shutdownDfs(cluster);

+ 8 - 11
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java

@@ -24,6 +24,7 @@ import java.util.TreeMap;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
@@ -64,18 +65,21 @@ public class TestSplit extends MultiRegionTable {
    * @throws Exception
    * @throws Exception
    */
    */
   public void testBasicSplit() throws Exception {
   public void testBasicSplit() throws Exception {
+    MiniDFSCluster cluster = null;
     HRegion region = null;
     HRegion region = null;
-    HLog hlog = new HLog(this.localFs, this.testDir, this.conf, null);
     try {
     try {
+      cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
       HTableDescriptor htd = createTableDescriptor(getName());
       HTableDescriptor htd = createTableDescriptor(getName());
-      HRegionInfo hri = new HRegionInfo(htd, null, null);
-      region = new HRegion(testDir, hlog, this.localFs, this.conf, hri, null, null);
+      region = createNewHRegion(htd, null, null);
       basicSplit(region);
       basicSplit(region);
     } finally {
     } finally {
       if (region != null) {
       if (region != null) {
         region.close();
         region.close();
+        region.getLog().closeAndDelete();
+      }
+      if (cluster != null) {
+        StaticTestEnvironment.shutdownDfs(cluster);
       }
       }
-      hlog.closeAndDelete();
     }
     }
   }
   }
   
   
@@ -166,13 +170,6 @@ public class TestSplit extends MultiRegionTable {
     }
     }
   }
   }
   
   
-  private HRegion openClosedRegion(final HRegion closedRegion)
-  throws IOException {
-    return new HRegion(closedRegion.getRootDir(), closedRegion.getLog(),
-      closedRegion.getFilesystem(), closedRegion.getConf(),
-      closedRegion.getRegionInfo(), null, null);
-  }
-
   private void assertGet(final HRegion r, final String family, final Text k)
   private void assertGet(final HRegion r, final String family, final Text k)
   throws IOException {
   throws IOException {
     // Now I have k, get values out and assert they are as expected.
     // Now I have k, get values out and assert they are as expected.

+ 25 - 3
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.TreeMap;
 import java.util.TreeMap;
 
 
+import org.apache.hadoop.dfs.MiniDFSCluster;
+
 import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
 import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -43,7 +45,29 @@ public class TestTimestamp extends HBaseTestCase {
   // When creating column descriptor, how many versions of a cell to allow.
   // When creating column descriptor, how many versions of a cell to allow.
   private static final int VERSIONS = 3;
   private static final int VERSIONS = 3;
   
   
+  private MiniDFSCluster cluster;
+
+  /** constructor */
+  public TestTimestamp() {
+    super();
+    this.cluster = null;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void setUp() throws Exception {
+    this.cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
+    super.setUp();
+  }
 
 
+  /** {@inheritDoc} */
+  @Override
+  public void tearDown() throws Exception {
+    if (this.cluster != null) {
+      StaticTestEnvironment.shutdownDfs(cluster);
+    }
+  }
+  
   /**
   /**
    * Test that delete works according to description in <a
    * Test that delete works according to description in <a
    * href="https://issues.apache.org/jira/browse/HADOOP-1784">hadoop-1784</a>.
    * href="https://issues.apache.org/jira/browse/HADOOP-1784">hadoop-1784</a>.
@@ -310,11 +334,9 @@ public class TestTimestamp extends HBaseTestCase {
   }
   }
   
   
   private HRegion createRegion() throws IOException {
   private HRegion createRegion() throws IOException {
-    HLog hlog = new HLog(this.localFs, this.testDir, this.conf, null);
     HTableDescriptor htd = createTableDescriptor(getName());
     HTableDescriptor htd = createTableDescriptor(getName());
     htd.addFamily(new HColumnDescriptor(COLUMN, VERSIONS,
     htd.addFamily(new HColumnDescriptor(COLUMN, VERSIONS,
       CompressionType.NONE, false, Integer.MAX_VALUE, null));
       CompressionType.NONE, false, Integer.MAX_VALUE, null));
-    HRegionInfo hri = new HRegionInfo(htd, null, null);
-    return new HRegion(testDir, hlog, this.localFs, this.conf, hri, null, null);
+    return createNewHRegion(htd, null, null);
   }
   }
 }
 }

+ 11 - 9
src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java

@@ -76,15 +76,12 @@ public class TestTableIndex extends MultiRegionTable {
   private HTableDescriptor desc;
   private HTableDescriptor desc;
 
 
   private MiniDFSCluster dfsCluster = null;
   private MiniDFSCluster dfsCluster = null;
-  private FileSystem fs;
   private Path dir;
   private Path dir;
   private MiniHBaseCluster hCluster = null;
   private MiniHBaseCluster hCluster = null;
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public void setUp() throws Exception {
   public void setUp() throws Exception {
-    super.setUp();
-
     // Make sure the cache gets flushed so we trigger a compaction(s) and
     // Make sure the cache gets flushed so we trigger a compaction(s) and
     // hence splits.
     // hence splits.
     conf.setInt("hbase.hregion.memcache.flush.size", 1024 * 1024);
     conf.setInt("hbase.hregion.memcache.flush.size", 1024 * 1024);
@@ -101,21 +98,25 @@ public class TestTableIndex extends MultiRegionTable {
     desc.addFamily(new HColumnDescriptor(OUTPUT_COLUMN));
     desc.addFamily(new HColumnDescriptor(OUTPUT_COLUMN));
 
 
     dfsCluster = new MiniDFSCluster(conf, 1, true, (String[]) null);
     dfsCluster = new MiniDFSCluster(conf, 1, true, (String[]) null);
-    try {
-      fs = dfsCluster.getFileSystem();
 
 
+    // Must call super.setUp after mini dfs cluster is started or else
+    // filesystem ends up being local
+    
+    super.setUp();
+
+    try {
       dir = new Path("/hbase");
       dir = new Path("/hbase");
       fs.mkdirs(dir);
       fs.mkdirs(dir);
 
 
       // Start up HBase cluster
       // Start up HBase cluster
-      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
+      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster, true);
 
 
       // Create a table.
       // Create a table.
       HBaseAdmin admin = new HBaseAdmin(conf);
       HBaseAdmin admin = new HBaseAdmin(conf);
       admin.createTable(desc);
       admin.createTable(desc);
 
 
       // Populate a table into multiple regions
       // Populate a table into multiple regions
-      makeMultiRegionTable(conf, hCluster, null, TABLE_NAME, INPUT_COLUMN);
+      makeMultiRegionTable(conf, hCluster, this.fs, TABLE_NAME, INPUT_COLUMN);
 
 
       // Verify table indeed has multiple regions
       // Verify table indeed has multiple regions
       HTable table = new HTable(conf, new Text(TABLE_NAME));
       HTable table = new HTable(conf, new Text(TABLE_NAME));
@@ -256,10 +257,11 @@ public class TestTableIndex extends MultiRegionTable {
       // ignore
       // ignore
     }
     }
 
 
-    Path localDir = new Path(this.testDir, "index_" +
+    Path localDir = new Path(getUnitTestdir(getName()), "index_" +
       Integer.toString(new Random().nextInt()));
       Integer.toString(new Random().nextInt()));
     this.fs.copyToLocalFile(new Path(INDEX_DIR), localDir);
     this.fs.copyToLocalFile(new Path(INDEX_DIR), localDir);
-    Path [] indexDirs = this.localFs.listPaths(new Path [] {localDir});
+    FileSystem localfs = FileSystem.getLocal(conf);
+    Path [] indexDirs = localfs.listPaths(new Path [] {localDir});
     Searcher searcher = null;
     Searcher searcher = null;
     HScannerInterface scanner = null;
     HScannerInterface scanner = null;
     try {
     try {

+ 11 - 10
src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java

@@ -27,7 +27,6 @@ import java.util.TreeMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -69,7 +68,6 @@ public class TestTableMapReduce extends MultiRegionTable {
   };
   };
 
 
   private MiniDFSCluster dfsCluster = null;
   private MiniDFSCluster dfsCluster = null;
-  private FileSystem fs;
   private Path dir;
   private Path dir;
   private MiniHBaseCluster hCluster = null;
   private MiniHBaseCluster hCluster = null;
   
   
@@ -116,16 +114,19 @@ public class TestTableMapReduce extends MultiRegionTable {
    */
    */
   @Override
   @Override
   public void setUp() throws Exception {
   public void setUp() throws Exception {
-    super.setUp();
     dfsCluster = new MiniDFSCluster(conf, 1, true, (String[])null);
     dfsCluster = new MiniDFSCluster(conf, 1, true, (String[])null);
+
+    // Must call super.setup() after starting mini dfs cluster. Otherwise
+    // we get a local file system instead of hdfs
+    
+    super.setUp();
     try {
     try {
-      fs = dfsCluster.getFileSystem();
       dir = new Path("/hbase");
       dir = new Path("/hbase");
       fs.mkdirs(dir);
       fs.mkdirs(dir);
       // Start up HBase cluster
       // Start up HBase cluster
       // Only one region server.  MultiRegionServer manufacturing code below
       // Only one region server.  MultiRegionServer manufacturing code below
       // depends on there being one region server only.
       // depends on there being one region server only.
-      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
+      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster, true);
       LOG.info("Master is at " + this.conf.get(HConstants.MASTER_ADDRESS));
       LOG.info("Master is at " + this.conf.get(HConstants.MASTER_ADDRESS));
     } catch (Exception e) {
     } catch (Exception e) {
       StaticTestEnvironment.shutdownDfs(dfsCluster);
       StaticTestEnvironment.shutdownDfs(dfsCluster);
@@ -252,13 +253,13 @@ public class TestTableMapReduce extends MultiRegionTable {
             IdentityTableReduce.class, jobConf);
             IdentityTableReduce.class, jobConf);
         LOG.info("Started " + SINGLE_REGION_TABLE_NAME);
         LOG.info("Started " + SINGLE_REGION_TABLE_NAME);
         JobClient.runJob(jobConf);
         JobClient.runJob(jobConf);
-        
+
         LOG.info("Print table contents after map/reduce for " +
         LOG.info("Print table contents after map/reduce for " +
           SINGLE_REGION_TABLE_NAME);
           SINGLE_REGION_TABLE_NAME);
-        scanTable(SINGLE_REGION_TABLE_NAME, true);
+      scanTable(SINGLE_REGION_TABLE_NAME, true);
 
 
-        // verify map-reduce results
-        verify(SINGLE_REGION_TABLE_NAME);
+      // verify map-reduce results
+      verify(SINGLE_REGION_TABLE_NAME);
       } finally {
       } finally {
         mrCluster.shutdown();
         mrCluster.shutdown();
       }
       }
@@ -306,7 +307,7 @@ public class TestTableMapReduce extends MultiRegionTable {
             IdentityTableReduce.class, jobConf);
             IdentityTableReduce.class, jobConf);
         LOG.info("Started " + MULTI_REGION_TABLE_NAME);
         LOG.info("Started " + MULTI_REGION_TABLE_NAME);
         JobClient.runJob(jobConf);
         JobClient.runJob(jobConf);
-        
+
         // verify map-reduce results
         // verify map-reduce results
         verify(MULTI_REGION_TABLE_NAME);
         verify(MULTI_REGION_TABLE_NAME);
       } finally {
       } finally {

+ 168 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/util/TestMigrate.java

@@ -0,0 +1,168 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+
+import java.util.zip.ZipInputStream;
+import java.util.zip.ZipEntry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+
+import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.StaticTestEnvironment;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ * 
+ */
+public class TestMigrate extends HBaseTestCase {
+  static final Log LOG = LogFactory.getLog(TestMigrate.class);
+
+  /**
+   * 
+   */
+  public TestMigrate() {
+    super();
+    Logger.getRootLogger().setLevel(Level.WARN);
+    Logger.getLogger(this.getClass().getPackage().getName()).
+      setLevel(Level.DEBUG);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+  }
+  
+  /**
+   * 
+   */
+  public void testUpgrade() {
+    MiniDFSCluster dfsCluster = null;
+    try {
+      dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
+      FileSystem dfs = dfsCluster.getFileSystem();
+      Path root = dfs.makeQualified(new Path(
+          conf.get(HConstants.HBASE_DIR, HConstants.DEFAULT_HBASE_DIR)));
+      dfs.mkdirs(root);
+
+      /*
+       * First load files from an old style HBase file structure
+       */
+      
+      // Current directory is .../workspace/project/build/contrib/hbase/test/data
+      
+      FileSystem localfs = FileSystem.getLocal(conf);
+      
+      // Get path for zip file
+
+      FSDataInputStream hs = localfs.open(new Path(Path.CUR_DIR,
+          
+          // this path is for running test with ant
+          
+          "../../../../../src/contrib/hbase/src/testdata/HADOOP-2478-testdata.zip")
+      
+          // and this path is for when you want to run inside eclipse
+      
+          /*"src/contrib/hbase/src/testdata/HADOOP-2478-testdata.zip")*/
+      );
+      
+      ZipInputStream zip = new ZipInputStream(hs);
+      
+      unzip(zip, dfs, root);
+      
+      zip.close();
+      hs.close();
+      
+      listPaths(dfs, root, root.toString().length() + 1);
+      
+      Migrate u = new Migrate(conf);
+      u.run((String[]) null);
+
+      listPaths(dfs, root, root.toString().length() + 1);
+      
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      if (dfsCluster != null) {
+        StaticTestEnvironment.shutdownDfs(dfsCluster);
+      }
+    }
+  }
+
+  private void unzip(ZipInputStream zip, FileSystem dfs, Path root)
+  throws IOException {
+
+    ZipEntry e = null;
+    while ((e = zip.getNextEntry()) != null)  {
+      if (e.isDirectory()) {
+        dfs.mkdirs(new Path(root, e.getName()));
+        
+      } else {
+        FSDataOutputStream out = dfs.create(new Path(root, e.getName()));
+        byte[] buffer = new byte[4096];
+        int len;
+        do {
+          len = zip.read(buffer);
+          if (len > 0) {
+            out.write(buffer, 0, len);
+          }
+        } while (len > 0);
+        out.close();
+      }
+      zip.closeEntry();
+    }
+  }
+  
+  private void listPaths(FileSystem fs, Path dir, int rootdirlength)
+  throws IOException {
+    FileStatus[] stats = fs.listStatus(dir);
+    if (stats == null || stats.length == 0) {
+      return;
+    }
+    for (int i = 0; i < stats.length; i++) {
+      String relativePath =
+        stats[i].getPath().toString().substring(rootdirlength);
+      if (stats[i].isDir()) {
+        System.out.println("d " + relativePath);
+        listPaths(fs, stats[i].getPath(), rootdirlength);
+      } else {
+        System.out.println("f " + relativePath + " size=" + stats[i].getLen());
+      }
+    }
+  }
+}