You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2007/09/30 18:09:39 UTC

svn commit: r580745 - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/util/ src/test/ src/test/org/apache/hadoop/hbase/

Author: jimk
Date: Sun Sep 30 09:09:38 2007
New Revision: 580745

URL: http://svn.apache.org/viewvc?rev=580745&view=rev
Log:
HADOOP-1966 Make HBase unit tests more reliable in the Hudson environment.

Set hbase.root in test/hbase-site.xml; when running a test, the default does not work consistantly.

When a HBase mini cluster is started on top of an existing mini dfs cluster, it should not shut down the mini dfs cluster when the mini HBase cluster is shut down.

TestDFSAbort catches exceptions, prints the stack trace and re-throws the exception, so you can see when the exception happened in the log.

Catch runtime exceptions that were escaping from FSUtils.isFileSystemAvailable, enabling more reliable detection of dfs failure. HRegionServer also now checks to see if it is still accepting client requests.


Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/FSUtils.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/hbase-site.xml
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDFSAbort.java

Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?rev=580745&r1=580744&r2=580745&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Sun Sep 30 09:09:38 2007
@@ -62,6 +62,7 @@
                  down is inconsistent b) TestDFSAbort failed in nightly #242
     HADOOP-1929 Add hbase-default.xml to hbase jar
     HADOOP-1941 StopRowFilter throws NPE when passed null row
+    HADOOP-1966 Make HBase unit tests more reliable in the Hudson environment.
 
   IMPROVEMENTS
     HADOOP-1737 Make HColumnDescriptor data publically members settable

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java?rev=580745&r1=580744&r2=580745&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConstants.java Sun Sep 30 09:09:38 2007
@@ -71,9 +71,6 @@
   /** Used to construct the name of the directory in which a HRegion resides */
   static final String HREGIONDIR_PREFIX = "hregion_";
   
-  // TODO: Someone may try to name a column family 'log'.  If they
-  // do, it will clash with the HREGION log dir subdirectory. FIX.
-  
   /** Used to construct the name of the log directory for a region server */
   static final String HREGION_LOGDIR_NAME = "log";
 

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java?rev=580745&r1=580744&r2=580745&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMaster.java Sun Sep 30 09:09:38 2007
@@ -504,7 +504,7 @@
             LOG.error("Scan ROOT region", e);
             if (tries == numRetries - 1) {
               // We ran out of tries. Make sure the file system is still available
-              if (checkFileSystem()) {
+              if (!checkFileSystem()) {
                 continue; // Avoid sleeping.
               }
             }
@@ -654,7 +654,7 @@
             if (tries == numRetries - 1) {
               // We ran out of tries. Make sure the file system is still
               // available
-              if (checkFileSystem()) {
+              if (!checkFileSystem()) {
                 continue;                       // avoid sleeping
               }
             }
@@ -941,7 +941,7 @@
    */
   protected boolean checkFileSystem() {
     if (fsOk) {
-      if (!FSUtils.isFileSystemAvailable(fs, closed)) {
+      if (!FSUtils.isFileSystemAvailable(fs)) {
         LOG.fatal("Shutting down HBase cluster: file system not available");
         closed.set(true);
         fsOk = false;

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?rev=580745&r1=580744&r2=580745&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java Sun Sep 30 09:09:38 2007
@@ -139,6 +139,7 @@
         30 * 1000), stop);
     }
 
+    /** {@inheritDoc} */
     public void closing(final Text regionName) {
       lock.writeLock().lock();
       try {
@@ -154,6 +155,7 @@
       }
     }
     
+    /** {@inheritDoc} */
     public void closed(final Text regionName) {
       lock.writeLock().lock();
       try {
@@ -458,9 +460,17 @@
                   // get it when the master is panicing because for instance
                   // the HDFS has been yanked out from under it.  Be wary of
                   // this message.
-                  if (checkFileSystem()) {
-                    closeAllRegions();
-                    restart = true;
+                  try {
+                    if (checkFileSystem()) {
+                      closeAllRegions();
+                      restart = true;
+                    }
+                  } catch (Exception e) {
+                    LOG.fatal("file system available check failed. " +
+                        "Shutting down server.", e);
+                    this.stopRequested.set(true);
+                    this.fsOk = false;
+                    this.abortRequested = true;
                   }
                   
                   break;
@@ -944,7 +954,8 @@
   /** {@inheritDoc} */
   public byte [] get(final Text regionName, final Text row,
       final Text column) throws IOException {
-    
+
+    checkOpen();
     requestCount.incrementAndGet();
     try {
       return getRegion(regionName).get(row, column);
@@ -958,7 +969,8 @@
   /** {@inheritDoc} */
   public byte [][] get(final Text regionName, final Text row,
       final Text column, final int numVersions) throws IOException {
-    
+
+    checkOpen();
     requestCount.incrementAndGet();
     try {
       return getRegion(regionName).get(row, column, numVersions);
@@ -972,7 +984,8 @@
   /** {@inheritDoc} */
   public byte [][] get(final Text regionName, final Text row, final Text column, 
       final long timestamp, final int numVersions) throws IOException {
-    
+
+    checkOpen();
     requestCount.incrementAndGet();
     try {
       return getRegion(regionName).get(row, column, timestamp, numVersions);
@@ -986,7 +999,8 @@
   /** {@inheritDoc} */
   public MapWritable getRow(final Text regionName, final Text row)
     throws IOException {
-    
+
+    checkOpen();
     requestCount.incrementAndGet();
     try {
       HRegion region = getRegion(regionName);
@@ -1006,7 +1020,8 @@
 
   /** {@inheritDoc} */
   public MapWritable next(final long scannerId) throws IOException {
-    
+
+    checkOpen();
     requestCount.incrementAndGet();
     try {
       String scannerName = String.valueOf(scannerId);
@@ -1044,7 +1059,9 @@
 
   /** {@inheritDoc} */
   public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
-  throws IOException {  
+    throws IOException {
+
+    checkOpen();
     requestCount.incrementAndGet();
     // If timestamp == LATEST_TIMESTAMP and we have deletes, then they need
     // special treatment.  For these we need to first find the latest cell so
@@ -1093,9 +1110,12 @@
   // remote scanner interface
   //
 
+  /** {@inheritDoc} */
   public long openScanner(Text regionName, Text[] cols, Text firstRow,
-    final long timestamp, final RowFilterInterface filter)
-  throws IOException {
+      final long timestamp, final RowFilterInterface filter)
+    throws IOException {
+
+    checkOpen();
     requestCount.incrementAndGet();
     try {
       HRegion r = getRegion(regionName);
@@ -1110,7 +1130,7 @@
       leases.createLease(scannerId, scannerId, new ScannerListener(scannerName));
       return scannerId;
     } catch (IOException e) {
-      LOG.error("Opening scanner (fsOk: " + this.fsOk + ")",
+      LOG.error("Error opening scanner (fsOk: " + this.fsOk + ")",
           RemoteExceptionHandler.checkIOException(e));
       checkFileSystem();
       throw e;
@@ -1119,6 +1139,7 @@
   
   /** {@inheritDoc} */
   public void close(final long scannerId) throws IOException {
+    checkOpen();
     requestCount.incrementAndGet();
     try {
       String scannerName = String.valueOf(scannerId);
@@ -1255,6 +1276,20 @@
   }
 
   /**
+   * Called to verify that this server is up and running.
+   * 
+   * @throws IOException
+   */
+  private void checkOpen() throws IOException {
+    if (stopRequested.get() || abortRequested) {
+      throw new IOException("Server not running");
+    }
+    if (!fsOk) {
+      throw new IOException("File system not available");
+    }
+  }
+  
+  /**
    * Checks to see if the file system is still accessible.
    * If not, sets abortRequested and stopRequested
    * 
@@ -1265,10 +1300,14 @@
       FileSystem fs = null;
       try {
         fs = FileSystem.get(this.conf);
-      } catch (IOException e) {
+        if (fs != null && !FSUtils.isFileSystemAvailable(fs)) {
+          LOG.fatal("Shutting down HRegionServer: file system not available");
+          this.abortRequested = true;
+          this.stopRequested.set(true);
+          fsOk = false;
+        }
+      } catch (Exception e) {
         LOG.error("Failed get of filesystem", e);
-      }
-      if (fs != null && !FSUtils.isFileSystemAvailable(fs, stopRequested)) {
         LOG.fatal("Shutting down HRegionServer: file system not available");
         this.abortRequested = true;
         this.stopRequested.set(true);
@@ -1301,6 +1340,7 @@
     return regionsToCheck;
   }
 
+  /** {@inheritDoc} */
   public long getProtocolVersion(final String protocol, 
       @SuppressWarnings("unused") final long clientVersion)
   throws IOException {  

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/FSUtils.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/FSUtils.java?rev=580745&r1=580744&r2=580745&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/FSUtils.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/FSUtils.java Sun Sep 30 09:09:38 2007
@@ -26,7 +26,6 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.dfs.DistributedFileSystem;
 
 /**
@@ -38,48 +37,37 @@
   /**
    * Not instantiable
    */
-  private FSUtils() {super();}
+  private FSUtils() {}
   
   /**
    * Checks to see if the specified file system is available
    * 
    * @param fs
-   * @param closed Optional flag.  If non-null and set, will abort test of
-   * filesytem.  Presumption is a flag shared by multiple threads.  Another
-   * may have already determined the filesystem -- or something else -- bad.
    * @return true if the specified file system is available.
    */
-  public static boolean isFileSystemAvailable(final FileSystem fs,
-      final AtomicBoolean closed) {
+  public static boolean isFileSystemAvailable(final FileSystem fs) {
     if (!(fs instanceof DistributedFileSystem)) {
       return true;
     }
+    String exception = "";
     boolean available = false;
     DistributedFileSystem dfs = (DistributedFileSystem) fs;
-    int maxTries = dfs.getConf().getInt("hbase.client.retries.number", 3);
-    Path root =
-      fs.makeQualified(new Path(dfs.getConf().get(HConstants.HBASE_DIR, "/")));
-    for (int i = 0; i < maxTries && (closed == null || !closed.get()); i++) {
-      IOException ex = null;
-      try {
-        if (dfs.exists(root)) {
-          available = true;
-          break;
-        }
-      } catch (IOException e) {
-        ex = e;
+    try {
+      if (dfs.exists(new Path("/"))) {
+        available = true;
       }
-      String exception = (ex == null)? "": ": " + ex.getMessage();
-      LOG.info("Failed exists test on " + root + " by thread " +
-        Thread.currentThread().getName() + " (Attempt " + i + " of " +
-        maxTries  +"): " + exception);
+    } catch (IOException e) {
+      exception = e.getMessage();
     }
+    LOG.info("Failed file system available test. Thread: " +
+        Thread.currentThread().getName() + ": " + exception);
+    
     try {
       if (!available) {
         fs.close();
       }
         
-    } catch (IOException e) {
+    } catch (Exception e) {
         LOG.error("file system close failed: ", e);
     }
     return available;

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/hbase-site.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/hbase-site.xml?rev=580745&r1=580744&r2=580745&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/hbase-site.xml (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/hbase-site.xml Sun Sep 30 09:09:38 2007
@@ -75,4 +75,8 @@
     the master will notice a dead region server sooner. The default is 15 seconds.
     </description>
   </property>
+  <property>
+  	<name>hbase.rootdir</name>
+  	<value>/hbase</value>
+  	<description>location of HBase instance in dfs</description></property>
 </configuration>

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java?rev=580745&r1=580744&r2=580745&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java Sun Sep 30 09:09:38 2007
@@ -33,14 +33,14 @@
 /**
  * This class creates a single process HBase cluster for junit testing.
  * One thread is created for each server.
- * 
+ *
  * <p>TestCases do not need to subclass to start a HBaseCluster.  Call
  * {@link #startMaster(Configuration)} and
  * {@link #startRegionServers(Configuration, int)} to startup master and
  * region servers.  Save off the returned values and pass them to
  * {@link #shutdown(org.apache.hadoop.hbase.MiniHBaseCluster.MasterThread, List)}
  * to shut it all down when done.
- * 
+ *
  */
 public class MiniHBaseCluster implements HConstants {
   static final Logger LOG =
@@ -48,6 +48,7 @@
   private Configuration conf;
   private MiniDFSCluster cluster;
   private FileSystem fs;
+  private boolean shutdownDFS;
   private Path parentdir;
   private MasterThread masterThread = null;
   ArrayList<RegionServerThread> regionThreads =
@@ -56,21 +57,21 @@
 
   /**
    * Starts a MiniHBaseCluster on top of a new MiniDFSCluster
-   * 
+   *
    * @param conf
    * @param nRegionNodes
-   * @throws IOException 
+   * @throws IOException
    */
   public MiniHBaseCluster(Configuration conf, int nRegionNodes)
     throws IOException {
-    
+
     this(conf, nRegionNodes, true, true, true);
   }
 
   /**
    * Start a MiniHBaseCluster. Use the native file system unless
    * miniHdfsFilesystem is set to true.
-   * 
+   *
    * @param conf
    * @param nRegionNodes
    * @param miniHdfsFilesystem
@@ -83,14 +84,20 @@
 
   /**
    * Starts a MiniHBaseCluster on top of an existing HDFSCluster
-   * 
-   * Note that if you use this constructor, you should shut down the mini dfs
-   * cluster in your test case.
-   * 
+   *
+   ****************************************************************************
+   *            *  *  *  *  *  N O T E  *  *  *  *  *
+   *
+   * If you use this constructor, you should shut down the mini dfs cluster
+   * in your test case.
+   *
+   *            *  *  *  *  *  N O T E  *  *  *  *  *
+   ****************************************************************************
+   *
    * @param conf
    * @param nRegionNodes
    * @param dfsCluster
-   * @throws IOException 
+   * @throws IOException
    */
   public MiniHBaseCluster(Configuration conf, int nRegionNodes,
       MiniDFSCluster dfsCluster) throws IOException {
@@ -98,6 +105,7 @@
     this.conf = conf;
     this.fs = dfsCluster.getFileSystem();
     this.cluster = dfsCluster;
+    this.shutdownDFS = false;
     init(nRegionNodes);
   }
 
@@ -110,17 +118,19 @@
    * filesystem configured in <code>conf</code>.
    * @param format the mini hdfs cluster
    * @param deleteOnExit clean up mini hdfs files
-   * @throws IOException 
+   * @throws IOException
    */
   public MiniHBaseCluster(Configuration conf, int nRegionNodes,
-      final boolean miniHdfsFilesystem, boolean format, boolean deleteOnExit) 
+      final boolean miniHdfsFilesystem, boolean format, boolean deleteOnExit)
     throws IOException {
-    
+
     this.conf = conf;
     this.deleteOnExit = deleteOnExit;
+    this.shutdownDFS = false;
     if (miniHdfsFilesystem) {
       this.cluster = new MiniDFSCluster(this.conf, 2, format, (String[])null);
       this.fs = cluster.getFileSystem();
+      this.shutdownDFS = true;
     } else {
       this.cluster = null;
       this.fs = FileSystem.get(conf);
@@ -139,7 +149,7 @@
       throw e;
     }
   }
-  
+
   /** runs the master server */
   public static class MasterThread extends Thread {
     private final HMaster master;
@@ -147,20 +157,20 @@
       super(m, "Master:" + m.getMasterAddress().toString());
       this.master = m;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public void run() {
       LOG.info("Starting " + getName());
       super.run();
     }
-    
+
     /** @return master server */
     public HMaster getMaster() {
       return this.master;
     }
   }
-  
+
   /** runs region servers */
   public static class RegionServerThread extends Thread {
     private final HRegionServer regionServer;
@@ -168,20 +178,20 @@
       super(r, "RegionServer:" + index);
       this.regionServer = r;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public void run() {
       LOG.info("Starting " + getName());
       super.run();
     }
-    
+
     /** @return the region server */
     public HRegionServer getRegionServer() {
       return this.regionServer;
     }
   }
-  
+
   /**
    * Use this method to start a master.
    * If you want to start an hbase cluster
@@ -197,7 +207,7 @@
    */
   public static MasterThread startMaster(final Configuration c)
     throws IOException {
-    
+
     if(c.get(MASTER_ADDRESS) == null) {
       c.set(MASTER_ADDRESS, "localhost:0");
     }
@@ -221,7 +231,7 @@
    */
   public static ArrayList<RegionServerThread> startRegionServers(
     final Configuration c, final int count) throws IOException {
-    
+
     // Start the HRegionServers.  Always have regionservers come up on
     // port '0' so there won't be clashes over default port as unit tests
     // start/stop ports at different times during the life of the test.
@@ -234,10 +244,10 @@
     }
     return threads;
   }
-  
+
   /**
    * Starts a region server thread running
-   * 
+   *
    * @throws IOException
    * @return Name of regionserver started.
    */
@@ -247,10 +257,10 @@
     this.regionThreads.add(t);
     return t.getName();
   }
-  
+
   private static RegionServerThread startRegionServer(final Configuration c,
     final int index)
-  throws IOException {  
+  throws IOException {
     final HRegionServer hrs = new HRegionServer(c);
     RegionServerThread t = new RegionServerThread(hrs, index);
     t.setName("regionserver" +
@@ -261,14 +271,14 @@
 
   /**
    * Get the cluster on which this HBase cluster is running
-   * 
+   *
    * @return MiniDFSCluster
    */
   public MiniDFSCluster getDFSCluster() {
     return cluster;
   }
 
-  /** 
+  /**
    * @return Returns the rpc address actually used by the master server, because
    * the supplied port is not necessarily the actual port used.
    */
@@ -278,7 +288,7 @@
 
   /**
    * Cause a region server to exit without cleaning up
-   * 
+   *
    * @param serverNumber
    */
   public void abortRegionServer(int serverNumber) {
@@ -290,7 +300,7 @@
 
   /**
    * Shut down the specified region server cleanly
-   * 
+   *
    * @param serverNumber
    * @return the region server that was stopped
    */
@@ -320,7 +330,7 @@
     }
     return regionServerThread.getName();
   }
-  
+
   /**
    * Wait for Mini HBase Cluster to shut down.
    */
@@ -346,7 +356,7 @@
       }
     }
   }
-  
+
   /**
    * Shut down HBase cluster started by calling
    * {@link #startMaster(Configuration)} and then
@@ -389,14 +399,17 @@
       ((masterThread != null)? masterThread.getName(): "0 masters") + " " +
       regionServerThreads.size() + " region server(s)");
   }
-  
+
+  /**
+   * Shut down the mini HBase cluster
+   */
   public void shutdown() {
     MiniHBaseCluster.shutdown(this.masterThread, this.regionThreads);
-    
+
     try {
-      if (cluster != null) {
+      if (shutdownDFS && cluster != null) {
         FileSystem fs = cluster.getFileSystem();
-        
+
         LOG.info("Shutting down Mini DFS cluster");
         cluster.shutdown();
 
@@ -405,10 +418,10 @@
           fs.close();
         }
       }
-      
+
     } catch (IOException e) {
       LOG.error("shutdown", e);
-      
+
     } finally {
       // Delete all DFS files
       if(deleteOnExit) {
@@ -428,7 +441,7 @@
     }
     f.delete();
   }
-  
+
   /**
    * Call flushCache on all regions on all participating regionservers.
    * @throws IOException

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDFSAbort.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDFSAbort.java?rev=580745&r1=580744&r2=580745&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDFSAbort.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDFSAbort.java Sun Sep 30 09:09:38 2007
@@ -30,32 +30,35 @@
  */
 public class TestDFSAbort extends HBaseClusterTestCase {
 
-  /** constructor */
-  public TestDFSAbort() {
-    super();
-    Logger.getRootLogger().setLevel(Level.WARN);
-    Logger.getLogger(this.getClass().getPackage().getName()).setLevel(Level.DEBUG);
-  }
-  
   /** {@inheritDoc} */
   @Override
   public void setUp() throws Exception {
-    super.setUp();
-    HTableDescriptor desc = new HTableDescriptor(getName());
-    desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY_STR));
-    HBaseAdmin admin = new HBaseAdmin(conf);
-    admin.createTable(desc);
+    try {
+      super.setUp();
+      HTableDescriptor desc = new HTableDescriptor(getName());
+      desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY_STR));
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      admin.createTable(desc);
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
   }
   
   /**
    * @throws Exception
    */
   public void testDFSAbort() throws Exception {
-    // By now the Mini DFS is running, Mini HBase is running and we have
-    // created a table. Now let's yank the rug out from HBase
-    cluster.getDFSCluster().shutdown();
-    // Now wait for Mini HBase Cluster to shut down
-    cluster.join();
+    try {
+      // By now the Mini DFS is running, Mini HBase is running and we have
+      // created a table. Now let's yank the rug out from HBase
+      cluster.getDFSCluster().shutdown();
+      // Now wait for Mini HBase Cluster to shut down
+      cluster.join();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
   }
   
   /**