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 2010/08/08 07:40:06 UTC

svn commit: r983345 - in /hbase/branches/0.90_master_rewrite/src: main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/util/ test/java/org/apache/hadoop/hbase/ test/java/org/apache/...

Author: stack
Date: Sun Aug  8 05:40:05 2010
New Revision: 983345

URL: http://svn.apache.org/viewvc?rev=983345&view=rev
Log:
Starting in on unit tests, making more pass

M  src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
  Added a bit of javadoc.
  Added utilty to the cleanup test dir methods.
M src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
  Started test of our testing utility.
M src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java
  Fixed failing test (moved it to junit4 while I was at it).
M src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
  Fixed compilation error.
M src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
  Removed implementation of regionserver Stoppable (yes, we had 
  a Stoppable in the regionserver package already)
  (stopRequested) Changed to be like master, stopped.
  Passed the sleeper this because Sleeper now takes Stoppable
  Fixed up all the calls to 'stop' now passing message on why to stop.
D src/main/java/org/apache/hadoop/hbase/regionserver/Stoppable.java
M src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java
M src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
M src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
M src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
M src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
  Modify to use new Stoppable interface.
M src/main/java/org/apache/hadoop/hbase/master/HMaster.java
  Fix name of master thread (was master-null)

Added:
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
Removed:
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/Stoppable.java
Modified:
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java
    hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
    hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Sun Aug  8 05:40:05 2010
@@ -171,8 +171,6 @@ implements HMasterInterface, HMasterRegi
   public HMaster(Configuration conf) throws IOException, KeeperException {
     // initialize some variables
     this.conf = conf;
-    // set the thread name
-    setName(MASTER + "-" + this.address);
 
     /*
      * 1. Determine address and initialize RPC server (but do not start).
@@ -184,6 +182,9 @@ implements HMasterInterface, HMasterRegi
       numHandlers, false, conf);
     this.address = new HServerAddress(rpcServer.getListenerAddress());
 
+    // set the thread name now we have an address
+    setName(MASTER + "-" + this.address);
+
     /*
      * 2. Determine if this is a fresh cluster startup or failed over master.
      *  This is done by checking for the existence of any ephemeral

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Sun Aug  8 05:40:05 2010
@@ -68,12 +68,11 @@ public class CompactSplitThread extends 
 
   @Override
   public void run() {
-    int count = 0;
-    while (!this.server.isStopRequested()) {
+    while (!this.server.isStopped()) {
       HRegion r = null;
       try {
         r = compactionQueue.poll(this.frequency, TimeUnit.MILLISECONDS);
-        if (r != null && !this.server.isStopRequested()) {
+        if (r != null && !this.server.isStopped()) {
           synchronized (regionsInQueue) {
             regionsInQueue.remove(r);
           }
@@ -81,7 +80,7 @@ public class CompactSplitThread extends 
           try {
             // Don't interrupt us while we are working
             byte [] midKey = r.compactStores();
-            if (midKey != null && !this.server.isStopRequested()) {
+            if (midKey != null && !this.server.isStopped()) {
               split(r, midKey);
             }
           } finally {
@@ -127,7 +126,7 @@ public class CompactSplitThread extends 
    */
   public synchronized void compactionRequested(final HRegion r,
       final boolean force, final String why) {
-    if (this.server.stopRequested.get()) {
+    if (this.server.isStopped()) {
       return;
     }
     r.setForceMajorCompaction(force);

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Sun Aug  8 05:40:05 2010
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.LocalHBas
 import org.apache.hadoop.hbase.MasterAddressTracker;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
+import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.UnknownRowLockException;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.YouAreDeadException;
@@ -123,7 +124,7 @@ import org.apache.zookeeper.KeeperExcept
  * the HMaster. There are many HRegionServers in a single HBase deployment.
  */
 public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
-    Runnable, Stoppable, RegionServerController {
+    Runnable, RegionServerController {
   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
   private static final HMsg REPORT_EXITING = new HMsg(Type.MSG_REPORT_EXITING);
   private static final HMsg REPORT_QUIESCED = new HMsg(Type.MSG_REPORT_QUIESCED);
@@ -131,10 +132,8 @@ public class HRegionServer implements HR
 
   // Set when a report to the master comes back with a message asking us to
   // shutdown. Also set by call to stop when debugging or running unit tests
-  // of HRegionServer in isolation. We use AtomicBoolean rather than
-  // plain boolean so we can pass a reference to Chore threads. Otherwise,
-  // Chore threads need to know about the hosting class.
-  protected final AtomicBoolean stopRequested = new AtomicBoolean(false);
+  // of HRegionServer in isolation.
+  protected volatile boolean stopped = false;
 
   protected final AtomicBoolean quiesced = new AtomicBoolean(false);
 
@@ -283,7 +282,7 @@ public class HRegionServer implements HR
         10 * 1000);
     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 1 * 1000);
 
-    sleeper = new Sleeper(this.msgInterval, this.stopRequested);
+    sleeper = new Sleeper(this.msgInterval, this);
 
     this.maxScannerResultSize = conf.getLong(
         HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
@@ -311,7 +310,7 @@ public class HRegionServer implements HR
    */
   private void initialize() throws IOException {
     this.abortRequested = false;
-    this.stopRequested.set(false);
+    this.stopped = false;
 
     // Server to handle client requests
     this.server = HBaseRPC.getServer(this, address.getBindAddress(), address
@@ -390,7 +389,7 @@ public class HRegionServer implements HR
     int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY
         + ".multiplier", 1000);
     this.majorCompactionChecker = new MajorCompactionChecker(this,
-        this.threadWakeFrequency * multiplier, this.stopRequested);
+        this.threadWakeFrequency * multiplier, this);
 
     this.leases = new Leases((int) conf.getLong(
         HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
@@ -408,7 +407,7 @@ public class HRegionServer implements HR
     boolean quiesceRequested = false;
     try {
       MapWritable w = null;
-      while (!stopRequested.get()) {
+      while (!this.stopped) {
         w = reportForDuty();
         if (w != null) {
           init(w);
@@ -421,7 +420,7 @@ public class HRegionServer implements HR
       List<HMsg> outboundMessages = new ArrayList<HMsg>();
       long lastMsg = 0;
       // Now ask master what it wants us to do and tell it what we have done
-      for (int tries = 0; !stopRequested.get() && isHealthy();) {
+      for (int tries = 0; !this.stopped && isHealthy();) {
         // Try to get the root region location from zookeeper.
         if (!haveRootRegion.get()) {
           HServerAddress rootServer = catalogTracker.getRootLocation();
@@ -461,22 +460,21 @@ public class HRegionServer implements HR
             if (this.quiesced.get() && onlineRegions.size() == 0) {
               // We've just told the master we're exiting because we aren't
               // serving any regions. So set the stop bit and exit.
-              LOG.info("Server quiesced and not serving any regions. "
-                  + "Starting shutdown");
-              stopRequested.set(true);
+              stop("Server quiesced and not serving any regions. " +
+                "Starting shutdown");
               this.outboundMsgs.clear();
               continue;
             }
 
             // Queue up the HMaster's instruction stream for processing
             boolean restart = false;
-            for (int i = 0; !restart && !stopRequested.get() && i < msgs.length; i++) {
+            for (int i = 0; !restart && !stopped && i < msgs.length; i++) {
               LOG.info(msgs[i].toString());
               this.connection.unsetRootRegionLocation();
               switch (msgs[i].getType()) {
 
                 case MSG_REGIONSERVER_STOP:
-                  stopRequested.set(true);
+                  stop("MSG_REGIONSERVER_STOP");
                   break;
 
                 case MSG_REGIONSERVER_QUIESCE:
@@ -505,7 +503,7 @@ public class HRegionServer implements HR
             // Reset tries count if we had a successful transaction.
             tries = 0;
 
-            if (restart || this.stopRequested.get()) {
+            if (restart || this.stopped) {
               toDo.clear();
               continue;
             }
@@ -524,7 +522,7 @@ public class HRegionServer implements HR
               // Check filesystem every so often.
               checkFileSystem();
             }
-            if (this.stopRequested.get()) {
+            if (this.stopped) {
               LOG.info("Stop requested, clearing toDo despite exception");
               toDo.clear();
               continue;
@@ -723,7 +721,7 @@ public class HRegionServer implements HR
       isOnline = true;
     } catch (Throwable e) {
       this.isOnline = false;
-      this.stopRequested.set(true);
+      stop("Failed initialization");
       throw convertThrowableToIOE(cleanup(e, "Failed init"),
           "Region server startup failed");
     }
@@ -864,8 +862,8 @@ public class HRegionServer implements HR
     private final HRegionServer instance;
 
     MajorCompactionChecker(final HRegionServer h, final int sleepTime,
-        final AtomicBoolean stopper) {
-      super("MajorCompactionChecker", sleepTime, stopper);
+        final Stoppable stopper) {
+      super("MajorCompactionChecker", sleepTime, h);
       this.instance = h;
       LOG.info("Runs every " + sleepTime + "ms");
     }
@@ -1067,8 +1065,7 @@ public class HRegionServer implements HR
     if (!(leases.isAlive() && compactSplitThread.isAlive()
         && cacheFlusher.isAlive() && hlogRoller.isAlive()
         && workerThread.isAlive() && this.majorCompactionChecker.isAlive())) {
-      // One or more threads are no longer alive - shut down
-      stop();
+      stop("One or more threads are no longer alive -- stop");
       return false;
     }
     return true;
@@ -1079,12 +1076,10 @@ public class HRegionServer implements HR
     return this.hlog;
   }
 
-  /**
-   * Sets a flag that will cause all the HRegionServer threads to shut down in
-   * an orderly fashion. Used by unit tests.
-   */
-  public void stop() {
-    this.stopRequested.set(true);
+  @Override
+  public void stop(final String msg) {
+    this.stopped = true;
+    LOG.info(msg);
     synchronized (this) {
       // Wakes run() if it is sleeping
       notifyAll(); // FindBugs NN_NAKED_NOTIFY
@@ -1112,7 +1107,7 @@ public class HRegionServer implements HR
     if (this.metrics != null) {
       LOG.info("Dump of metrics: " + this.metrics.toString());
     }
-    stop();
+    stop(reason);
   }
 
   /**
@@ -1156,7 +1151,7 @@ public class HRegionServer implements HR
   private boolean getMaster() {
     HServerAddress masterAddress = null;
     while ((masterAddress = masterAddressManager.getMasterAddress()) == null) {
-      if (stopRequested.get()) {
+      if (stopped) {
         return false;
       }
       LOG.debug("No master found, will retry");
@@ -1164,7 +1159,7 @@ public class HRegionServer implements HR
     }
     LOG.info("Telling master at " + masterAddress + " that we are up");
     HMasterRegionInterface master = null;
-    while (!stopRequested.get() && master == null) {
+    while (!stopped && master == null) {
       try {
         // Do initial RPC setup. The final argument indicates that the RPC
         // should retry indefinitely.
@@ -1186,14 +1181,14 @@ public class HRegionServer implements HR
    * us by the master.
    */
   private MapWritable reportForDuty() {
-    while (!stopRequested.get() && !getMaster()) {
+    while (!stopped && !getMaster()) {
       sleeper.sleep();
       LOG.warn("Unable to get master for initialization");
     }
 
     MapWritable result = null;
     long lastMsg = 0;
-    while (!stopRequested.get()) {
+    while (!stopped) {
       try {
         this.requestCount.set(0);
         MemoryUsage memory = ManagementFactory.getMemoryMXBean()
@@ -1269,11 +1264,11 @@ public class HRegionServer implements HR
 
     public void run() {
       try {
-        while (!stopRequested.get()) {
+        while (!stopped) {
           ToDoEntry e = null;
           try {
             e = toDo.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
-            if (e == null || stopRequested.get()) {
+            if (e == null || stopped) {
               continue;
             }
             LOG.info("Worker: " + e.msg);
@@ -1309,11 +1304,10 @@ public class HRegionServer implements HR
                 break;
 
               case TESTING_MSG_BLOCK_RS:
-                while (!stopRequested.get()) {
+                while (!stopped) {
                   Threads.sleep(1000);
                   LOG.info("Regionserver blocked by "
-                      + HMsg.Type.TESTING_MSG_BLOCK_RS + "; "
-                      + stopRequested.get());
+                      + HMsg.Type.TESTING_MSG_BLOCK_RS + "; " + stopped);
                 }
                 break;
 
@@ -1951,8 +1945,8 @@ public class HRegionServer implements HR
 
   @Override
   public void openRegion(HRegionInfo region) {
-    LOG.info("Received request to open region: "
-        + region.getRegionNameAsString());
+    LOG.info("Received request to open region: " +
+      region.getRegionNameAsString());
     if(region.isRootRegion()) {
       new OpenRootHandler(this, catalogTracker, region).submit();
     } else if(region.isMetaRegion()) {
@@ -2023,8 +2017,8 @@ public class HRegionServer implements HR
   /**
    * @return true if a stop has been requested.
    */
-  public boolean isStopRequested() {
-    return this.stopRequested.get();
+  public boolean isStopped() {
+    return this.stopped;
   }
 
   /**
@@ -2195,7 +2189,7 @@ public class HRegionServer implements HR
    * @throws IOException
    */
   protected void checkOpen() throws IOException {
-    if (this.stopRequested.get() || this.abortRequested) {
+    if (this.stopped || this.abortRequested) {
       throw new IOException("Server not running"
           + (this.abortRequested ? ", aborting" : ""));
     }

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java Sun Aug  8 05:40:05 2010
@@ -56,7 +56,7 @@ class LogRoller extends Thread implement
 
   @Override
   public void run() {
-    while (!server.isStopRequested()) {
+    while (!server.isStopped()) {
       long now = System.currentTimeMillis();
       boolean periodic = false;
       if (!rollLog.get()) {

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java Sun Aug  8 05:40:05 2010
@@ -136,7 +136,7 @@ class MemStoreFlusher extends Thread imp
 
   @Override
   public void run() {
-    while (!this.server.isStopRequested()) {
+    while (!this.server.isStopped()) {
       FlushQueueEntry fqe = null;
       try {
         fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java Sun Aug  8 05:40:05 2010
@@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.util.Threads;
 
 /**
@@ -99,7 +100,7 @@ class ShutdownHook {
       LOG.info("Shutdown hook starting; " + RUN_SHUTDOWN_HOOK + "=" + b +
         "; fsShutdownHook=" + this.fsShutdownHook);
       if (b) {
-        this.stop.stop();
+        this.stop.stop("Shutdown hook");
         Threads.shutdown(this.threadToJoin);
         if (this.fsShutdownHook != null) {
           LOG.info("Starting fs shutdown hook thread.");
@@ -198,8 +199,14 @@ class ShutdownHook {
   // Stoppable with nothing to stop.  Used below in main testing.
   static class DoNothingStoppable implements Stoppable {
     @Override
-    public void stop() {
-      // Nothing to do.
+    public boolean isStopped() {
+      // TODO Auto-generated method stub
+      return false;
+    }
+
+    @Override
+    public void stop(String why) {
+      // TODO Auto-generated method stub
     }
   }
 

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java Sun Aug  8 05:40:05 2010
@@ -60,7 +60,7 @@ public class JVMClusterUtil {
       // cases, we'll jump out of the run without setting online flag.  Check
       // stopRequested so we don't wait here a flag that will never be flipped.
       while (!this.regionServer.isOnline() &&
-          !this.regionServer.isStopRequested()) {
+          !this.regionServer.isStopped()) {
         try {
           Thread.sleep(1000);
         } catch (InterruptedException e) {

Modified: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Sun Aug  8 05:40:05 2010
@@ -88,11 +88,14 @@ public class HBaseTestingUtility {
 
   /**
    * System property key to get test directory value.
+   * Name is as it is because mini dfs has hard-codings to put test data here.
    */
   public static final String TEST_DIRECTORY_KEY = "test.build.data";
 
   /**
    * Default parent direccounttory for test output.
+   * Target is at it is because minidfscluster has hard-codings to put its data
+   * here.
    */
   public static final String DEFAULT_TEST_DIRECTORY = "target/build/data";
 
@@ -115,6 +118,8 @@ public class HBaseTestingUtility {
    * @return Where to write test data on local filesystem; usually
    * {@link #DEFAULT_TEST_DIRECTORY}
    * @see #setupClusterTestBuildDir()
+   * @see #clusterTestBuildDir()
+   * @see #getTestFileSystem()
    */
   public static Path getTestDir() {
     return new Path(System.getProperty(TEST_DIRECTORY_KEY,
@@ -126,15 +131,19 @@ public class HBaseTestingUtility {
    * @return Path to a subdirectory named <code>subdirName</code> under
    * {@link #getTestDir()}.
    * @see #setupClusterTestBuildDir()
+   * @see #clusterTestBuildDir(String)
+   * @see #getTestFileSystem()
    */
   public static Path getTestDir(final String subdirName) {
     return new Path(getTestDir(), subdirName);
   }
 
   /**
-   * Home our cluster in a dir under target/test.  Give it a random name
+   * 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 test.build.data System property.  Its what minidfscluster bases
+   * amend the {@link #TEST_DIRECTORY_KEY} System property.  Its 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;
@@ -850,8 +859,34 @@ public class HBaseTestingUtility {
     return FileSystem.get(conf);
   }
 
-  public void cleanupTestDir() throws IOException {
-    getTestDir().getFileSystem(conf).delete(getTestDir(), true);
+  /**
+   * @return True if we removed the test dir
+   * @throws IOException
+   */
+  public boolean cleanupTestDir() throws IOException {
+    return deleteDir(getTestDir());
+  }
+
+  /**
+   * @param subdir Test subdir name.
+   * @return True if we removed the test dir
+   * @throws IOException
+   */
+  public boolean cleanupTestDir(final String subdir) throws IOException {
+    return deleteDir(getTestDir(subdir));
+  }
+
+  /**
+   * @param dir Directory to delete
+   * @return True if we deleted it.
+   * @throws IOException 
+   */
+  public boolean deleteDir(final Path dir) throws IOException {
+    FileSystem fs = getTestFileSystem();
+    if (fs.exists(dir)) {
+      return fs.delete(getTestDir(), true);
+    }
+    return false;
   }
 
   public void waitTableAvailable(byte[] table, long timeoutMillis)

Modified: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java (original)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java Sun Aug  8 05:40:05 2010
@@ -304,7 +304,7 @@ public class MiniHBaseCluster {
     JVMClusterUtil.RegionServerThread server =
       hbaseCluster.getRegionServers().get(serverNumber);
     LOG.info("Stopping " + server.toString());
-    server.getRegionServer().stop();
+    server.getRegionServer().stop("Stopping rs " + serverNumber);
     return server;
   }
 
@@ -467,4 +467,4 @@ public class MiniHBaseCluster {
   throws IOException {
     ((MiniHBaseClusterMaster)getMaster()).addMessage(hrs.getHServerInfo(), msg);
   }
-}
+}
\ No newline at end of file

Added: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java?rev=983345&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java (added)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java Sun Aug  8 05:40:05 2010
@@ -0,0 +1,66 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test our testing utility class
+ */
+public class TestHBaseTestingUtility {
+  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 {
+  }
+
+  @Test public void testTestDir() throws IOException {
+    Path testdir = HBaseTestingUtility.getTestDir();
+    FileSystem fs = this.hbt.getTestFileSystem();
+    assertTrue(!fs.exists(testdir));
+    assertTrue(fs.mkdirs(testdir));
+    assertTrue(this.hbt.cleanupTestDir());
+  }
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java?rev=983345&r1=983344&r2=983345&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java (original)
+++ hbase/branches/0.90_master_rewrite/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java Sun Aug  8 05:40:05 2010
@@ -20,53 +20,56 @@
 package org.apache.hadoop.hbase.zookeeper;
 
 import java.io.ByteArrayInputStream;
+import java.io.IOException;
 import java.io.InputStream;
 import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
+import org.junit.Before;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertEquals;
+import static org.junit.Assert.*;
 
 /**
  * Test for HQuorumPeer.
  */
-public class TestHQuorumPeer extends HBaseTestCase {
+public class TestHQuorumPeer {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private Path dataDir;
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    String userName = System.getProperty("user.name");
-    dataDir = new Path("/tmp/hbase-" + userName, "zookeeper");
-    if (fs.exists(dataDir)) {
-      if (!fs.isDirectory(dataDir)) {
-        fail();
-      }
-    } else {
-      if (!fs.mkdirs(dataDir)) {
-        fail();
+
+  @Before public void setup() throws IOException {
+    // Set it to a non-standard port.
+    TEST_UTIL.getConfiguration().setInt("hbase.zookeeper.property.clientPort",
+      21810);
+    this.dataDir = HBaseTestingUtility.getTestDir(this.getClass().getName());
+    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
+    if (fs.exists(this.dataDir)) {
+      if (!fs.delete(this.dataDir, true)) {
+        throw new IOException("Failed cleanup of " + this.dataDir);
       }
     }
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    if (fs.exists(dataDir) && !fs.delete(dataDir, true)) {
-      fail();
+    if (!fs.mkdirs(this.dataDir)) {
+      throw new IOException("Failed create of " + this.dataDir);
     }
-    super.tearDown();
   }
 
-  /** */
-  public void testMakeZKProps() {
+  @Test public void testMakeZKProps() {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    conf.set("hbase.zookeeper.property.dataDir", this.dataDir.toString());
     Properties properties = ZKConfig.makeZKProps(conf);
-    assertEquals(dataDir.toString(), properties.get("dataDir"));
-    assertEquals(Integer.valueOf(21810), Integer.valueOf(properties.getProperty("clientPort")));
+    assertEquals(dataDir.toString(), (String)properties.get("dataDir"));
+    assertEquals(Integer.valueOf(21810),
+      Integer.valueOf(properties.getProperty("clientPort")));
     assertEquals("localhost:2888:3888", properties.get("server.0"));
     assertEquals(null, properties.get("server.1"));
 
@@ -82,25 +85,26 @@ public class TestHQuorumPeer extends HBa
     conf.set(HConstants.ZOOKEEPER_QUORUM, oldValue);
   }
 
-  /** @throws Exception */
-  public void testConfigInjection() throws Exception {
+  @Test public void testConfigInjection() throws Exception {
     String s =
-      "dataDir=${hbase.tmp.dir}/zookeeper\n" +
+      "dataDir=" + this.dataDir.toString() + "\n" +
       "clientPort=2181\n" +
       "server.0=${hbase.master.hostname}:2888:3888\n";
 
     System.setProperty("hbase.master.hostname", "localhost");
     InputStream is = new ByteArrayInputStream(s.getBytes());
+    Configuration conf = TEST_UTIL.getConfiguration();
     Properties properties = ZKConfig.parseZooCfg(conf, is);
 
-    assertEquals(dataDir.toString(), properties.get("dataDir"));
-    assertEquals(Integer.valueOf(2181), Integer.valueOf(properties.getProperty("clientPort")));
+    assertEquals(this.dataDir.toString(), properties.get("dataDir"));
+    assertEquals(Integer.valueOf(2181),
+      Integer.valueOf(properties.getProperty("clientPort")));
     assertEquals("localhost:2888:3888", properties.get("server.0"));
 
     QuorumPeerConfig config = new QuorumPeerConfig();
     config.parseProperties(properties);
 
-    assertEquals(dataDir.toString(), config.getDataDir());
+    assertEquals(this.dataDir.toString(), config.getDataDir());
     assertEquals(2181, config.getClientPortAddress().getPort());
     Map<Long,QuorumServer> servers = config.getServers();
     assertEquals(1, servers.size());
@@ -124,11 +128,11 @@ public class TestHQuorumPeer extends HBa
   /**
    * Test Case for HBASE-2305
    */
-  public void testShouldAssignDefaultZookeeperClientPort() {
+  @Test public void testShouldAssignDefaultZookeeperClientPort() {
     Configuration config = HBaseConfiguration.create();
     config.clear();
     Properties p = ZKConfig.makeZKProps(config);
     assertNotNull(p);
     assertEquals(2181, p.get("hbase.zookeeper.property.clientPort"));
   }
-}
+}
\ No newline at end of file