You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/10/26 22:29:44 UTC

svn commit: r1189429 [1/2] - in /hbase/trunk: ./ src/test/java/org/apache/hadoop/hbase/ src/test/java/org/apache/hadoop/hbase/catalog/ src/test/java/org/apache/hadoop/hbase/client/ src/test/java/org/apache/hadoop/hbase/client/replication/ src/test/java...

Author: stack
Date: Wed Oct 26 20:29:41 2011
New Revision: 1189429

URL: http://svn.apache.org/viewvc?rev=1189429&view=rev
Log:
HBASE-4634 'test.build.data' property overused leading to write data at the wrong place

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLogsCleaner.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogBench.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationPeer.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Wed Oct 26 20:29:41 2011
@@ -405,6 +405,8 @@ Release 0.92.0 - Unreleased
    HBASE-4367  Deadlock in MemStore flusher due to JDK internally synchronizing
                on current thread
    HBASE-4645  Edits Log recovery losing data across column families
+   HBASE-4634  "test.build.data" property overused leading to write data at the
+               wrong place (nkeywal)
 
   TESTS
    HBASE-4450  test for number of blocks read: to serve as baseline for expected

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java Wed Oct 26 20:29:41 2011
@@ -54,8 +54,10 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 public abstract class HBaseTestCase extends TestCase {
   private static final Log LOG = LogFactory.getLog(HBaseTestCase.class);
 
-  /** configuration parameter name for test directory */
-  public static final String TEST_DIRECTORY_KEY = "test.build.data";
+  /** configuration parameter name for test directory
+   * @deprecated see HBaseTestingUtility#TEST_DIRECTORY_KEY
+   **/
+  private static final String TEST_DIRECTORY_KEY = "test.build.data";
 
 /*
   protected final static byte [] fam1 = Bytes.toBytes("colfamily1");
@@ -153,20 +155,27 @@ public abstract class HBaseTestCase exte
     super.tearDown();
   }
 
-  protected Path getUnitTestdir(String testName) {
-    return new Path(
-        conf.get(TEST_DIRECTORY_KEY, "target/test/data"), testName);
-  }
+  /**
+   * @see HBaseTestingUtility#getBaseTestDir
+   * @param testName
+   * @return directory to use for this test
+   */
+    protected Path getUnitTestdir(String testName) {
+      return new Path(
+          System.getProperty(
+            HBaseTestingUtility.BASE_TEST_DIRECTORY_KEY,
+            HBaseTestingUtility.DEFAULT_BASE_TEST_DIRECTORY
+            ),
+        testName
+      );
+    }
 
   protected HRegion createNewHRegion(HTableDescriptor desc, byte [] startKey,
       byte [] endKey)
   throws IOException {
     FileSystem filesystem = FileSystem.get(conf);
-    Path rootdir = filesystem.makeQualified(
-        new Path(conf.get(HConstants.HBASE_DIR)));
-    filesystem.mkdirs(rootdir);
     HRegionInfo hri = new HRegionInfo(desc.getName(), startKey, endKey);
-    return HRegion.createHRegion(hri, rootdir, conf, desc);
+    return HRegion.createHRegion(hri, testDir, conf, desc);
   }
 
   protected HRegion openClosedRegion(final HRegion closedRegion)

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Wed Oct 26 20:29:41 2011
@@ -72,9 +72,6 @@ import org.apache.hadoop.hbase.zookeeper
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
@@ -82,15 +79,16 @@ import org.apache.zookeeper.KeeperExcept
 
 /**
  * Facility for testing HBase. Replacement for
- * old HBaseTestCase and HBaseCluserTestCase functionality.
+ * old HBaseTestCase and HBaseClusterTestCase functionality.
  * Create an instance and keep it around testing HBase.  This class is
  * meant to be your one-stop shop for anything you might need testing.  Manages
- * one cluster at a time only.  Depends on log4j being on classpath and
+ * one cluster at a time only.
+ * Depends on log4j being on classpath and
  * hbase-site.xml for logging and test-run configuration.  It does not set
  * logging levels nor make changes to configuration parameters.
  */
 public class HBaseTestingUtility {
-  private final static Log LOG = LogFactory.getLog(HBaseTestingUtility.class);
+  private static final Log LOG = LogFactory.getLog(HBaseTestingUtility.class);
   private Configuration conf;
   private MiniZooKeeperCluster zkCluster = null;
   /**
@@ -102,19 +100,33 @@ public class HBaseTestingUtility {
 
   private MiniHBaseCluster hbaseCluster = null;
   private MiniMRCluster mrCluster = null;
-  // If non-null, then already a cluster running.
-  private File clusterTestBuildDir = null;
+
+  // Directory where we put the data for this instance of HBaseTestingUtility
+  private File dataTestDir = null;
+
+  // Directory (usually a subdirectory of dataTestDir) used by the dfs cluster
+  //  if any
+  private File clusterTestDir = null;
 
   /**
    * System property key to get test directory value.
    * Name is as it is because mini dfs has hard-codings to put test data here.
+   * It should NOT be used directly in HBase, as it's a property used in
+   *  mini dfs.
+   *  @deprecated can be used only with mini dfs
+   */
+  private static final String TEST_DIRECTORY_KEY = "test.build.data";
+
+  /**
+   * System property key to get base test directory value
    */
-  public static final String TEST_DIRECTORY_KEY = "test.build.data";
+  public static final String BASE_TEST_DIRECTORY_KEY =
+    "test.build.data.basedirectory";
 
   /**
-   * Default parent directory for test output.
+   * Default base directory for test output.
    */
-  public static final String DEFAULT_TEST_DIRECTORY = "target/test-data";
+  public static final String DEFAULT_BASE_TEST_DIRECTORY = "target/test-data";
 
   /** Compression algorithms to use in parameterized JUnit 4 tests */
   public static final List<Object[]> COMPRESSION_ALGORITHMS_PARAMETERIZED =
@@ -124,10 +136,9 @@ public class HBaseTestingUtility {
     });
 
   /** Compression algorithms to use in testing */
-  public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS =
-      new Compression.Algorithm[] {
-        Compression.Algorithm.NONE, Compression.Algorithm.GZ
-      };
+  public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS ={
+      Compression.Algorithm.NONE, Compression.Algorithm.GZ
+    };
 
   /**
    * Create all combinations of Bloom filters and compression algorithms for
@@ -155,10 +166,6 @@ public class HBaseTestingUtility {
     this.conf = conf;
   }
 
-  public MiniHBaseCluster getHbaseCluster() {
-    return hbaseCluster;
-  }
-
   /**
    * Returns this classes's instance of {@link Configuration}.  Be careful how
    * you use the returned Configuration since {@link HConnection} instances
@@ -175,69 +182,109 @@ public class HBaseTestingUtility {
   }
 
   /**
-   * Makes sure the test directory is set up so that {@link #getTestDir()}
-   * returns a valid directory. Useful in unit tests that do not run a
-   * mini-cluster.
-   */
-  public void initTestDir() {
-    if (System.getProperty(TEST_DIRECTORY_KEY) == null) {
-      clusterTestBuildDir = setupClusterTestBuildDir();
-      System.setProperty(TEST_DIRECTORY_KEY, clusterTestBuildDir.getPath());
+   * @return Where to write test data on local filesystem; usually
+   * {@link #DEFAULT_BASE_TEST_DIRECTORY}
+   * Should not be used by the unit tests, hence its's private.
+   * Unit test will use a subdirectory of this directory.
+   * @see #setupDataTestDir()
+   * @see #getTestFileSystem()
+   */
+  private Path getBaseTestDir() {
+    String PathName = System.getProperty(
+      BASE_TEST_DIRECTORY_KEY, DEFAULT_BASE_TEST_DIRECTORY);
+
+    return new Path(PathName);
+  }
+
+  /**
+   * @return Where to write test data on local filesystem, specific to
+   *  the test.  Useful for tests that do not use a cluster.
+   * Creates it if it does not exist already.
+   * @see #getTestFileSystem()
+   */
+  public Path getDataTestDir() {
+    if (dataTestDir == null){
+      setupDataTestDir();
     }
+    return new Path(dataTestDir.getAbsolutePath());
   }
 
   /**
-   * @return Where to write test data on local filesystem; usually
-   * {@link #DEFAULT_TEST_DIRECTORY}
-   * @see #setupClusterTestBuildDir()
-   * @see #clusterTestBuildDir()
+   * @return Where the DFS cluster will write data on the local subsystem.
+   * Creates it if it does not exist already.
    * @see #getTestFileSystem()
    */
-  public static Path getTestDir() {
-    return new Path(System.getProperty(TEST_DIRECTORY_KEY,
-      DEFAULT_TEST_DIRECTORY));
+  public Path getClusterTestDir() {
+    if (clusterTestDir == null){
+      setupClusterTestDir();
+    }
+    return new Path(clusterTestDir.getAbsolutePath());
   }
 
   /**
    * @param subdirName
    * @return Path to a subdirectory named <code>subdirName</code> under
-   * {@link #getTestDir()}.
-   * @see #setupClusterTestBuildDir()
-   * @see #clusterTestBuildDir(String)
-   * @see #getTestFileSystem()
+   * {@link #getDataTestDir()}.
+   * Does *NOT* create it if it does not exist.
    */
-  public static Path getTestDir(final String subdirName) {
-    return new Path(getTestDir(), subdirName);
+  public Path getDataTestDir(final String subdirName) {
+    return new Path(getDataTestDir(), subdirName);
   }
 
   /**
-   * Home our cluster in a dir under {@link #DEFAULT_TEST_DIRECTORY}.  Give it a
-   * random name
-   * so can have many concurrent clusters running if we need to.  Need to
-   * amend the {@link #TEST_DIRECTORY_KEY} System property.  Its what
-   * minidfscluster bases
+   * Home our data in a dir under {@link #DEFAULT_BASE_TEST_DIRECTORY}.
+   * Give it a random name so can have many concurrent tests running if
+   * we need to.  It needs to amend the {@link #TEST_DIRECTORY_KEY}
+   * System property, as it's what minidfscluster bases
    * it data dir on.  Moding a System property is not the way to do concurrent
    * instances -- another instance could grab the temporary
    * value unintentionally -- but not anything can do about it at moment;
    * single instance only is how the minidfscluster works.
-   * @return The calculated cluster test build directory.
+   * @return The calculated data test build directory.
    */
-  public File setupClusterTestBuildDir() {
+  private void setupDataTestDir() {
+    if (dataTestDir != null) {
+      LOG.warn("Data test dir already setup in " +
+        dataTestDir.getAbsolutePath());
+      return;
+    }
+
     String randomStr = UUID.randomUUID().toString();
-    String dirStr = getTestDir(randomStr).toString();
-    File dir = new File(dirStr).getAbsoluteFile();
+    Path testDir= new Path(
+      getBaseTestDir(),
+      randomStr
+    );
+
+    dataTestDir = new File(testDir.toString()).getAbsoluteFile();
+    // Have it cleaned up on exit
+    dataTestDir.deleteOnExit();
+  }
+
+  /**
+   * Creates a directory for the DFS cluster, under the test data
+   */
+  private void setupClusterTestDir() {
+    if (clusterTestDir != null) {
+      LOG.warn("Cluster test dir already setup in " +
+        clusterTestDir.getAbsolutePath());
+      return;
+    }
+
+    // Using randomUUID ensures that multiple clusters can be launched by
+    //  a same test, if it stops & starts them
+    Path testDir = getDataTestDir("dfscluster_" + UUID.randomUUID().toString());
+    clusterTestDir = new File(testDir.toString()).getAbsoluteFile();
     // Have it cleaned up on exit
-    dir.deleteOnExit();
-    return dir;
+    clusterTestDir.deleteOnExit();
   }
 
   /**
    * @throws IOException If a cluster -- zk, dfs, or hbase -- already running.
    */
-  void isRunningCluster(String passedBuildPath) throws IOException {
-    if (this.clusterTestBuildDir == null || passedBuildPath != null) return;
+  public void isRunningCluster() throws IOException {
+    if (dfsCluster == null) return;
     throw new IOException("Cluster already running at " +
-      this.clusterTestBuildDir);
+      this.clusterTestDir);
   }
 
   /**
@@ -248,7 +295,7 @@ public class HBaseTestingUtility {
    * @return The mini dfs cluster created.
    */
   public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception {
-    return startMiniDFSCluster(servers, null, null);
+    return startMiniDFSCluster(servers, null);
   }
 
   /**
@@ -265,62 +312,58 @@ public class HBaseTestingUtility {
   public MiniDFSCluster startMiniDFSCluster(final String hosts[])
     throws Exception {
     if ( hosts != null && hosts.length != 0) {
-      return startMiniDFSCluster(hosts.length, null, hosts);
+      return startMiniDFSCluster(hosts.length, hosts);
     } else {
-      return startMiniDFSCluster(1, null, null);
+      return startMiniDFSCluster(1, null);
     }
   }
 
   /**
    * Start a minidfscluster.
    * Can only create one.
-   * @param dir Where to home your dfs cluster.
-   * @param servers How many DNs to start.
-   * @throws Exception
-   * @see {@link #shutdownMiniDFSCluster()}
-   * @return The mini dfs cluster created.
-   */
-  public MiniDFSCluster startMiniDFSCluster(int servers, final File dir)
-  throws Exception {
-    return startMiniDFSCluster(servers, dir, null);
-  }
-
-  
-  /**
-   * Start a minidfscluster.
-   * Can only create one.
    * @param servers How many DNs to start.
-   * @param dir Where to home your dfs cluster.
    * @param hosts hostnames DNs to run on.
    * @throws Exception
    * @see {@link #shutdownMiniDFSCluster()}
    * @return The mini dfs cluster created.
    */
-  public MiniDFSCluster startMiniDFSCluster(int servers, final File dir, final String hosts[])
+  public MiniDFSCluster startMiniDFSCluster(int servers, final String hosts[])
   throws Exception {
-    // This does the following to home the minidfscluster
-    //     base_dir = new File(System.getProperty("test.build.data", "build/test/data"), "dfs/");
+
+    // Check that there is not already a cluster running
+    isRunningCluster();
+
+    // Initialize the local directory used by the MiniDFS
+    if (clusterTestDir == null) {
+      setupClusterTestDir();
+    }
+
+    // We have to set this property as it is used by MiniCluster
+    System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestDir.toString());
+
     // Some tests also do this:
     //  System.getProperty("test.cache.data", "build/test/cache");
-    if (dir == null) {
-      this.clusterTestBuildDir = setupClusterTestBuildDir();
-    } else {
-      this.clusterTestBuildDir = dir;
-    }
-    System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestBuildDir.toString());
-    System.setProperty("test.cache.data", this.clusterTestBuildDir.toString());
+    // It's also deprecated
+    System.setProperty("test.cache.data", this.clusterTestDir.toString());
+
+    // Ok, now we can start
     this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
       true, null, null, hosts, null);
-    // Set this just-started cluser as our filesystem.
+
+    // 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());
+
+    // Wait for the cluster to be totally up
+    this.dfsCluster.waitClusterUp();
+
     return this.dfsCluster;
   }
 
   /**
-   * Shuts down instance created by call to {@link #startMiniDFSCluster(int, File)}
+   * Shuts down instance created by call to {@link #startMiniDFSCluster(int)}
    * or does nothing.
    * @throws Exception
    */
@@ -328,7 +371,9 @@ public class HBaseTestingUtility {
     if (this.dfsCluster != null) {
       // The below throws an exception per dn, AsynchronousCloseException.
       this.dfsCluster.shutdown();
+      dfsCluster = null;
     }
+
   }
 
   /**
@@ -339,8 +384,7 @@ public class HBaseTestingUtility {
    * @return zk cluster started.
    */
   public MiniZooKeeperCluster startMiniZKCluster() throws Exception {
-    return startMiniZKCluster(setupClusterTestBuildDir(),1);
-
+    return startMiniZKCluster(1);
   }
   
   /**
@@ -353,7 +397,8 @@ public class HBaseTestingUtility {
    */
   public MiniZooKeeperCluster startMiniZKCluster(int zooKeeperServerNum) 
       throws Exception {
-    return startMiniZKCluster(setupClusterTestBuildDir(), zooKeeperServerNum);
+    File zkClusterFile = new File(getClusterTestDir().toString());
+    return startMiniZKCluster(zkClusterFile, zooKeeperServerNum);
 
   }
   
@@ -365,12 +410,12 @@ public class HBaseTestingUtility {
   private MiniZooKeeperCluster startMiniZKCluster(final File dir, 
       int zooKeeperServerNum)
   throws Exception {
-    this.passedZkCluster = false;
     if (this.zkCluster != null) {
       throw new IOException("Cluster already running at " + dir);
     }
+    this.passedZkCluster = false;
     this.zkCluster = new MiniZooKeeperCluster();
-    int clientPort = this.zkCluster.startup(dir,zooKeeperServerNum);
+    int clientPort =   this.zkCluster.startup(dir,zooKeeperServerNum);
     this.conf.set("hbase.zookeeper.property.clientPort",
       Integer.toString(clientPort));
     return this.zkCluster;
@@ -465,26 +510,20 @@ public class HBaseTestingUtility {
     
     LOG.info("Starting up minicluster with " + numMasters + " master(s) and " +
         numSlaves + " regionserver(s) and " + numDataNodes + " datanode(s)");
+
     // If we already put up a cluster, fail.
-    String testBuildPath = conf.get(TEST_DIRECTORY_KEY, null);
-    isRunningCluster(testBuildPath);
-    if (testBuildPath != null) {
-      LOG.info("Using passed path: " + testBuildPath);
-    }
-    // Make a new random dir to home everything in.  Set it as system property.
-    // minidfs reads home from system property.
-    this.clusterTestBuildDir = testBuildPath == null?
-      setupClusterTestBuildDir() : new File(testBuildPath);
-    System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestBuildDir.getPath());
+    isRunningCluster();
+
     // Bring up mini dfs cluster. This spews a bunch of warnings about missing
     // scheme. Complaints are 'Scheme is undefined for build/test/data/dfs/name1'.
-    startMiniDFSCluster(numDataNodes, this.clusterTestBuildDir, dataNodeHosts);
-    this.dfsCluster.waitClusterUp();
+    startMiniDFSCluster(numDataNodes, dataNodeHosts);
 
     // Start up a zk cluster.
     if (this.zkCluster == null) {
-      startMiniZKCluster(this.clusterTestBuildDir);
+      startMiniZKCluster(clusterTestDir);
     }
+
+    // Start the MiniHBaseCluster
     return startMiniHBaseCluster(numMasters, numSlaves);
   }
 
@@ -528,7 +567,7 @@ public class HBaseTestingUtility {
     HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
     ResultScanner s = t.getScanner(new Scan());
     while (s.next() != null) {
-      continue;
+      // do nothing
     }
     LOG.info("HBase has been restarted");
   }
@@ -547,22 +586,22 @@ public class HBaseTestingUtility {
    * @throws IOException
    * @see {@link #startMiniCluster(int)}
    */
-  public void shutdownMiniCluster() throws IOException {
+  public void shutdownMiniCluster() throws Exception {
     LOG.info("Shutting down minicluster");
     shutdownMiniHBaseCluster();
-    if (!this.passedZkCluster) shutdownMiniZKCluster();
-    if (this.dfsCluster != null) {
-      // The below throws an exception per dn, AsynchronousCloseException.
-      this.dfsCluster.shutdown();
+    if (!this.passedZkCluster){
+      shutdownMiniZKCluster();
     }
+    shutdownMiniDFSCluster();
+
     // Clean up our directory.
-    if (this.clusterTestBuildDir != null && this.clusterTestBuildDir.exists()) {
+    if (this.clusterTestDir != null && this.clusterTestDir.exists()) {
       // Need to use deleteDirectory because File.delete required dir is empty.
       if (!FSUtils.deleteDirectory(FileSystem.getLocal(this.conf),
-          new Path(this.clusterTestBuildDir.toString()))) {
-        LOG.warn("Failed delete of " + this.clusterTestBuildDir.toString());
+          new Path(this.clusterTestDir.toString()))) {
+        LOG.warn("Failed delete of " + this.clusterTestDir.toString());
       }
-      this.clusterTestBuildDir = null;
+      this.clusterTestDir = null;
     }
     LOG.info("Minicluster is down");
   }
@@ -576,8 +615,8 @@ public class HBaseTestingUtility {
       this.hbaseCluster.shutdown();
       // Wait till hbase is down before going on to shutdown zk.
       this.hbaseCluster.join();
+      this.hbaseCluster = null;
     }
-    this.hbaseCluster = null;
   }
 
   /**
@@ -931,8 +970,8 @@ public class HBaseTestingUtility {
    * Creates the specified number of regions in the specified table.
    * @param c
    * @param table
-   * @param columnFamily
-   * @param startKeys
+   * @param family
+   * @param numRegions
    * @return
    * @throws IOException
    */
@@ -1107,7 +1146,6 @@ public class HBaseTestingUtility {
     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
     // add custom ones
-    int count = 0;
     for (int i = 0; i < startKeys.length; i++) {
       int j = (i + 1) % startKeys.length;
       HRegionInfo hri = new HRegionInfo(htd.getName(), startKeys[i],
@@ -1118,7 +1156,6 @@ public class HBaseTestingUtility {
       meta.put(put);
       LOG.info("createMultiRegionsInMeta: inserted " + hri.toString());
       newRegions.add(hri);
-      count++;
     }
     return newRegions;
   }
@@ -1221,6 +1258,7 @@ public class HBaseTestingUtility {
     LOG.info("Stopping mini mapreduce cluster...");
     if (mrCluster != null) {
       mrCluster.shutdown();
+      mrCluster = null;
     }
     // Restore configuration to point to local jobtracker
     conf.set("mapred.job.tracker", "local");
@@ -1382,7 +1420,13 @@ public class HBaseTestingUtility {
    * @throws IOException
    */
   public boolean cleanupTestDir() throws IOException {
-    return deleteDir(getTestDir());
+    if (dataTestDir == null ){
+      return false;
+    } else {
+      boolean ret = deleteDir(getDataTestDir());
+      dataTestDir = null;
+      return ret;
+    }
   }
 
   /**
@@ -1391,7 +1435,10 @@ public class HBaseTestingUtility {
    * @throws IOException
    */
   public boolean cleanupTestDir(final String subdir) throws IOException {
-    return deleteDir(getTestDir(subdir));
+    if (dataTestDir == null){
+      return false;
+    }
+    return deleteDir(getDataTestDir(subdir));
   }
 
   /**
@@ -1402,7 +1449,7 @@ public class HBaseTestingUtility {
   public boolean deleteDir(final Path dir) throws IOException {
     FileSystem fs = getTestFileSystem();
     if (fs.exists(dir)) {
-      return fs.delete(getTestDir(), true);
+      return fs.delete(getDataTestDir(), true);
     }
     return false;
   }
@@ -1436,6 +1483,9 @@ public class HBaseTestingUtility {
     return false;
   }
 
+
+
+
   /**
    * This method clones the passed <code>c</code> configuration setting a new
    * user into the clone.  Use it getting new instances of FileSystem.  Only
@@ -1500,7 +1550,6 @@ public class HBaseTestingUtility {
    * Wait until <code>countOfRegion</code> in .META. have a non-empty
    * info:server.  This means all regions have been deployed, master has been
    * informed and updated .META. with the regions deployed server.
-   * @param conf Configuration
    * @param countOfRegions How many regions in .META.
    * @throws IOException
    */
@@ -1572,7 +1621,7 @@ public class HBaseTestingUtility {
    * Creates an znode with OPENED state.
    * @param TEST_UTIL
    * @param region
-   * @param regionServer
+   * @param serverName
    * @return
    * @throws IOException
    * @throws ZooKeeperConnectionException
@@ -1639,5 +1688,4 @@ public class HBaseTestingUtility {
       return "<out_of_range>";
     }
   }
-
 }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java Wed Oct 26 20:29:41 2011
@@ -28,32 +28,21 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.*;
 
 public class TestFSTableDescriptorForceCreation {
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
-  @BeforeClass
-  public static void setUpCluster() throws Exception {
-    UTIL.startMiniDFSCluster(1);
-  }
-
-  @AfterClass
-  public static void shutDownCluster() throws Exception {
-    UTIL.shutdownMiniDFSCluster();
-  }
-
   @Test
   public void testShouldCreateNewTableDescriptorIfForcefulCreationIsFalse()
       throws IOException {
     final String name = "newTable2";
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
-    Path rootdir = new Path(fs.getWorkingDirectory(), name);
+    Path rootdir = new Path(UTIL.getDataTestDir(), name);
     HTableDescriptor htd = new HTableDescriptor(name);
-    assertTrue("Should create new table descriptor", FSUtils
-        .createTableDescriptor(fs, rootdir, htd, false));
+
+    assertTrue("Should create new table descriptor",
+      FSUtils.createTableDescriptor(fs, rootdir, htd, false));
   }
 
   @Test
@@ -62,12 +51,12 @@ public class TestFSTableDescriptorForceC
     final String name = "testAlreadyExists";
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any detrius laying around.
-    Path rootdir = new Path(fs.getWorkingDirectory(), name);
+    Path rootdir = new Path(UTIL.getDataTestDir(), name);
     TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
     HTableDescriptor htd = new HTableDescriptor(name);
     htds.add(htd);
     assertFalse("Should not create new table descriptor", FSUtils
-        .createTableDescriptor(fs, rootdir, htd, false));
+      .createTableDescriptor(fs, rootdir, htd, false));
   }
 
   @Test
@@ -75,10 +64,10 @@ public class TestFSTableDescriptorForceC
       throws Exception {
     final String name = "createNewTableNew2";
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
-    Path rootdir = new Path(fs.getWorkingDirectory(), name);
+    Path rootdir = new Path(UTIL.getDataTestDir(), name);
     HTableDescriptor htd = new HTableDescriptor(name);
     FSUtils.createTableDescriptor(fs, rootdir, htd, false);
     assertTrue("Should create new table descriptor", FSUtils
-        .createTableDescriptor(fs, rootdir, htd, true));
+      .createTableDescriptor(fs, rootdir, htd, true));
   }
 }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java Wed Oct 26 20:29:41 2011
@@ -50,26 +50,6 @@ import org.junit.Test;
 public class TestHBaseTestingUtility {
   private final Log LOG = LogFactory.getLog(this.getClass());
 
-  private HBaseTestingUtility hbt;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    this.hbt = new HBaseTestingUtility();
-    this.hbt.cleanupTestDir();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
   /**
    * Basic sanity test that spins up multiple HDFS and HBase clusters that share
    * the same ZK ensemble. We then create the same table in both and make sure
@@ -136,57 +116,80 @@ public class TestHBaseTestingUtility {
   }
 
   @Test public void testMiniCluster() throws Exception {
-    MiniHBaseCluster cluster = this.hbt.startMiniCluster();
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+
+    MiniHBaseCluster cluster = hbt.startMiniCluster();
     try {
       assertEquals(1, cluster.getLiveRegionServerThreads().size());
     } finally {
-      cluster.shutdown();
+      hbt.shutdownMiniCluster();
     }
   }
-  
+
+  /**
+   *  Test that we can start and stop multiple time a cluster
+   *   with the same HBaseTestingUtility.
+   */
+  @Test public void testMultipleStartStop() throws Exception{
+    HBaseTestingUtility htu1 = new HBaseTestingUtility();
+    Path foo = new Path("foo");
+
+    htu1.startMiniCluster();
+    htu1.getDFSCluster().getFileSystem().create(foo);
+    assertTrue( htu1.getDFSCluster().getFileSystem().exists(foo));
+    htu1.shutdownMiniCluster();
+
+    htu1.startMiniCluster();
+    assertFalse( htu1.getDFSCluster().getFileSystem().exists(foo));
+    htu1.getDFSCluster().getFileSystem().create(foo);
+    assertTrue( htu1.getDFSCluster().getFileSystem().exists(foo));
+    htu1.shutdownMiniCluster();
+  }
+
+
   @Test public void testMiniZooKeeper() throws Exception {
-    MiniZooKeeperCluster cluster1 = this.hbt.startMiniZKCluster();
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+    MiniZooKeeperCluster cluster1 = hbt.startMiniZKCluster();
     try {
-      assertEquals(0, cluster1.getBackupZooKeeperServerNum());    
+      assertEquals(0, cluster1.getBackupZooKeeperServerNum());
       assertTrue((cluster1.killCurrentActiveZooKeeperServer() == -1));
     } finally {
-      cluster1.shutdown();
+      hbt.shutdownMiniZKCluster();
     }
-    
-    this.hbt.shutdownMiniZKCluster();
-    
+
     // set up zookeeper cluster with 5 zk servers
-    MiniZooKeeperCluster cluster2 = this.hbt.startMiniZKCluster(5);
+    MiniZooKeeperCluster cluster2 = hbt.startMiniZKCluster(5);
     int defaultClientPort = 21818;
     cluster2.setDefaultClientPort(defaultClientPort);
     try {
       assertEquals(4, cluster2.getBackupZooKeeperServerNum());
-      
+
       // killing the current active zk server
       assertTrue((cluster2.killCurrentActiveZooKeeperServer() >= defaultClientPort));
-      assertTrue((cluster2.killCurrentActiveZooKeeperServer() >= defaultClientPort));     
+      assertTrue((cluster2.killCurrentActiveZooKeeperServer() >= defaultClientPort));
       assertEquals(2, cluster2.getBackupZooKeeperServerNum());
       assertEquals(3, cluster2.getZooKeeperServerNum());
-      
+
       // killing the backup zk servers
       cluster2.killOneBackupZooKeeperServer();
       cluster2.killOneBackupZooKeeperServer();
       assertEquals(0, cluster2.getBackupZooKeeperServerNum());
       assertEquals(1, cluster2.getZooKeeperServerNum());
-      
+
       // killing the last zk server
       assertTrue((cluster2.killCurrentActiveZooKeeperServer() == -1));
       // this should do nothing.
       cluster2.killOneBackupZooKeeperServer();
       assertEquals(-1, cluster2.getBackupZooKeeperServerNum());
-      assertEquals(0, cluster2.getZooKeeperServerNum());         
+      assertEquals(0, cluster2.getZooKeeperServerNum());
     } finally {
-      cluster2.shutdown();
+      hbt.shutdownMiniZKCluster();
     }
   }
 
   @Test public void testMiniDFSCluster() throws Exception {
-    MiniDFSCluster cluster = this.hbt.startMiniDFSCluster(1);
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+    MiniDFSCluster cluster = hbt.startMiniDFSCluster(1);
     FileSystem dfs = cluster.getFileSystem();
     Path dir = new Path("dir");
     Path qualifiedDir = dfs.makeQualified(dir);
@@ -194,26 +197,32 @@ public class TestHBaseTestingUtility {
     assertFalse(dfs.exists(qualifiedDir));
     assertTrue(dfs.mkdirs(qualifiedDir));
     assertTrue(dfs.delete(qualifiedDir, true));
-    try {
-    } finally {
-      cluster.shutdown();
-    }
+    hbt.shutdownMiniCluster();
   }
 
-  @Test public void testSetupClusterTestBuildDir() {
-    File testdir = this.hbt.setupClusterTestBuildDir();
+  @Test public void testSetupClusterTestBuildDir() throws Exception {
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+    Path testdir = hbt.getClusterTestDir();
     LOG.info("uuid-subdir=" + testdir);
-    assertFalse(testdir.exists());
-    assertTrue(testdir.mkdirs());
-    assertTrue(testdir.exists());
+    FileSystem fs = hbt.getTestFileSystem();
+
+    assertFalse(fs.exists(testdir));
+
+    hbt.startMiniDFSCluster(1);
+    assertTrue(fs.exists(testdir));
+
+    hbt.shutdownMiniCluster();
+    assertFalse(fs.exists(testdir));
+
   }
 
-  @Test public void testTestDir() throws IOException {
-    Path testdir = HBaseTestingUtility.getTestDir();
+  @Test public void testTestDir() throws Exception {
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+    Path testdir = hbt.getDataTestDir();
     LOG.info("testdir=" + testdir);
-    FileSystem fs = this.hbt.getTestFileSystem();
+    FileSystem fs = hbt.getTestFileSystem();
     assertTrue(!fs.exists(testdir));
     assertTrue(fs.mkdirs(testdir));
-    assertTrue(this.hbt.cleanupTestDir());
+    assertTrue(hbt.cleanupTestDir());
   }
 }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java Wed Oct 26 20:29:41 2011
@@ -49,7 +49,7 @@ public class TestInfoServers {
   }
 
   @AfterClass
-  public static void afterClass() throws IOException {
+  public static void afterClass() throws Exception {
     UTIL.shutdownMiniCluster();
   }
 
@@ -60,10 +60,10 @@ public class TestInfoServers {
   public void testInfoServersRedirect() throws Exception {
     // give the cluster time to start up
     new HTable(UTIL.getConfiguration(), ".META.");
-    int port = UTIL.getHbaseCluster().getMaster().getInfoServer().getPort();
+    int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
     assertHasExpectedContent(new URL("http://localhost:" + port +
       "/index.html"), "master-status");
-    port = UTIL.getHbaseCluster().getRegionServerThreads().get(0).getRegionServer().
+    port = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer().
       getInfoServer().getPort();
     assertHasExpectedContent(new URL("http://localhost:" + port +
       "/index.html"), "rs-status");
@@ -80,10 +80,10 @@ public class TestInfoServers {
   public void testInfoServersStatusPages() throws Exception {
     // give the cluster time to start up
     new HTable(UTIL.getConfiguration(), ".META.");
-    int port = UTIL.getHbaseCluster().getMaster().getInfoServer().getPort();
+    int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
     assertHasExpectedContent(new URL("http://localhost:" + port +
       "/master-status"), "META");
-    port = UTIL.getHbaseCluster().getRegionServerThreads().get(0).getRegionServer().
+    port = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer().
       getInfoServer().getPort();
     assertHasExpectedContent(new URL("http://localhost:" + port +
       "/rs-status"), "META");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java Wed Oct 26 20:29:41 2011
@@ -97,7 +97,7 @@ public class TestMultiVersions {
     Incommon incommon = new HTableIncommon(table);
     TimestampTestBase.doTestDelete(incommon, new FlushCache() {
       public void flushcache() throws IOException {
-        UTIL.getHbaseCluster().flushcache();
+        UTIL.getHBaseCluster().flushcache();
       }
      });
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java Wed Oct 26 20:29:41 2011
@@ -56,7 +56,7 @@ public class TestRegionRebalancing {
   }
 
   @AfterClass
-  public static void afterClass() throws IOException {
+  public static void afterClass() throws Exception {
     UTIL.shutdownMiniCluster();
   }
 
@@ -94,35 +94,35 @@ public class TestRegionRebalancing {
 
     // add a region server - total of 2
     LOG.info("Started second server=" +
-      UTIL.getHbaseCluster().startRegionServer().getRegionServer().getServerName());
-    UTIL.getHbaseCluster().getMaster().balance();
+      UTIL.getHBaseCluster().startRegionServer().getRegionServer().getServerName());
+    UTIL.getHBaseCluster().getMaster().balance();
     assertRegionsAreBalanced();
 
     // add a region server - total of 3
     LOG.info("Started third server=" +
-        UTIL.getHbaseCluster().startRegionServer().getRegionServer().getServerName());
-    UTIL.getHbaseCluster().getMaster().balance();
+        UTIL.getHBaseCluster().startRegionServer().getRegionServer().getServerName());
+    UTIL.getHBaseCluster().getMaster().balance();
     assertRegionsAreBalanced();
 
     // kill a region server - total of 2
-    LOG.info("Stopped third server=" + UTIL.getHbaseCluster().stopRegionServer(2, false));
-    UTIL.getHbaseCluster().waitOnRegionServer(2);
-    UTIL.getHbaseCluster().getMaster().balance();
+    LOG.info("Stopped third server=" + UTIL.getHBaseCluster().stopRegionServer(2, false));
+    UTIL.getHBaseCluster().waitOnRegionServer(2);
+    UTIL.getHBaseCluster().getMaster().balance();
     assertRegionsAreBalanced();
 
     // start two more region servers - total of 4
     LOG.info("Readding third server=" +
-        UTIL.getHbaseCluster().startRegionServer().getRegionServer().getServerName());
+        UTIL.getHBaseCluster().startRegionServer().getRegionServer().getServerName());
     LOG.info("Added fourth server=" +
-        UTIL.getHbaseCluster().startRegionServer().getRegionServer().getServerName());
-    UTIL.getHbaseCluster().getMaster().balance();
+        UTIL.getHBaseCluster().startRegionServer().getRegionServer().getServerName());
+    UTIL.getHBaseCluster().getMaster().balance();
     assertRegionsAreBalanced();
 
     for (int i = 0; i < 6; i++){
       LOG.info("Adding " + (i + 5) + "th region server");
-      UTIL.getHbaseCluster().startRegionServer();
+      UTIL.getHBaseCluster().startRegionServer();
     }
-    UTIL.getHbaseCluster().getMaster().balance();
+    UTIL.getHBaseCluster().getMaster().balance();
     assertRegionsAreBalanced();
   }
 
@@ -154,7 +154,7 @@ public class TestRegionRebalancing {
 
       int regionCount = getRegionCount();
       List<HRegionServer> servers = getOnlineRegionServers();
-      double avg = UTIL.getHbaseCluster().getMaster().getAverageLoad();
+      double avg = UTIL.getHBaseCluster().getMaster().getAverageLoad();
       int avgLoadPlusSlop = (int)Math.ceil(avg * (1 + slop));
       int avgLoadMinusSlop = (int)Math.floor(avg * (1 - slop)) - 1;
       LOG.debug("There are " + servers.size() + " servers and " + regionCount
@@ -179,7 +179,7 @@ public class TestRegionRebalancing {
           Thread.sleep(10000);
         } catch (InterruptedException e) {}
 
-        UTIL.getHbaseCluster().getMaster().balance();
+        UTIL.getHBaseCluster().getMaster().balance();
         continue;
       }
 
@@ -194,7 +194,7 @@ public class TestRegionRebalancing {
   private List<HRegionServer> getOnlineRegionServers() {
     List<HRegionServer> list = new ArrayList<HRegionServer>();
     for (JVMClusterUtil.RegionServerThread rst :
-        UTIL.getHbaseCluster().getRegionServerThreads()) {
+        UTIL.getHBaseCluster().getRegionServerThreads()) {
       if (rst.getRegionServer().isOnline()) {
         list.add(rst.getRegionServer());
       }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java Wed Oct 26 20:29:41 2011
@@ -85,7 +85,7 @@ public class TestMetaReaderEditor {
     ct.start();
   }
 
-  @AfterClass public static void afterClass() throws IOException {
+  @AfterClass public static void afterClass() throws Exception {
     UTIL.shutdownMiniCluster();
   }
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Wed Oct 26 20:29:41 2011
@@ -1241,7 +1241,7 @@ public class TestAdmin {
       throws IOException {
     // When the META table can be opened, the region servers are running
     new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
-    HRegionServer regionServer = TEST_UTIL.getHbaseCluster()
+    HRegionServer regionServer = TEST_UTIL.getHBaseCluster()
         .getRegionServerThreads().get(0).getRegionServer();
 
     // Create the test table and open it

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Wed Oct 26 20:29:41 2011
@@ -44,7 +44,6 @@ import java.util.concurrent.ExecutorServ
 import java.util.concurrent.Executors;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -225,7 +224,7 @@ public class TestFromClientSide {
      ResultScanner scanner = table.getScanner(s);
      while (scanner.next() != null) continue;
 
-     Path tempPath = new Path(HBaseTestingUtility.getTestDir(), "regions.dat");
+     Path tempPath = new Path(TEST_UTIL.getDataTestDir(), "regions.dat");
 
      final String tempFileName = tempPath.toString();
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java Wed Oct 26 20:29:41 2011
@@ -51,7 +51,7 @@ public class TestHTablePool {
 		}
 
     @AfterClass
-		public static void tearDownAfterClass() throws IOException {
+		public static void tearDownAfterClass() throws Exception {
 			TEST_UTIL.shutdownMiniCluster();
 		}
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java Wed Oct 26 20:29:41 2011
@@ -62,9 +62,9 @@ public class TestReplicationAdmin {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true);
     admin = new ReplicationAdmin(conf);
-    Path oldLogDir = new Path(TEST_UTIL.getTestDir(),
+    Path oldLogDir = new Path(TEST_UTIL.getDataTestDir(),
         HConstants.HREGION_OLDLOGDIR_NAME);
-    Path logDir = new Path(TEST_UTIL.getTestDir(),
+    Path logDir = new Path(TEST_UTIL.getDataTestDir(),
         HConstants.HREGION_LOGDIR_NAME);
     manager = new ReplicationSourceManager(admin.getReplicationZk(), conf,
         // The following stopper never stops so that we can respond

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java Wed Oct 26 20:29:41 2011
@@ -157,8 +157,8 @@ public class TestClassLoading {
     // compose a java source file.
     String javaCode = "import org.apache.hadoop.hbase.coprocessor.*;" +
       "public class " + className + " extends BaseRegionObserver {}";
-    Path baseDir = HBaseTestingUtility.getTestDir();
-    Path srcDir = new Path(HBaseTestingUtility.getTestDir(), "src");
+    Path baseDir = TEST_UTIL.getDataTestDir();
+    Path srcDir = new Path(TEST_UTIL.getDataTestDir(), "src");
     File srcDirPath = new File(srcDir.toString());
     srcDirPath.mkdirs();
     File sourceCodeFile = new File(srcDir.toString(), className + ".java");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java Wed Oct 26 20:29:41 2011
@@ -479,7 +479,7 @@ public class TestMasterObserver {
   }
 
   @AfterClass
-  public static void teardownAfterClass() throws Exception {
+  public static void tearDownAfterClass() throws Exception {
     UTIL.shutdownMiniCluster();
   }
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java Wed Oct 26 20:29:41 2011
@@ -51,8 +51,8 @@ public class TestColumnPrefixFilter {
     HTableDescriptor htd = new HTableDescriptor("TestColumnPrefixFilter");
     htd.addFamily(new HColumnDescriptor(family));
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-    HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
-        getTestDir(), TEST_UTIL.getConfiguration(), htd);
+    HRegion region = HRegion.createHRegion(info, TEST_UTIL.
+      getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
 
     List<String> rows = generateRandomWords(100, "row");
     List<String> columns = generateRandomWords(10000, "column");
@@ -107,8 +107,8 @@ public class TestColumnPrefixFilter {
     HTableDescriptor htd = new HTableDescriptor("TestColumnPrefixFilter");
     htd.addFamily(new HColumnDescriptor(family));
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-    HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
-        getTestDir(), TEST_UTIL.getConfiguration(), htd);
+    HRegion region = HRegion.createHRegion(info, TEST_UTIL.
+      getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
 
     List<String> rows = generateRandomWords(100, "row");
     List<String> columns = generateRandomWords(10000, "column");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java Wed Oct 26 20:29:41 2011
@@ -76,7 +76,7 @@ public class TestDependentColumnFilter e
     htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
     htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-    this.region = HRegion.createHRegion(info, testUtil.getTestDir(),
+    this.region = HRegion.createHRegion(info, testUtil.getDataTestDir(),
         testUtil.getConfiguration(), htd);
     addData();
   }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java Wed Oct 26 20:29:41 2011
@@ -52,8 +52,8 @@ public class TestMultipleColumnPrefixFil
     htd.addFamily(new HColumnDescriptor(family));
     // HRegionInfo info = new HRegionInfo(htd, null, null, false);
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-    HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
-        getTestDir(), TEST_UTIL.getConfiguration(), htd);
+    HRegion region = HRegion.createHRegion(info, TEST_UTIL.
+      getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
 
     List<String> rows = generateRandomWords(100, "row");
     List<String> columns = generateRandomWords(10000, "column");
@@ -111,8 +111,8 @@ public class TestMultipleColumnPrefixFil
     htd.addFamily(new HColumnDescriptor(family1));
     htd.addFamily(new HColumnDescriptor(family2));
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-    HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
-        getTestDir(), TEST_UTIL.getConfiguration(), htd);
+    HRegion region = HRegion.createHRegion(info, TEST_UTIL.
+      getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
 
     List<String> rows = generateRandomWords(100, "row");
     List<String> columns = generateRandomWords(10000, "column");
@@ -174,8 +174,8 @@ public class TestMultipleColumnPrefixFil
     HTableDescriptor htd = new HTableDescriptor("TestMultipleColumnPrefixFilter");
     htd.addFamily(new HColumnDescriptor(family));
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-    HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
-        getTestDir(), TEST_UTIL.getConfiguration(),htd);
+    HRegion region = HRegion.createHRegion(info, TEST_UTIL.
+      getDataTestDir(), TEST_UTIL.getConfiguration(),htd);
 
     List<String> rows = generateRandomWords(100, "row");
     List<String> columns = generateRandomWords(10000, "column");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java Wed Oct 26 20:29:41 2011
@@ -59,7 +59,7 @@ public class TestHalfStoreFileReader {
   @Test
   public void testHalfScanAndReseek() throws IOException {
     HBaseTestingUtility test_util = new HBaseTestingUtility();
-    String root_dir = HBaseTestingUtility.getTestDir("TestHalfStoreFile").toString();
+    String root_dir = test_util.getDataTestDir("TestHalfStoreFile").toString();
     Path p = new Path(root_dir, "test");
 
     Configuration conf = test_util.getConfiguration();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java Wed Oct 26 20:29:41 2011
@@ -34,14 +34,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.Compression;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileBlock;
-import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
-import org.apache.hadoop.hbase.io.hfile.HFileReaderV2;
-import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.junit.After;
@@ -216,7 +208,7 @@ public class TestCacheOnWrite {
   }
 
   public void writeStoreFile() throws IOException {
-    Path storeFileParentDir = new Path(HBaseTestingUtility.getTestDir(),
+    Path storeFileParentDir = new Path(TEST_UTIL.getDataTestDir(),
         "test_cache_on_write");
     StoreFile.Writer sfw = StoreFile.createWriter(fs, storeFileParentDir,
         DATA_BLOCK_SIZE, compress, KeyValue.COMPARATOR, conf,

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java Wed Oct 26 20:29:41 2011
@@ -127,7 +127,7 @@ public class TestFixedFileTrailer {
     }
 
     // Now check what happens if the trailer is corrupted.
-    Path trailerPath = new Path(HBaseTestingUtility.getTestDir(), "trailer_"
+    Path trailerPath = new Path(util.getDataTestDir(), "trailer_"
         + version);
 
     {

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java Wed Oct 26 20:29:41 2011
@@ -49,12 +49,16 @@ import org.apache.hadoop.io.Writable;
 public class TestHFile extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestHFile.class);
 
-  private static String ROOT_DIR =
-    HBaseTestingUtility.getTestDir("TestHFile").toString();
+  private String ROOT_DIR;
   private final int minBlockSize = 512;
   private static String localFormatter = "%010d";
   private static CacheConfig cacheConf = null;
 
+  public void setUp() throws Exception {
+    super.setUp();
+    ROOT_DIR = this.getUnitTestdir("TestHFile").toString();
+  }
+
   /**
    * Test empty HFile.
    * Test all features work reasonably when hfile is empty of entries.
@@ -123,7 +127,7 @@ public class TestHFile extends HBaseTest
   }
 
   private FSDataOutputStream createFSOutput(Path name) throws IOException {
-    if (fs.exists(name)) fs.delete(name, true);
+    //if (fs.exists(name)) fs.delete(name, true);
     FSDataOutputStream fout = fs.create(name);
     return fout;
   }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java Wed Oct 26 20:29:41 2011
@@ -69,14 +69,13 @@ public class TestHFileBlock {
   private static final int NUM_READER_THREADS = 26;
 
   private static final HBaseTestingUtility TEST_UTIL =
-      new HBaseTestingUtility();
+    new HBaseTestingUtility();
   private FileSystem fs;
   private int uncompressedSizeV1;
 
   @Before
   public void setUp() throws IOException {
     fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    TEST_UTIL.initTestDir();
   }
 
   public void writeTestBlockContents(DataOutputStream dos) throws IOException {
@@ -154,8 +153,8 @@ public class TestHFileBlock {
     for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
       for (boolean pread : new boolean[] { false, true }) {
         byte[] block = createTestV1Block(algo);
-        Path path = new Path(HBaseTestingUtility.getTestDir(), "blocks_v1_"
-            + algo);
+        Path path = new Path(TEST_UTIL.getDataTestDir(),
+          "blocks_v1_"+ algo);
         LOG.info("Creating temporary file at " + path);
         FSDataOutputStream os = fs.create(path);
         int totalSize = 0;
@@ -188,7 +187,7 @@ public class TestHFileBlock {
   public void testReaderV2() throws IOException {
     for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
       for (boolean pread : new boolean[] { false, true }) {
-        Path path = new Path(HBaseTestingUtility.getTestDir(), "blocks_v2_"
+        Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
             + algo);
         FSDataOutputStream os = fs.create(path);
         HFileBlock.Writer hbw = new HFileBlock.Writer(algo);
@@ -244,7 +243,7 @@ public class TestHFileBlock {
         for (boolean cacheOnWrite : BOOLEAN_VALUES) {
           Random rand = defaultRandom();
           LOG.info("Compression algorithm: " + algo + ", pread=" + pread);
-          Path path = new Path(HBaseTestingUtility.getTestDir(), "prev_offset");
+          Path path = new Path(TEST_UTIL.getDataTestDir(), "prev_offset");
           List<Long> expectedOffsets = new ArrayList<Long>();
           List<Long> expectedPrevOffsets = new ArrayList<Long>();
           List<BlockType> expectedTypes = new ArrayList<BlockType>();
@@ -400,7 +399,7 @@ public class TestHFileBlock {
   public void testConcurrentReading() throws Exception {
     for (Compression.Algorithm compressAlgo : COMPRESSION_ALGORITHMS) {
       Path path =
-          new Path(HBaseTestingUtility.getTestDir(), "concurrent_reading");
+          new Path(TEST_UTIL.getDataTestDir(), "concurrent_reading");
       Random rand = defaultRandom();
       List<Long> offsets = new ArrayList<Long>();
       List<BlockType> types = new ArrayList<BlockType>();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java Wed Oct 26 20:29:41 2011
@@ -43,8 +43,6 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader;
 import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexChunk;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 
@@ -113,7 +111,7 @@ public class TestHFileBlockIndex {
 
   @Test
   public void testBlockIndex() throws IOException {
-    path = new Path(HBaseTestingUtility.getTestDir(), "block_index_" + compr);
+    path = new Path(TEST_UTIL.getDataTestDir(), "block_index_" + compr);
     writeWholeIndex();
     readIndex();
   }
@@ -458,7 +456,7 @@ public class TestHFileBlockIndex {
    */
   @Test
   public void testHFileWriterAndReader() throws IOException {
-    Path hfilePath = new Path(HBaseTestingUtility.getTestDir(),
+    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
         "hfile_for_block_index");
     CacheConfig cacheConf = new CacheConfig(conf);
     BlockCache blockCache = cacheConf.getBlockCache();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java Wed Oct 26 20:29:41 2011
@@ -45,8 +45,9 @@ import org.apache.hadoop.io.compress.Gzi
  * instead.</p>
  */
 public class TestHFilePerformance extends TestCase {
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static String ROOT_DIR =
-    HBaseTestingUtility.getTestDir("TestHFilePerformance").toString();
+    TEST_UTIL.getDataTestDir("TestHFilePerformance").toString();
   private FileSystem fs;
   private Configuration conf;
   private long startTimeEpoch;

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java Wed Oct 26 20:29:41 2011
@@ -244,8 +244,9 @@ public class TestHFileSeek extends TestC
     int minWordLen = 5;
     int maxWordLen = 20;
 
+    private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
     String rootDir =
-      HBaseTestingUtility.getTestDir("TestTFileSeek").toString();
+      TEST_UTIL.getDataTestDir("TestTFileSeek").toString();
     String file = "TestTFileSeek";
     // String compress = "lzo"; DISABLED
     String compress = "none";

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java Wed Oct 26 20:29:41 2011
@@ -63,7 +63,7 @@ public class TestHFileWriterV2 {
 
   @Test
   public void testHFileFormatV2() throws IOException {
-    Path hfilePath = new Path(HBaseTestingUtility.getTestDir(),
+    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
         "testHFileFormatV2");
 
     final Compression.Algorithm COMPRESS_ALGO = Compression.Algorithm.GZ;

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java Wed Oct 26 20:29:41 2011
@@ -40,7 +40,7 @@ public class TestReseekTo {
   @Test
   public void testReseekTo() throws Exception {
 
-    Path ncTFile = new Path(HBaseTestingUtility.getTestDir(), "basic.hfile");
+    Path ncTFile = new Path(TEST_UTIL.getDataTestDir(), "basic.hfile");
     FSDataOutputStream fout = TEST_UTIL.getTestFileSystem().create(ncTFile);
     CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
     HFile.Writer writer = HFile.getWriterFactory(

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java Wed Oct 26 20:29:41 2011
@@ -80,7 +80,7 @@ public class TestTableMapReduce {
   }
 
   @AfterClass
-  public static void afterClass() throws IOException {
+  public static void afterClass() throws Exception {
     UTIL.shutdownMiniMapReduceCluster();
     UTIL.shutdownMiniCluster();
   }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java Wed Oct 26 20:29:41 2011
@@ -175,7 +175,7 @@ public class TestHFileOutputFormat  {
     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
     TaskAttemptContext context = null;
     Path dir =
-      HBaseTestingUtility.getTestDir("test_LATEST_TIMESTAMP_isReplaced");
+      util.getDataTestDir("test_LATEST_TIMESTAMP_isReplaced");
     try {
       Job job = new Job(conf);
       FileOutputFormat.setOutputPath(job, dir);
@@ -243,7 +243,7 @@ public class TestHFileOutputFormat  {
     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
     TaskAttemptContext context = null;
     Path dir =
-      HBaseTestingUtility.getTestDir("test_TIMERANGE_present");
+      util.getDataTestDir("test_TIMERANGE_present");
     LOG.info("Timerange dir writing to dir: "+ dir);
     try {
       // build a record writer using HFileOutputFormat
@@ -307,7 +307,7 @@ public class TestHFileOutputFormat  {
   @Test
   public void testWritingPEData() throws Exception {
     Configuration conf = util.getConfiguration();
-    Path testDir = HBaseTestingUtility.getTestDir("testWritingPEData");
+    Path testDir = util.getDataTestDir("testWritingPEData");
     FileSystem fs = testDir.getFileSystem(conf);
     
     // Set down this value or we OOME in eclipse.
@@ -372,7 +372,7 @@ public class TestHFileOutputFormat  {
   private void doIncrementalLoadTest(
       boolean shouldChangeRegions) throws Exception {
     Configuration conf = util.getConfiguration();
-    Path testDir = HBaseTestingUtility.getTestDir("testLocalMRIncrementalLoad");
+    Path testDir = util.getDataTestDir("testLocalMRIncrementalLoad");
     byte[][] startKeys = generateRandomStartKeys(5);
     
     try {
@@ -557,7 +557,7 @@ public class TestHFileOutputFormat  {
     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
     TaskAttemptContext context = null;
     Path dir =
-        HBaseTestingUtility.getTestDir("testColumnFamilyCompression");
+        util.getDataTestDir("testColumnFamilyCompression");
 
     HTable table = Mockito.mock(HTable.class);
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java Wed Oct 26 20:29:41 2011
@@ -252,7 +252,7 @@ public class TestImportTsv {
       }
       assertTrue(verified);
     } finally {
-      cluster.shutdown();
+      htu1.shutdownMiniCluster();
     }
   }
   

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java Wed Oct 26 20:29:41 2011
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.io.hfile.
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
+import org.junit.*;
 
 /**
  * Test cases for the "load" half of the HFileOutputFormat bulk load
@@ -49,8 +49,6 @@ import org.junit.Test;
  * tests in TestHFileOutputFormat
  */
 public class TestLoadIncrementalHFiles {
-
-  private static final byte[] TABLE = Bytes.toBytes("mytable");
   private static final byte[] QUALIFIER = Bytes.toBytes("myqual");
   private static final byte[] FAMILY = Bytes.toBytes("myfam");
 
@@ -63,7 +61,17 @@ public class TestLoadIncrementalHFiles {
   public static String COMPRESSION =
     Compression.Algorithm.NONE.getName();
 
-  private HBaseTestingUtility util = new HBaseTestingUtility();
+  private static HBaseTestingUtility util = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    util.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
 
   /**
    * Test case that creates some regions and loads
@@ -117,7 +125,7 @@ public class TestLoadIncrementalHFiles {
 
   private void runTest(String testName, BloomType bloomType, 
           byte[][][] hfileRanges) throws Exception {
-    Path dir = HBaseTestingUtility.getTestDir(testName);
+    Path dir = util.getDataTestDir(testName);
     FileSystem fs = util.getTestFileSystem();
     dir = dir.makeQualified(fs);
     Path familyDir = new Path(dir, Bytes.toString(FAMILY));
@@ -131,31 +139,27 @@ public class TestLoadIncrementalHFiles {
     }
     int expectedRows = hfileIdx * 1000;
 
+    final byte[] TABLE = Bytes.toBytes("mytable_"+testName);
 
-    util.startMiniCluster();
-    try {
-      HBaseAdmin admin = new HBaseAdmin(util.getConfiguration());
-      HTableDescriptor htd = new HTableDescriptor(TABLE);
-      HColumnDescriptor familyDesc = new HColumnDescriptor(FAMILY);
-      familyDesc.setBloomFilterType(bloomType);
-      htd.addFamily(familyDesc);
-      admin.createTable(htd, SPLIT_KEYS);
-
-      HTable table = new HTable(util.getConfiguration(), TABLE);
-      util.waitTableAvailable(TABLE, 30000);
-      LoadIncrementalHFiles loader = new LoadIncrementalHFiles(
-          util.getConfiguration());
-      loader.doBulkLoad(dir, table);
+    HBaseAdmin admin = new HBaseAdmin(util.getConfiguration());
+    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HColumnDescriptor familyDesc = new HColumnDescriptor(FAMILY);
+    familyDesc.setBloomFilterType(bloomType);
+    htd.addFamily(familyDesc);
+    admin.createTable(htd, SPLIT_KEYS);
+
+    HTable table = new HTable(util.getConfiguration(), TABLE);
+    util.waitTableAvailable(TABLE, 30000);
+    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(
+      util.getConfiguration());
+    loader.doBulkLoad(dir, table);
 
-      assertEquals(expectedRows, util.countRows(table));
-    } finally {
-      util.shutdownMiniCluster();
-    }
+    assertEquals(expectedRows, util.countRows(table));
   }
 
   @Test
   public void testSplitStoreFile() throws IOException {
-    Path dir = HBaseTestingUtility.getTestDir("testSplitHFile");
+    Path dir = util.getDataTestDir("testSplitHFile");
     FileSystem fs = util.getTestFileSystem();
     Path testIn = new Path(dir, "testhfile");
     HColumnDescriptor familyDesc = new HColumnDescriptor(FAMILY);

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java Wed Oct 26 20:29:41 2011
@@ -76,7 +76,7 @@ public class TestTableMapReduce {
   }
 
   @AfterClass
-  public static void afterClass() throws IOException {
+  public static void afterClass() throws Exception {
     UTIL.shutdownMiniMapReduceCluster();
     UTIL.shutdownMiniCluster();
   }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Wed Oct 26 20:29:41 2011
@@ -433,7 +433,7 @@ public class TestCatalogJanitor {
   private String setRootDirAndCleanIt(final HBaseTestingUtility htu,
       final String subdir)
   throws IOException {
-    Path testdir = HBaseTestingUtility.getTestDir(subdir);
+    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());

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java Wed Oct 26 20:29:41 2011
@@ -83,11 +83,6 @@ public class TestDistributedLogSplitting
   Configuration conf;
   HBaseTestingUtility TEST_UTIL;
 
-  @Before
-  public void before() throws Exception {
-
-  }
-
   private void startCluster(int num_rs) throws Exception{
     ZKSplitLog.Counters.resetCounters();
     LOG.info("Starting cluster");
@@ -106,7 +101,7 @@ public class TestDistributedLogSplitting
 
   @After
   public void after() throws Exception {
-    cluster.shutdown();
+    TEST_UTIL.shutdownMiniCluster();
   }
 
   @Test (timeout=300000)

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLogsCleaner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLogsCleaner.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLogsCleaner.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLogsCleaner.java Wed Oct 26 20:29:41 2011
@@ -73,7 +73,7 @@ public class TestLogsCleaner {
     ReplicationZookeeper zkHelper =
         new ReplicationZookeeper(server, new AtomicBoolean(true));
 
-    Path oldLogDir = new Path(HBaseTestingUtility.getTestDir(),
+    Path oldLogDir = new Path(TEST_UTIL.getDataTestDir(),
         HConstants.HREGION_OLDLOGDIR_NAME);
     String fakeMachineName =
       URLEncoder.encode(server.getServerName().toString(), "UTF8");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java Wed Oct 26 20:29:41 2011
@@ -61,7 +61,7 @@ public class TestMaster {
   }
 
   @AfterClass
-  public static void afterAllTests() throws IOException {
+  public static void afterAllTests() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java Wed Oct 26 20:29:41 2011
@@ -105,6 +105,7 @@ public class TestMasterRestartAfterDisab
     assertEquals(
         "The assigned regions were not onlined after master switch except for the catalog tables.",
         6, regions.size());
+    TEST_UTIL.shutdownMiniCluster();
   }
 
   private void log(String msg) {

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java?rev=1189429&r1=1189428&r2=1189429&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java Wed Oct 26 20:29:41 2011
@@ -66,7 +66,7 @@ public class TestMasterTransitions {
     addToEachStartKey(countOfRegions);
   }
 
-  @AfterClass public static void afterAllTests() throws IOException {
+  @AfterClass public static void afterAllTests() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }