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/20 09:37:35 UTC

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

Author: jimk
Date: Thu Sep 20 00:37:34 2007
New Revision: 577603

URL: http://svn.apache.org/viewvc?rev=577603&view=rev
Log:
HADOOP-1923, HADOOP-1924 a) tests fail sporadically because set up and tear down is inconsistent b) TestDFSAbort failed in nightly #242

Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
    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/HStore.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/util/FSUtils.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java
    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/TestBatchUpdate.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestDFSAbort.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTableMapReduce.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.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=577603&r1=577602&r2=577603&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Thu Sep 20 00:37:34 2007
@@ -52,6 +52,8 @@
                 flush to disk.
     HADOOP-1920 Wrapper scripts broken when hadoop in one location and hbase in
                 another
+    HADOOP-1923, HADOOP-1924 a) tests fail sporadically because set up and tear
+                 down is inconsistent b) TestDFSAbort failed in nightly #242
 
   IMPROVEMENTS
     HADOOP-1737 Make HColumnDescriptor data publically members settable

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=577603&r1=577602&r2=577603&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 Thu Sep 20 00:37:34 2007
@@ -22,7 +22,6 @@
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.lang.reflect.Constructor;
-import java.net.ConnectException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -73,6 +72,7 @@
 HMasterRegionInterface {
   static final Log LOG = LogFactory.getLog(HMaster.class.getName());
 
+  /** {@inheritDoc} */
   public long getProtocolVersion(String protocol,
       @SuppressWarnings("unused") long clientVersion)
   throws IOException {
@@ -655,12 +655,7 @@
               // We ran out of tries. Make sure the file system is still
               // available
               if (checkFileSystem()) {
-                // If filesystem is OK, is the exception a ConnectionException?
-                // If so, mark the server as down.  No point scanning either
-                // if no server to put meta region on. TODO.
-                if (e instanceof ConnectException) {
-                  LOG.debug("Region hosting server is gone.");
-                }
+                continue;                       // avoid sleeping
               }
             }
           }
@@ -962,6 +957,7 @@
   }
 
   /** Main processing loop */
+  @Override
   public void run() {
     final String threadName = "HMaster";
     Thread.currentThread().setName(threadName);
@@ -1201,11 +1197,17 @@
       // Note that cancelling the server's lease takes care of updating
       // serversToServerInfo, etc.
 
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Region server " + serverName +
+            ": MSG_REPORT_EXITING -- cancelling lease");
+      }
+      
       if (cancelLease(serverName, serverLabel)) {
         // Only process the exit message if the server still has a lease.
         // Otherwise we could end up processing the server exit twice.
 
-        LOG.info("Region server " + serverName + ": MSG_REPORT_EXITING");
+        LOG.info("Region server " + serverName +
+            ": MSG_REPORT_EXITING -- lease cancelled");
 
         // Get all the regions the server was serving reassigned
         // (if we are not shutting down).
@@ -1244,8 +1246,8 @@
     synchronized (serversToServerInfo) {
       storedInfo = serversToServerInfo.get(serverName);
     }
-    if(storedInfo == null) {
-      if(LOG.isDebugEnabled()) {
+    if (storedInfo == null) {
+      if (LOG.isDebugEnabled()) {
         LOG.debug("received server report from unknown server: " + serverName);
       }
 
@@ -2564,7 +2566,7 @@
                     server.close(scannerId);
                   } catch (IOException e) {
                     e = RemoteExceptionHandler.checkIOException(e);
-                    LOG.error("", e);
+                    LOG.error("closing scanner", e);
                   }
                 }
                 scannerId = -1L;

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=577603&r1=577602&r2=577603&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 Thu Sep 20 00:37:34 2007
@@ -67,6 +67,7 @@
 public class HRegionServer implements HConstants, HRegionInterface, Runnable {
   static final Log LOG = LogFactory.getLog(HRegionServer.class);
   
+  /** {@inheritDoc} */
   public long getProtocolVersion(final String protocol, 
       @SuppressWarnings("unused") final long clientVersion)
   throws IOException {  
@@ -141,11 +142,15 @@
     private HTable root = null;
     private HTable meta = null;
 
+    /**
+     * @param stop
+     */
     public SplitOrCompactChecker(final AtomicBoolean stop) {
       super(conf.getInt("hbase.regionserver.thread.splitcompactcheckfrequency",
         30 * 1000), stop);
     }
 
+    /** {@inheritDoc} */
     public void closing(final Text regionName) {
       lock.writeLock().lock();
       try {
@@ -161,6 +166,7 @@
       }
     }
     
+    /** {@inheritDoc} */
     public void closed(final Text regionName) {
       lock.writeLock().lock();
       try {
@@ -176,6 +182,7 @@
     /**
      * Scan for splits or compactions to run.  Run any we find.
      */
+    @Override
     protected void chore() {
       // Don't interrupt us while we're working
       synchronized (splitOrCompactLock) {
@@ -275,10 +282,16 @@
   /* Runs periodically to flush memcache.
    */
   class Flusher extends Chore {
+    /**
+     * @param period
+     * @param stop
+     */
     public Flusher(final int period, final AtomicBoolean stop) {
       super(period, stop);
     }
     
+    /** {@inheritDoc} */
+    @Override
     protected void chore() {
       synchronized(cacheFlusherLock) {
         checkForFlushesToRun();
@@ -323,10 +336,16 @@
     private int MAXLOGENTRIES =
       conf.getInt("hbase.regionserver.maxlogentries", 30 * 1000);
     
+    /**
+     * @param period
+     * @param stop
+     */
     public LogRoller(final int period, final AtomicBoolean stop) {
       super(period, stop);
     }
  
+    /** {@inheritDoc} */
+    @Override
     protected void chore() {
       synchronized(logRollerLock) {
         checkForLogRoll();
@@ -592,7 +611,7 @@
             } catch (IOException e) {
               e = RemoteExceptionHandler.checkIOException(e);
               if(tries < this.numRetries) {
-                LOG.warn("", e);
+                LOG.warn("Processing message (Retry: " + tries + ")", e);
                 tries++;
               } else {
                 LOG.error("Exceeded max retries: " + this.numRetries, e);
@@ -646,7 +665,8 @@
       try {
         log.closeAndDelete();
       } catch (IOException e) {
-        LOG.error("", RemoteExceptionHandler.checkIOException(e));
+        LOG.error("Close and delete failed",
+            RemoteExceptionHandler.checkIOException(e));
       }
       try {
         if (!masterRequestedStop && closedRegions != null) {
@@ -664,7 +684,8 @@
           hbaseMaster.regionServerReport(serverInfo, exitMsg);
         }
       } catch (IOException e) {
-        LOG.warn("", RemoteExceptionHandler.checkIOException(e));
+        LOG.warn("Failed to send exiting message to master: ",
+            RemoteExceptionHandler.checkIOException(e));
       }
       LOG.info("stopping server at: " +
         serverInfo.getServerAddress().toString());
@@ -799,6 +820,7 @@
       }
     }
     
+    /** {@inheritDoc} */
     public void run() {
       try {
         for(ToDoEntry e = null; !stopRequested.get(); ) {
@@ -1101,7 +1123,8 @@
       leases.createLease(scannerId, scannerId, new ScannerListener(scannerName));
       return scannerId;
     } catch (IOException e) {
-      LOG.error("", RemoteExceptionHandler.checkIOException(e));
+      LOG.error("Opening scanner (fsOk: " + this.fsOk + ")",
+          RemoteExceptionHandler.checkIOException(e));
       checkFileSystem();
       throw e;
     }
@@ -1243,14 +1266,14 @@
       this.lock.readLock().unlock();
     }
   }
-  
+
   /**
    * Checks to see if the file system is still accessible.
    * If not, sets abortRequested and stopRequested
    * 
    * @return false if file system is not available
    */
-  protected synchronized boolean checkFileSystem() {
+  protected boolean checkFileSystem() {
     if (this.fsOk) {
       if (!FSUtils.isFileSystemAvailable(fs)) {
         LOG.fatal("Shutting down HRegionServer: file system not available");

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java?rev=577603&r1=577602&r2=577603&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java Thu Sep 20 00:37:34 2007
@@ -1336,7 +1336,7 @@
         midKey.set(((HStoreKey)midkey).getRow());
       }
     } catch(IOException e) {
-      LOG.warn("", e);
+      LOG.warn("Failed getting store size", e);
     } finally {
       this.lock.releaseReadLock();
     }

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=577603&r1=577602&r2=577603&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 Thu Sep 20 00:37:34 2007
@@ -24,6 +24,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.dfs.DistributedFileSystem;
 
 /**
@@ -32,7 +33,10 @@
 public class FSUtils {
   private static final Log LOG = LogFactory.getLog(FSUtils.class);
 
-  private FSUtils() {}                                  // not instantiable
+  /**
+   * Not instantiable
+   */
+  private FSUtils() {}
   
   /**
    * Checks to see if the specified file system is available
@@ -41,31 +45,37 @@
    * @return true if the specified file system is available.
    */
   public static boolean isFileSystemAvailable(FileSystem fs) {
+    if (!(fs instanceof DistributedFileSystem)) {
+      return true;
+    }
     boolean available = false;
-    if (fs instanceof DistributedFileSystem) {
+    DistributedFileSystem dfs = (DistributedFileSystem) fs;
+    int maxTries = dfs.getConf().getInt("hbase.client.retries.number", 3);
+    Path root = new Path(dfs.getConf().get("hbase.dir", "/"));
+    for (int i = 0; i < maxTries; i++) {
+      IOException ex = null;
       try {
-        if (((DistributedFileSystem) fs).getDataNodeStats().length > 0) {
+        if (dfs.exists(root)) {
           available = true;
-          
-        } else {
-          LOG.fatal("file system unavailable: no data nodes");
+          break;
         }
-        
       } catch (IOException e) {
-        LOG.fatal("file system unavailable because: ", e);
+        ex = e;
       }
-
-      try {
-        if (!available) {
-          fs.close();
-        }
-        
-      } catch (IOException e) {
-        LOG.error("file system close", e);
+      String exception = "";
+      if (ex != null) {
+        exception = ": " + ex.getMessage();
       }
-      
-    } else {
-      available = true;
+      LOG.info("Failed exists test on " + root + " (Attempt " + i + ")" +
+          exception);
+    }
+    try {
+      if (!available) {
+        fs.close();
+      }
+        
+    } catch (IOException e) {
+        LOG.error("file system close failed: ", e);
     }
     return available;
   }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java?rev=577603&r1=577602&r2=577603&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java Thu Sep 20 00:37:34 2007
@@ -119,6 +119,9 @@
     if (dfsCluster != null) {
       dfsCluster.shutdown();
     }
+    if (fs != null) {
+      fs.close();
+    }
   }
 
   private HRegion createAregion(Text startKey, Text endKey, int firstRow,

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=577603&r1=577602&r2=577603&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 Thu Sep 20 00:37:34 2007
@@ -84,6 +84,9 @@
   /**
    * 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.
+   * 
    * @param conf
    * @param nRegionNodes
    * @param dfsCluster
@@ -93,7 +96,8 @@
       MiniDFSCluster dfsCluster) throws IOException {
 
     this.conf = conf;
-    this.cluster = dfsCluster;
+    this.fs = dfsCluster.getFileSystem();
+    this.cluster = null;
     init(nRegionNodes);
   }
 
@@ -116,13 +120,16 @@
     this.deleteOnExit = deleteOnExit;
     if (miniHdfsFilesystem) {
       this.cluster = new MiniDFSCluster(this.conf, 2, format, (String[])null);
+      this.fs = cluster.getFileSystem();
+    } else {
+      this.cluster = null;
+      this.fs = FileSystem.get(conf);
     }
     init(nRegionNodes);
   }
 
   private void init(final int nRegionNodes) throws IOException {
     try {
-      this.fs = FileSystem.get(conf);
       this.parentdir = new Path(conf.get(HBASE_DIR, DEFAULT_HBASE_DIR));
       fs.mkdirs(parentdir);
       this.masterThread = startMaster(this.conf);

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBatchUpdate.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBatchUpdate.java?rev=577603&r1=577602&r2=577603&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBatchUpdate.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestBatchUpdate.java Thu Sep 20 00:37:34 2007
@@ -40,6 +40,7 @@
    * @throws UnsupportedEncodingException
    */
   public TestBatchUpdate() throws UnsupportedEncodingException {
+    super();
     value = "abcd".getBytes(HConstants.UTF8_ENCODING);
   }
   

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=577603&r1=577602&r2=577603&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 Thu Sep 20 00:37:34 2007
@@ -60,7 +60,10 @@
     cluster.join();
   }
   
-  public static void main(String[] args) {
+  /**
+   * @param args unused
+   */
+  public static void main(@SuppressWarnings("unused") String[] args) {
     TestRunner.run(new TestSuite(TestDFSAbort.class));
   }
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java?rev=577603&r1=577602&r2=577603&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java Thu Sep 20 00:37:34 2007
@@ -31,7 +31,6 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Text;
@@ -177,7 +176,7 @@
     HBaseAdmin admin = new HBaseAdmin(conf);
     Text tableName = new Text(getName());
     admin.createTable(new HTableDescriptor(tableName.toString()));
-    List<HRegionInfo> regions = scan(conf, metaTable);
+    List<HRegionInfo> regions = scan(metaTable);
     assertEquals("Expected one region", regions.size(), 1);
     HRegionInfo region = regions.get(0);
     assertTrue("Expected region named for test",
@@ -197,10 +196,10 @@
         homedir, this.conf, null));
     try {
       for (HRegion r : newRegions) {
-        addRegionToMETA(conf, metaTable, r, this.cluster.getHMasterAddress(),
+        addRegionToMETA(metaTable, r, this.cluster.getHMasterAddress(),
           -1L);
       }
-      regions = scan(conf, metaTable);
+      regions = scan(metaTable);
       assertEquals("Should be two regions only", 2, regions.size());
     } finally {
       for (HRegion r : newRegions) {
@@ -210,7 +209,7 @@
     }
   }
   
-  private List<HRegionInfo> scan(final Configuration conf, final HTable t)
+  private List<HRegionInfo> scan(final HTable t)
   throws IOException {
     List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
     HRegionInterface regionServer = null;
@@ -262,8 +261,7 @@
     return regions;
   }
   
-  private void addRegionToMETA(final Configuration conf,
-      final HTable t, final HRegion region,
+  private void addRegionToMETA(final HTable t, final HRegion region,
       final HServerAddress serverAddress,
       final long startCode)
   throws IOException {

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java?rev=577603&r1=577602&r2=577603&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestSplit.java Thu Sep 20 00:37:34 2007
@@ -32,7 +32,8 @@
  * {@Link TestHRegion} does a split but this TestCase adds testing of fast
  * split and manufactures odd-ball split scenarios.
  */
-public class TestSplit extends HBaseTestCase {
+public class TestSplit extends MultiRegionTable {
+  @SuppressWarnings("hiding")
   static final Log LOG = LogFactory.getLog(TestSplit.class.getName());
   
   /** constructor */

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTableMapReduce.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTableMapReduce.java?rev=577603&r1=577602&r2=577603&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTableMapReduce.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTableMapReduce.java Thu Sep 20 00:37:34 2007
@@ -45,7 +45,8 @@
 /**
  * Test Map/Reduce job over HBase tables
  */
-public class TestTableMapReduce extends HBaseTestCase {
+public class TestTableMapReduce extends MultiRegionTable {
+  @SuppressWarnings("hiding")
   private static final Log LOG =
     LogFactory.getLog(TestTableMapReduce.class.getName());
   
@@ -115,6 +116,10 @@
     if (dfsCluster != null) {
       dfsCluster.shutdown();
     }
+    
+    if (fs != null) {
+      fs.close();
+    }
   }
 
   /**
@@ -254,7 +259,7 @@
     admin.createTable(desc);
 
     // Populate a table into multiple regions
-    MultiRegionTable.makeMultiRegionTable(conf, hCluster, null,
+    MultiRegionTable.makeMultiRegionTable(conf, hCluster, fs,
         MULTI_REGION_TABLE_NAME, INPUT_COLUMN);
     
     // Verify table indeed has multiple regions

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java?rev=577603&r1=577602&r2=577603&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java Thu Sep 20 00:37:34 2007
@@ -43,10 +43,12 @@
   private ByteArrayOutputStream baos;
   private HBaseAdmin admin;
   
+  /** constructor */
   public TestHBaseShell() {
     super(1 /*One region server only*/);
   }
   
+  /** {@inheritDoc} */
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -100,6 +102,9 @@
         sglQuotedColumnFamily + "');", tmpTableName, sglQuotedColumnFamily);
   }
 
+  /**
+   * @throws Exception
+   */
   public void testInsertSelectDelete() throws Exception {
     final String tableName = getName();
     final String columnFamily = tableName;