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

svn commit: r1460614 - in /hbase/branches/0.95/hbase-server/src: main/java/org/apache/hadoop/hbase/io/hfile/ main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/regionserver/wal/ m...

Author: mbertozzi
Date: Mon Mar 25 11:13:32 2013
New Revision: 1460614

URL: http://svn.apache.org/r1460614
Log:
HBASE-8006 use FSUtils to get/set hbase.rootdir

Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRepair.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java Mon Mar 25 11:13:32 2013
@@ -169,11 +169,8 @@ public class HFilePrettyPrinter {
    */
   public int run(String[] args) {
     conf = HBaseConfiguration.create();
-    conf.set("fs.defaultFS",
-        conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
-    conf.set("fs.default.name",
-        conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
     try {
+      FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
       if (!parseOptions(args))
         return 1;
     } catch (IOException ex) {

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Mon Mar 25 11:13:32 2013
@@ -115,9 +115,7 @@ public class MasterFileSystem {
     // Cover both bases, the old way of setting default fs and the new.
     // We're supposed to run on 0.20 and 0.21 anyways.
     this.fs = this.rootdir.getFileSystem(conf);
-    String fsUri = this.fs.getUri().toString();
-    conf.set("fs.default.name", fsUri);
-    conf.set("fs.defaultFS", fsUri);
+    FSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
     // make sure the fs has the same conf
     fs.setConf(conf);
     this.distributedLogSplitting =

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Mon Mar 25 11:13:32 2013
@@ -1103,10 +1103,10 @@ public class HRegionServer implements Cl
       // to match the filesystem on hbase.rootdir else underlying hadoop hdfs
       // accessors will be going against wrong filesystem (unless all is set
       // to defaults).
-      this.conf.set("fs.defaultFS", this.conf.get("hbase.rootdir"));
+      FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
       // Get fs instance used by this RS
       this.fs = new HFileSystem(this.conf, this.useHBaseChecksum);
-      this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
+      this.rootDir = FSUtils.getRootDir(this.conf);
       this.tableDescriptors = new FSTableDescriptors(this.fs, this.rootDir, true);
       this.hlog = setupWALAndReplication();
       // Init in here rather than in constructor after thread name has been set

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java Mon Mar 25 11:13:32 2013
@@ -1397,7 +1397,7 @@ class FSHLog implements HLog, Syncable {
     if (!fs.exists(p)) {
       throw new FileNotFoundException(p.toString());
     }
-    final Path baseDir = new Path(conf.get(HConstants.HBASE_DIR));
+    final Path baseDir = FSUtils.getRootDir(conf);
     final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
     if (!fs.getFileStatus(p).isDir()) {
       throw new IOException(p + " is not a directory");
@@ -1443,9 +1443,8 @@ class FSHLog implements HLog, Syncable {
       Configuration conf = HBaseConfiguration.create();
       for (int i = 1; i < args.length; i++) {
         try {
-          conf.set("fs.default.name", args[i]);
-          conf.set("fs.defaultFS", args[i]);
           Path logPath = new Path(args[i]);
+          FSUtils.setFsDefault(conf, logPath);
           split(conf, logPath);
         } catch (Throwable t) {
           t.printStackTrace(System.err);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogPrettyPrinter.java Mon Mar 25 11:13:32 2013
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HBaseConf
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 
 /**
@@ -360,10 +361,8 @@ public class HLogPrettyPrinter {
     }
     // get configuration, file system, and process the given files
     Configuration conf = HBaseConfiguration.create();
-    conf.set("fs.defaultFS",
-        conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
-    conf.set("fs.default.name",
-        conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
+    FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
+
     // begin output
     printer.beginPersistentOutput();
     for (Object f : files) {

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java Mon Mar 25 11:13:32 2013
@@ -570,7 +570,7 @@ public class HLogSplitter {
       final List<Path> corruptedLogs,
       final List<Path> processedLogs, final Path oldLogDir,
       final FileSystem fs, final Configuration conf) throws IOException {
-    final Path corruptDir = new Path(conf.get(HConstants.HBASE_DIR), conf.get(
+    final Path corruptDir = new Path(FSUtils.getRootDir(conf), conf.get(
         "hbase.regionserver.hlog.splitlog.corrupt.dir",  HConstants.CORRUPT_DIR_NAME));
 
     if (!fs.mkdirs(corruptDir)) {

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java Mon Mar 25 11:13:32 2013
@@ -760,6 +760,11 @@ public abstract class FSUtils {
     c.set(HConstants.HBASE_DIR, root.toString());
   }
 
+  public static void setFsDefault(final Configuration c, final Path root) throws IOException {
+    c.set("fs.defaultFS", root.toString());    // for hadoop 0.21+
+    c.set("fs.default.name", root.toString()); // for hadoop 0.20
+  }
+
   /**
    * Checks if root region exists
    *

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java Mon Mar 25 11:13:32 2013
@@ -1202,7 +1202,7 @@ public class HBaseFsck extends Configure
    */
   Path sidelineOldMeta() throws IOException {
     // put current .META. aside.
-    Path hbaseDir = new Path(getConf().get(HConstants.HBASE_DIR));
+    Path hbaseDir = FSUtils.getRootDir(getConf());
     FileSystem fs = hbaseDir.getFileSystem(getConf());
     Path backupDir = getSidelineDir();
     fs.mkdirs(backupDir);
@@ -1254,7 +1254,7 @@ public class HBaseFsck extends Configure
    * regionInfoMap
    */
   public void loadHdfsRegionDirs() throws IOException, InterruptedException {
-    Path rootDir = new Path(getConf().get(HConstants.HBASE_DIR));
+    Path rootDir = FSUtils.getRootDir(getConf());
     FileSystem fs = rootDir.getFileSystem(getConf());
 
     // list all tables from HDFS
@@ -1410,7 +1410,7 @@ public class HBaseFsck extends Configure
       return;
     }
 
-    Path hbaseDir = new Path(getConf().get(HConstants.HBASE_DIR));
+    Path hbaseDir = FSUtils.getRootDir(getConf());
     FileSystem fs = hbaseDir.getFileSystem(getConf());
     UserGroupInformation ugi = User.getCurrent().getUGI();
     FileStatus[] files = fs.listStatus(hbaseDir);
@@ -3494,10 +3494,9 @@ public class HBaseFsck extends Configure
   public static void main(String[] args) throws Exception {
     // create a fsck object
     Configuration conf = HBaseConfiguration.create();
-    Path hbasedir = new Path(conf.get(HConstants.HBASE_DIR));
+    Path hbasedir = FSUtils.getRootDir(conf);
     URI defaultFs = hbasedir.getFileSystem(conf).getUri();
-    conf.set("fs.defaultFS", defaultFs.toString());     // for hadoop 0.21+
-    conf.set("fs.default.name", defaultFs.toString());  // for hadoop 0.20
+    FSUtils.setFsDefault(conf, new Path(defaultFs));
 
     int ret = ToolRunner.run(new HBaseFsck(conf), args);
     System.exit(ret);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java Mon Mar 25 11:13:32 2013
@@ -398,7 +398,7 @@ public class RegionSplitter {
     final int MAX_OUTSTANDING =
         Math.max(table.getConnection().getCurrentNrHRS() / 2, minOS);
 
-    Path hbDir = new Path(conf.get(HConstants.HBASE_DIR));
+    Path hbDir = FSUtils.getRootDir(conf);
     Path tableDir = HTableDescriptor.getTableDir(hbDir, table.getTableName());
     Path splitFile = new Path(tableDir, "_balancedSplit");
     FileSystem fs = FileSystem.get(conf);
@@ -707,7 +707,7 @@ public class RegionSplitter {
 
   static LinkedList<Pair<byte[], byte[]>> getSplits(HTable table,
       SplitAlgorithm splitAlgo) throws IOException {
-    Path hbDir = new Path(table.getConfiguration().get(HConstants.HBASE_DIR));
+    Path hbDir = FSUtils.getRootDir(table.getConfiguration());
     Path tableDir = HTableDescriptor.getTableDir(hbDir, table.getTableName());
     Path splitFile = new Path(tableDir, "_balancedSplit");
     FileSystem fs = tableDir.getFileSystem(table.getConfiguration());

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java Mon Mar 25 11:13:32 2013
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.exceptions.CorruptHFileException;
 import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.FSUtils.FamilyDirFilter;
 import org.apache.hadoop.hbase.util.FSUtils.HFileFilter;
 import org.apache.hadoop.hbase.util.FSUtils.RegionDirFilter;
@@ -127,14 +128,14 @@ public class HFileCorruptionChecker {
    * @return path to where corrupted files are stored. This should be
    *         HBASE_DIR/.corrupt/table/region/cf/file.
    */
-  Path createQuarantinePath(Path hFile) {
+  Path createQuarantinePath(Path hFile) throws IOException {
     // extract the normal dirs structure
     Path cfDir = hFile.getParent();
     Path regionDir = cfDir.getParent();
     Path tableDir = regionDir.getParent();
 
     // build up the corrupted dirs strcture
-    Path corruptBaseDir = new Path(conf.get(HConstants.HBASE_DIR), conf.get(
+    Path corruptBaseDir = new Path(FSUtils.getRootDir(conf), conf.get(
         "hbase.hfile.quarantine.dir", HConstants.CORRUPT_DIR_NAME));
     Path corruptTableDir = new Path(corruptBaseDir, tableDir.getName());
     Path corruptRegionDir = new Path(corruptTableDir, regionDir.getName());

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRepair.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRepair.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRepair.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRepair.java Mon Mar 25 11:13:32 2013
@@ -24,8 +24,10 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.io.MultipleIOException;
 
@@ -70,8 +72,7 @@ public class OfflineMetaRepair {
     Configuration conf = HBaseConfiguration.create();
     // Cover both bases, the old way of setting default fs and the new.
     // We're supposed to run on 0.20 and 0.21 anyways.
-    conf.set("fs.defaultFS", conf.get(HConstants.HBASE_DIR));
-    conf.set("fs.default.name", conf.get(HConstants.HBASE_DIR));
+    FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
     HBaseFsck fsck = new HBaseFsck(conf);
     boolean fixHoles = false;
 
@@ -87,10 +88,8 @@ public class OfflineMetaRepair {
         }
         // update hbase root dir to user-specified base
         i++;
-        String path = args[i];
-        conf.set(HConstants.HBASE_DIR, path);
-        conf.set("fs.defaultFS", conf.get(HConstants.HBASE_DIR));
-        conf.set("fs.default.name", conf.get(HConstants.HBASE_DIR));
+        FSUtils.setRootDir(conf, new Path(args[i]));
+        FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
       } else if (cmd.equals("-sidelineDir")) {
         if (i == args.length - 1) {
           System.err.println("OfflineMetaRepair: -sidelineDir needs an HDFS path.");

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java Mon Mar 25 11:13:32 2013
@@ -111,8 +111,7 @@ public abstract class HBaseTestCase exte
           fs.delete(testDir, true);
         }
       } else {
-        this.testDir =
-          this.fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR)));
+        this.testDir = FSUtils.getRootDir(conf);
       }
     } catch (Exception e) {
       LOG.fatal("error during setup", e);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Mon Mar 25 11:13:32 2013
@@ -1,3 +1,5 @@
+
+
 /**
  *
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -482,9 +484,7 @@ public class HBaseTestingUtility extends
 
     // Set this just-started cluster as our filesystem.
     FileSystem fs = this.dfsCluster.getFileSystem();
-    this.conf.set("fs.defaultFS", fs.getUri().toString());
-    // Do old style too just to be safe.
-    this.conf.set("fs.default.name", fs.getUri().toString());
+    FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
 
     // Wait for the cluster to be totally up
     this.dfsCluster.waitClusterUp();
@@ -504,9 +504,7 @@ public class HBaseTestingUtility extends
 
     // Set this just-started cluster as our filesystem.
     FileSystem fs = this.dfsCluster.getFileSystem();
-    this.conf.set("fs.defaultFS", fs.getUri().toString());
-    // Do old style too just to be safe.
-    this.conf.set("fs.default.name", fs.getUri().toString());
+    FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
 
     // Wait for the cluster to be totally up
     this.dfsCluster.waitClusterUp();
@@ -598,8 +596,7 @@ public class HBaseTestingUtility extends
       this.dfsCluster.shutdown();
       dfsCluster = null;
       dataTestDirOnTestFS = null;
-      this.conf.set("fs.defaultFS", "file:///");
-      this.conf.set("fs.default.name", "file:///");
+      FSUtils.setFsDefault(this.conf, new Path("file:///"));
     }
   }
 
@@ -948,7 +945,7 @@ public class HBaseTestingUtility extends
   public Path createRootDir() throws IOException {
     FileSystem fs = FileSystem.get(this.conf);
     Path hbaseRootdir = getDefaultRootDirPath();
-    this.conf.set(HConstants.HBASE_DIR, hbaseRootdir.toString());
+    FSUtils.setRootDir(this.conf, hbaseRootdir);
     fs.mkdirs(hbaseRootdir);
     FSUtils.setVersion(fs, hbaseRootdir);
     return hbaseRootdir;

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java Mon Mar 25 11:13:32 2013
@@ -1527,10 +1527,10 @@ public class PerformanceEvaluation exten
       // mangle the conf so that the fs parameter points to the minidfs we
       // just started up
       FileSystem fs = dfsCluster.getFileSystem();
-      conf.set("fs.default.name", fs.getUri().toString());
+      FSUtils.setFsDefault(conf, new Path(fs.getUri()));
       conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.toString(zooKeeperPort));
       Path parentdir = fs.getHomeDirectory();
-      conf.set(HConstants.HBASE_DIR, parentdir.toString());
+      FSUtils.setRootDir(conf, parentdir);
       fs.mkdirs(parentdir);
       FSUtils.setVersion(fs, parentdir);
       hbaseMiniCluster = new MiniHBaseCluster(conf, N);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java Mon Mar 25 11:13:32 2013
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.junit.After;
@@ -92,7 +93,7 @@ public class TestWALObserver {
     Path hbaseRootDir = TEST_UTIL.getDFSCluster().getFileSystem()
         .makeQualified(new Path("/hbase"));
     LOG.info("hbase.rootdir=" + hbaseRootDir);
-    conf.set(HConstants.HBASE_DIR, hbaseRootDir.toString());
+    FSUtils.setRootDir(conf, hbaseRootDir);
   }
 
   @AfterClass
@@ -105,7 +106,7 @@ public class TestWALObserver {
     this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     // this.cluster = TEST_UTIL.getDFSCluster();
     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
-    this.hbaseRootDir = new Path(conf.get(HConstants.HBASE_DIR));
+    this.hbaseRootDir = FSUtils.getRootDir(conf);
     this.dir = new Path(this.hbaseRootDir, TestWALObserver.class.getName());
     this.oldLogDir = new Path(this.hbaseRootDir,
         HConstants.HREGION_OLDLOGDIR_NAME);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java Mon Mar 25 11:13:32 2013
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -252,9 +253,9 @@ public class TestBlockReorder {
     // make us fall in nasty hdfs bugs/issues. So we're going to try multiple times.
 
     // Now we need to find the log file, its locations, and look at it
-    String rootDir = FileSystem.get(conf).makeQualified(new Path(
-        conf.get(HConstants.HBASE_DIR) + "/" + HConstants.HREGION_LOGDIR_NAME +
-            "/" + targetRs.getServerName().toString())).toUri().getPath();
+
+    String rootDir = new Path(FSUtils.getRootDir(conf) + "/" + HConstants.HREGION_LOGDIR_NAME +
+            "/" + targetRs.getServerName().toString()).toUri().getPath();
 
     DistributedFileSystem mdfs = (DistributedFileSystem)
         hbm.getMaster().getMasterFileSystem().getFileSystem();

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Mon Mar 25 11:13:32 2013
@@ -111,8 +111,8 @@ public class TestCatalogJanitor {
           HRegionInfo.FIRST_META_REGIONINFO);
       // Set hbase.rootdir into test dir.
       FileSystem fs = FileSystem.get(this.c);
-      Path rootdir = fs.makeQualified(new Path(this.c.get(HConstants.HBASE_DIR)));
-      this.c.set(HConstants.HBASE_DIR, rootdir.toString());
+      Path rootdir = FSUtils.getRootDir(this.c);
+      FSUtils.setRootDir(this.c, rootdir);
       this.ct = Mockito.mock(CatalogTracker.class);
       AdminProtocol hri = Mockito.mock(AdminProtocol.class);
       Mockito.when(this.ct.getConnection()).thenReturn(this.connection);
@@ -751,8 +751,8 @@ public class TestCatalogJanitor {
     Path testdir = htu.getDataTestDir(subdir);
     FileSystem fs = FileSystem.get(htu.getConfiguration());
     if (fs.exists(testdir)) assertTrue(fs.delete(testdir, true));
-    htu.getConfiguration().set(HConstants.HBASE_DIR, testdir.toString());
-    return htu.getConfiguration().get(HConstants.HBASE_DIR);
+    FSUtils.setRootDir(htu.getConfiguration(), testdir);
+    return FSUtils.getRootDir(htu.getConfiguration()).toString();
   }
 
   /**

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java Mon Mar 25 11:13:32 2013
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@@ -191,8 +192,7 @@ public class TestMasterFailover {
     htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
 
     FileSystem filesystem = FileSystem.get(conf);
-    Path rootdir = filesystem.makeQualified(
-        new Path(conf.get(HConstants.HBASE_DIR)));
+    Path rootdir = FSUtils.getRootDir(conf);
     // Write the .tableinfo
     FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdEnabled);
 
@@ -498,8 +498,7 @@ public class TestMasterFailover {
     HTableDescriptor htdEnabled = new HTableDescriptor(enabledTable);
     htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
     FileSystem filesystem = FileSystem.get(conf);
-    Path rootdir = filesystem.makeQualified(
-           new Path(conf.get(HConstants.HBASE_DIR)));
+    Path rootdir = FSUtils.getRootDir(conf);
     // Write the .tableinfo
     FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdEnabled);
     HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(),

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java Mon Mar 25 11:13:32 2013
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.HC
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -80,7 +81,7 @@ public class TestMasterNoCluster {
   public static void setUpBeforeClass() throws Exception {
     Configuration c = TESTUTIL.getConfiguration();
     // We use local filesystem.  Set it so it writes into the testdir.
-    c.set(HConstants.HBASE_DIR, TESTUTIL.getDataTestDir().toString());
+    FSUtils.setRootDir(c, TESTUTIL.getDataTestDir());
     // Startup a mini zk cluster.
     TESTUTIL.startMiniZKCluster();
   }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java Mon Mar 25 11:13:32 2013
@@ -55,7 +55,7 @@ public class TestHFileLinkCleaner {
   @Test
   public void testHFileLinkCleaning() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDataTestDir().toString());
+    FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
     conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, HFileLinkCleaner.class.getName());
     Path rootDir = FSUtils.getRootDir(conf);
     FileSystem fs = FileSystem.get(conf);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Mon Mar 25 11:13:32 2013
@@ -716,21 +716,21 @@ public class TestHLog  {
 
   @Test
   public void testGetServerNameFromHLogDirectoryName() throws IOException {
-    String hl = conf.get(HConstants.HBASE_DIR) + "/"+
+    String hl = FSUtils.getRootDir(conf) + "/"+
         HLogUtil.getHLogDirectoryName(new ServerName("hn", 450, 1398).toString());
 
     // Must not throw exception
     Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, null));
     Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf,
-        conf.get(HConstants.HBASE_DIR) + "/"));
+        FSUtils.getRootDir(conf).toUri().toString()));
     Assert.assertNull( HLogUtil.getServerNameFromHLogDirectoryName(conf, "") );
     Assert.assertNull( HLogUtil.getServerNameFromHLogDirectoryName(conf, "                  ") );
     Assert.assertNull( HLogUtil.getServerNameFromHLogDirectoryName(conf, hl) );
     Assert.assertNull( HLogUtil.getServerNameFromHLogDirectoryName(conf, hl+"qdf") );
     Assert.assertNull( HLogUtil.getServerNameFromHLogDirectoryName(conf, "sfqf"+hl+"qdf") );
 
-    Assert.assertNotNull( HLogUtil.getServerNameFromHLogDirectoryName(conf, conf.get(
-        HConstants.HBASE_DIR) +
+    Assert.assertNotNull( HLogUtil.getServerNameFromHLogDirectoryName(conf,
+      FSUtils.getRootDir(conf).toUri().toString() +
         "/.logs/localhost,32984,1343316388997/localhost%2C32984%2C1343316388997.1343316390417"
         ));
     Assert.assertNotNull( HLogUtil.getServerNameFromHLogDirectoryName(conf, hl+"/qdf") );

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java Mon Mar 25 11:13:32 2013
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -134,7 +135,7 @@ public class TestHLogSplit {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setStrings("hbase.rootdir", HBASEDIR.toString());
+    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), HBASEDIR);
     TEST_UTIL.getConfiguration().setClass("hbase.regionserver.hlog.writer.impl",
       InstrumentedSequenceFileLogWriter.class, HLog.Writer.class);
     TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
@@ -1224,7 +1225,7 @@ public class TestHLogSplit {
     HLogSplitter.finishSplitLogFile(HBASEDIR, OLDLOGDIR, logfile.getPath()
         .toString(), conf);
 
-    final Path corruptDir = new Path(conf.get(HConstants.HBASE_DIR), conf.get(
+    final Path corruptDir = new Path(FSUtils.getRootDir(conf), conf.get(
         "hbase.regionserver.hlog.splitlog.corrupt.dir", ".corrupt"));
     assertEquals(1, fs.listStatus(corruptDir).length);
   }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Mon Mar 25 11:13:32 2013
@@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
@@ -107,7 +108,7 @@ public class TestWALReplay {
     Path hbaseRootDir =
       TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbase"));
     LOG.info("hbase.rootdir=" + hbaseRootDir);
-    conf.set(HConstants.HBASE_DIR, hbaseRootDir.toString());
+    FSUtils.setRootDir(conf, hbaseRootDir);
   }
 
   @AfterClass
@@ -119,7 +120,7 @@ public class TestWALReplay {
   public void setUp() throws Exception {
     this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
-    this.hbaseRootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
+    this.hbaseRootDir = FSUtils.getRootDir(this.conf);
     this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
     this.logName = HConstants.HREGION_LOGDIR_NAME;
     this.logDir = new Path(this.hbaseRootDir, logName);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java Mon Mar 25 11:13:32 2013
@@ -111,7 +111,7 @@ public class TestFlushSnapshotFromClient
     // Enable snapshot
     conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
     conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
-      ConstantSizeRegionSplitPolicy.class.getName());    
+      ConstantSizeRegionSplitPolicy.class.getName());
   }
 
   @Before
@@ -267,7 +267,7 @@ public class TestFlushSnapshotFromClient
     String snapshotName = "flushSnapshotCreateListDestroy";
     // test creating the snapshot
     admin.snapshot(snapshotName, STRING_TABLE_NAME, SnapshotDescription.Type.FLUSH);
-    logFSTree(new Path(UTIL.getConfiguration().get(HConstants.HBASE_DIR)));
+    logFSTree(FSUtils.getRootDir(UTIL.getConfiguration()));
 
     // make sure we only have 1 matching snapshot
     List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
@@ -391,14 +391,14 @@ public class TestFlushSnapshotFromClient
     }
 
     // dump for debugging
-    logFSTree(new Path(UTIL.getConfiguration().get(HConstants.HBASE_DIR)));
+    logFSTree(FSUtils.getRootDir(UTIL.getConfiguration()));
 
     List<SnapshotDescription> taken = admin.listSnapshots();
     int takenSize = taken.size();
     LOG.info("Taken " + takenSize + " snapshots:  " + taken);
     assertTrue("We expect at least 1 request to be rejected because of we concurrently" +
         " issued many requests", takenSize < ssNum && takenSize > 0);
-    // delete snapshots so subsequent tests are clean.  
+    // delete snapshots so subsequent tests are clean.
     for (SnapshotDescription ss : taken) {
       admin.deleteSnapshot(ss.getName());
     }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java Mon Mar 25 11:13:32 2013
@@ -274,7 +274,7 @@ public class TestHBaseFsck {
 
         if (regionInfoOnly) {
           LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString());
-          Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
+          Path rootDir = FSUtils.getRootDir(conf);
           FileSystem fs = rootDir.getFileSystem(conf);
           Path p = new Path(rootDir + "/" + htd.getNameAsString(), hri.getEncodedName());
           Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE);
@@ -283,7 +283,7 @@ public class TestHBaseFsck {
 
         if (hdfs) {
           LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
-          Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
+          Path rootDir = FSUtils.getRootDir(conf);
           FileSystem fs = rootDir.getFileSystem(conf);
           Path p = new Path(rootDir + "/" + htd.getNameAsString(), hri.getEncodedName());
           HBaseFsck.debugLsr(conf, p);
@@ -427,7 +427,8 @@ public class TestHBaseFsck {
       setupTable(table);
       HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
 
-      Path hbaseTableDir = new Path(conf.get(HConstants.HBASE_DIR) + "/" + table );
+      Path hbaseTableDir = HTableDescriptor.getTableDir(
+        FSUtils.getRootDir(conf), Bytes.toBytes(table));
       fs = hbaseTableDir.getFileSystem(conf);
       FileStatus status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       tableinfo = status.getPath();
@@ -1068,7 +1069,7 @@ public class TestHBaseFsck {
   @Test
   public void testNoVersionFile() throws Exception {
     // delete the hbase.version file
-    Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
+    Path rootDir = FSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
     Path versionFile = new Path(rootDir, HConstants.VERSION_FILE_NAME);
     fs.delete(versionFile, true);
@@ -1098,8 +1099,7 @@ public class TestHBaseFsck {
       ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
 
       FileSystem filesystem = FileSystem.get(conf);
-      Path rootdir = filesystem.makeQualified(new Path(conf
-          .get(HConstants.HBASE_DIR)));
+      Path rootdir = FSUtils.getRootDir(conf);
 
       byte[][] SPLIT_KEYS = new byte[][] { new byte[0], Bytes.toBytes("aaa"),
           Bytes.toBytes("bbb"), Bytes.toBytes("ccc"), Bytes.toBytes("ddd") };

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java Mon Mar 25 11:13:32 2013
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.experimental.categories.Category;
@@ -125,9 +126,9 @@ public class TestMergeTool extends HBase
     this.dfsCluster = TEST_UTIL.startMiniDFSCluster(2);
     this.fs = this.dfsCluster.getFileSystem();
     System.out.println("fs=" + this.fs);
-    this.conf.set("fs.defaultFS", fs.getUri().toString());
+    FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
     Path parentdir = fs.getHomeDirectory();
-    conf.set(HConstants.HBASE_DIR, parentdir.toString());
+    FSUtils.setRootDir(conf, parentdir);
     fs.mkdirs(parentdir);
     FSUtils.setVersion(fs, parentdir);
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java?rev=1460614&r1=1460613&r2=1460614&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java Mon Mar 25 11:13:32 2013
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.Re
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
@@ -181,7 +182,7 @@ public class OfflineMetaRebuildTestCore 
         TEST_UTIL.getHBaseAdmin().unassign(deleteRow, true);
 
         LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
-        Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
+        Path rootDir = FSUtils.getRootDir(conf);
         FileSystem fs = rootDir.getFileSystem(conf);
         Path p = new Path(rootDir + "/" + htd.getNameAsString(),
             hri.getEncodedName());
@@ -207,7 +208,7 @@ public class OfflineMetaRebuildTestCore 
     HRegionInfo hri = new HRegionInfo(htbl.getTableName(), startKey, endKey);
 
     LOG.info("manually adding regioninfo and hdfs data: " + hri.toString());
-    Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
+    Path rootDir = FSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
     Path p = new Path(rootDir + "/" + htd.getNameAsString(),
         hri.getEncodedName());