Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHRegion.java
 Sat Jan 12 17:36:08 2008
@@ -27,7 +27,6 @@
 import java.util.TreeMap;
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
@@ -39,7 +38,8 @@
  * A lot of the meta information for an HRegion now lives inside other
  * HRegions or in the HBaseMaster, so only basic testing is possible.
  */
-public class TestHRegion extends HBaseTestCase implements 
RegionUnavailableListener {
+public class TestHRegion extends HBaseTestCase
+implements RegionUnavailableListener {
   static final Logger LOG =
     Logger.getLogger(TestHRegion.class.getName());
   
@@ -61,6 +61,12 @@
       read();
       cleanup();
     } finally {
+      if (r != null) {
+        r.close();
+      }
+      if (log != null) {
+        log.closeAndDelete();
+      }
       StaticTestEnvironment.shutdownDfs(cluster);
     }
   }
@@ -78,9 +84,6 @@
   private static final Text ANCHOR_SECONDCOL = new Text("anchor:secondcol");
   
   private MiniDFSCluster cluster = null;
-  private FileSystem fs = null;
-  private Path parentdir = null;
-  private Path newlogdir = null;
   private HLog log = null;
   private HTableDescriptor desc = null;
   HRegion r = null;
@@ -93,17 +96,12 @@
   private void setup() throws IOException {
 
     cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
-    fs = cluster.getFileSystem();
-    parentdir = new Path("/hbase");
-    fs.mkdirs(parentdir);
-    newlogdir = new Path(parentdir, "log");
 
-    log = new HLog(fs, newlogdir, conf, null);
     desc = new HTableDescriptor("test");
     desc.addFamily(new HColumnDescriptor("contents:"));
     desc.addFamily(new HColumnDescriptor("anchor:"));
-    r = new HRegion(parentdir, log, fs, conf, 
-        new HRegionInfo(desc, null, null), null, null);
+    r = createNewHRegion(desc, null, null);
+    log = r.getLog();
     region = new HRegionIncommon(r);
   }
 
@@ -162,7 +160,7 @@
         + ((System.currentTimeMillis() - startTime) / 1000.0));
   }
   
-  private void badPuts() throws IOException {
+  private void badPuts() {
     
     // Try put with bad lockid.
     boolean exceptionThrown = false;
@@ -799,7 +797,10 @@
   
   private void cleanup() {
     try {
+      r.close();
+      r = null;
       log.closeAndDelete();
+      log = null;
     } catch (IOException e) {
       e.printStackTrace();
     }

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHStoreFile.java
 Sat Jan 12 17:36:08 2008
@@ -24,7 +24,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.MapFile;
@@ -39,7 +38,6 @@
   static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
   private static String DIR = "/";
   private MiniDFSCluster cluster;
-  private FileSystem fs;
   private Path dir = null;
   
   /** [EMAIL PROTECTED] */
@@ -47,7 +45,6 @@
   public void setUp() throws Exception {
     try {
       this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
-      this.fs = cluster.getFileSystem();
       this.dir = new Path(DIR, getName());
     } catch (IOException e) {
       StaticTestEnvironment.shutdownDfs(cluster);
@@ -121,8 +118,8 @@
   public void testReference()
   throws IOException {
     // Make a store file and write data to it.
-    HStoreFile hsf = new HStoreFile(this.conf, this.dir, getName(),
-        new Text("colfamily"), 1234567890L);
+    HStoreFile hsf = new HStoreFile(this.conf, this.fs, this.dir, getName(),
+        new Text("colfamily"), 1234567890L, null);
     MapFile.Writer writer =
       hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, null);
     writeStoreFile(writer);
@@ -138,16 +135,16 @@
     HStoreFile.Reference reference =
       new HStoreFile.Reference(hsf.getEncodedRegionName(), hsf.getFileId(),
           midkey, HStoreFile.Range.top);
-    HStoreFile refHsf = new HStoreFile(this.conf, new Path(DIR, getName()),
-        getName() + "_reference", hsf.getColFamily(), 456,
-        reference);
+    HStoreFile refHsf = new HStoreFile(this.conf, this.fs, 
+        new Path(DIR, getName()), getName() + "_reference", hsf.getColFamily(),
+        456, reference);
     // Assert that reference files are written and that we can write and
     // read the info reference file at least.
     refHsf.writeReferenceFiles(this.fs);
     assertTrue(this.fs.exists(refHsf.getMapFilePath()));
     assertTrue(this.fs.exists(refHsf.getInfoFilePath()));
     HStoreFile.Reference otherReference =
-      HStoreFile.readSplitInfo(refHsf.getInfoFilePath(), this.fs);
+      HStore.readSplitInfo(refHsf.getInfoFilePath(), this.fs);
     assertEquals(reference.getEncodedRegionName(),
         otherReference.getEncodedRegionName());
     assertEquals(reference.getFileId(),

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestInfoServers.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestInfoServers.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestInfoServers.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestInfoServers.java
 Sat Jan 12 17:36:08 2008
@@ -33,15 +33,20 @@
  */
 public class TestInfoServers extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestInfoServers.class);
-  
+
+  @Override  
   protected void setUp() throws Exception {
     super.setUp();
   }
 
+  @Override
   protected void tearDown() throws Exception {
     super.tearDown();
   }
   
+  /**
+   * @throws Exception
+   */
   public void testInfoServersAreUp() throws Exception {
     // Bring up info servers on 'odd' port numbers in case the test is not
     // sourcing the src/test/hbase-default.xml.

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestLogRolling.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestLogRolling.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestLogRolling.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestLogRolling.java
 Sat Jan 12 17:36:08 2008
@@ -118,7 +118,7 @@
   }
   
   private void startAndWriteData() throws Exception {
-    cluster = new MiniHBaseCluster(conf, 1, dfs);
+    cluster = new MiniHBaseCluster(conf, 1, dfs, true);
     try {
       Thread.sleep(10 * 1000);                  // Wait for region server to 
start
     } catch (InterruptedException e) {

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeMeta.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeMeta.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeMeta.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeMeta.java
 Sat Jan 12 17:36:08 2008
@@ -37,6 +37,6 @@
    */
   public void testMergeMeta() throws IOException {
     assertNotNull(dfsCluster);
-    HMerge.merge(conf, fs, HConstants.META_TABLE_NAME);
+    HMerge.merge(conf, dfsCluster.getFileSystem(), HConstants.META_TABLE_NAME);
   }
 }  

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeTable.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeTable.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeTable.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeTable.java
 Sat Jan 12 17:36:08 2008
@@ -32,9 +32,9 @@
    */
   public void testMergeTable() throws IOException {
     assertNotNull(dfsCluster);
-    MiniHBaseCluster hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
+    MiniHBaseCluster hCluster = new MiniHBaseCluster(conf, 1, dfsCluster, 
true);
     try {
-      HMerge.merge(conf, fs, desc.getName());
+      HMerge.merge(conf, dfsCluster.getFileSystem(), desc.getName());
     } finally {
       hCluster.shutdown();
     }

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner.java
 Sat Jan 12 17:36:08 2008
@@ -25,8 +25,6 @@
 import java.util.TreeMap;
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
 
@@ -50,9 +48,18 @@
   
   private static final long START_CODE = Long.MAX_VALUE;
 
+  private MiniDFSCluster cluster = null;
   private HRegion r;
   private HRegionIncommon region;
 
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void setUp() throws Exception {
+    cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
+    super.setUp();
+    
+  }
+  
   /** Compare the HRegionInfo we read from HBase to what we stored */
   private void validateRegionInfo(byte [] regionBytes) throws IOException {
     HRegionInfo info =
@@ -127,26 +134,8 @@
    * @throws IOException
    */
   public void testScanner() throws IOException {
-    MiniDFSCluster cluster = null;
-    FileSystem fs = null;
-    
     try {
-      
-      // Initialization
-      
-      HBaseConfiguration conf = new HBaseConfiguration();
-      cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
-      fs = cluster.getFileSystem();
-      Path dir = new Path("/hbase");
-      fs.mkdirs(dir);
-      
-      Path regionDir = HRegion.getRegionDir(dir,
-          HRegionInfo.encodeRegionName(REGION_INFO.getRegionName()));
-      fs.mkdirs(regionDir);
-      
-      HLog log = new HLog(fs, new Path(regionDir, "log"), conf, null);
-
-      r = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
+      r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
       region = new HRegionIncommon(r);
       
       // Write information to the meta table
@@ -168,8 +157,7 @@
       // Close and re-open
       
       r.close();
-      log.rollWriter();
-      r = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
+      r = openClosedRegion(r);
       region = new HRegionIncommon(r);
 
       // Verify we can get the data back now that it is on disk.
@@ -209,8 +197,7 @@
       // Close and reopen
       
       r.close();
-      log.rollWriter();
-      r = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
+      r = openClosedRegion(r);
       region = new HRegionIncommon(r);
 
       // Validate again
@@ -246,8 +233,7 @@
       // Close and reopen
       
       r.close();
-      log.rollWriter();
-      r = new HRegion(dir, log, fs, conf, REGION_INFO, null, null);
+      r = openClosedRegion(r);
       region = new HRegionIncommon(r);
 
       // Validate again
@@ -258,7 +244,7 @@
       // clean up
       
       r.close();
-      log.closeAndDelete();
+      r.getLog().closeAndDelete();
       
     } finally {
       StaticTestEnvironment.shutdownDfs(cluster);

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=611519&r1=611518&r2=611519&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
 Sat Jan 12 17:36:08 2008
@@ -24,6 +24,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -64,18 +65,21 @@
    * @throws Exception
    */
   public void testBasicSplit() throws Exception {
+    MiniDFSCluster cluster = null;
     HRegion region = null;
-    HLog hlog = new HLog(this.localFs, this.testDir, this.conf, null);
     try {
+      cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
       HTableDescriptor htd = createTableDescriptor(getName());
-      HRegionInfo hri = new HRegionInfo(htd, null, null);
-      region = new HRegion(testDir, hlog, this.localFs, this.conf, hri, null, 
null);
+      region = createNewHRegion(htd, null, null);
       basicSplit(region);
     } finally {
       if (region != null) {
         region.close();
+        region.getLog().closeAndDelete();
+      }
+      if (cluster != null) {
+        StaticTestEnvironment.shutdownDfs(cluster);
       }
-      hlog.closeAndDelete();
     }
   }
   
@@ -166,13 +170,6 @@
     }
   }
   
-  private HRegion openClosedRegion(final HRegion closedRegion)
-  throws IOException {
-    return new HRegion(closedRegion.getRootDir(), closedRegion.getLog(),
-      closedRegion.getFilesystem(), closedRegion.getConf(),
-      closedRegion.getRegionInfo(), null, null);
-  }
-
   private void assertGet(final HRegion r, final String family, final Text k)
   throws IOException {
     // Now I have k, get values out and assert they are as expected.

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTimestamp.java
 Sat Jan 12 17:36:08 2008
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.util.TreeMap;
 
+import org.apache.hadoop.dfs.MiniDFSCluster;
+
 import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
@@ -43,7 +45,29 @@
   // When creating column descriptor, how many versions of a cell to allow.
   private static final int VERSIONS = 3;
   
+  private MiniDFSCluster cluster;
+
+  /** constructor */
+  public TestTimestamp() {
+    super();
+    this.cluster = null;
+  }
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void setUp() throws Exception {
+    this.cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
+    super.setUp();
+  }
 
+  /** [EMAIL PROTECTED] */
+  @Override
+  public void tearDown() throws Exception {
+    if (this.cluster != null) {
+      StaticTestEnvironment.shutdownDfs(cluster);
+    }
+  }
+  
   /**
    * Test that delete works according to description in <a
    * href="https://issues.apache.org/jira/browse/HADOOP-1784";>hadoop-1784</a>.
@@ -310,11 +334,9 @@
   }
   
   private HRegion createRegion() throws IOException {
-    HLog hlog = new HLog(this.localFs, this.testDir, this.conf, null);
     HTableDescriptor htd = createTableDescriptor(getName());
     htd.addFamily(new HColumnDescriptor(COLUMN, VERSIONS,
       CompressionType.NONE, false, Integer.MAX_VALUE, null));
-    HRegionInfo hri = new HRegionInfo(htd, null, null);
-    return new HRegion(testDir, hlog, this.localFs, this.conf, hri, null, 
null);
+    return createNewHRegion(htd, null, null);
   }
 }

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java
 Sat Jan 12 17:36:08 2008
@@ -76,15 +76,12 @@
   private HTableDescriptor desc;
 
   private MiniDFSCluster dfsCluster = null;
-  private FileSystem fs;
   private Path dir;
   private MiniHBaseCluster hCluster = null;
 
   /** [EMAIL PROTECTED] */
   @Override
   public void setUp() throws Exception {
-    super.setUp();
-
     // Make sure the cache gets flushed so we trigger a compaction(s) and
     // hence splits.
     conf.setInt("hbase.hregion.memcache.flush.size", 1024 * 1024);
@@ -101,21 +98,25 @@
     desc.addFamily(new HColumnDescriptor(OUTPUT_COLUMN));
 
     dfsCluster = new MiniDFSCluster(conf, 1, true, (String[]) null);
-    try {
-      fs = dfsCluster.getFileSystem();
 
+    // Must call super.setUp after mini dfs cluster is started or else
+    // filesystem ends up being local
+    
+    super.setUp();
+
+    try {
       dir = new Path("/hbase");
       fs.mkdirs(dir);
 
       // Start up HBase cluster
-      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
+      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster, true);
 
       // Create a table.
       HBaseAdmin admin = new HBaseAdmin(conf);
       admin.createTable(desc);
 
       // Populate a table into multiple regions
-      makeMultiRegionTable(conf, hCluster, null, TABLE_NAME, INPUT_COLUMN);
+      makeMultiRegionTable(conf, hCluster, this.fs, TABLE_NAME, INPUT_COLUMN);
 
       // Verify table indeed has multiple regions
       HTable table = new HTable(conf, new Text(TABLE_NAME));
@@ -256,10 +257,11 @@
       // ignore
     }
 
-    Path localDir = new Path(this.testDir, "index_" +
+    Path localDir = new Path(getUnitTestdir(getName()), "index_" +
       Integer.toString(new Random().nextInt()));
     this.fs.copyToLocalFile(new Path(INDEX_DIR), localDir);
-    Path [] indexDirs = this.localFs.listPaths(new Path [] {localDir});
+    FileSystem localfs = FileSystem.getLocal(conf);
+    Path [] indexDirs = localfs.listPaths(new Path [] {localDir});
     Searcher searcher = null;
     HScannerInterface scanner = null;
     try {

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java?rev=611519&r1=611518&r2=611519&view=diff
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
 Sat Jan 12 17:36:08 2008
@@ -27,7 +27,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -69,7 +68,6 @@
   };
 
   private MiniDFSCluster dfsCluster = null;
-  private FileSystem fs;
   private Path dir;
   private MiniHBaseCluster hCluster = null;
   
@@ -116,16 +114,19 @@
    */
   @Override
   public void setUp() throws Exception {
-    super.setUp();
     dfsCluster = new MiniDFSCluster(conf, 1, true, (String[])null);
+
+    // Must call super.setup() after starting mini dfs cluster. Otherwise
+    // we get a local file system instead of hdfs
+    
+    super.setUp();
     try {
-      fs = dfsCluster.getFileSystem();
       dir = new Path("/hbase");
       fs.mkdirs(dir);
       // Start up HBase cluster
       // Only one region server.  MultiRegionServer manufacturing code below
       // depends on there being one region server only.
-      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
+      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster, true);
       LOG.info("Master is at " + this.conf.get(HConstants.MASTER_ADDRESS));
     } catch (Exception e) {
       StaticTestEnvironment.shutdownDfs(dfsCluster);
@@ -252,13 +253,13 @@
             IdentityTableReduce.class, jobConf);
         LOG.info("Started " + SINGLE_REGION_TABLE_NAME);
         JobClient.runJob(jobConf);
-        
+
         LOG.info("Print table contents after map/reduce for " +
           SINGLE_REGION_TABLE_NAME);
-        scanTable(SINGLE_REGION_TABLE_NAME, true);
+      scanTable(SINGLE_REGION_TABLE_NAME, true);
 
-        // verify map-reduce results
-        verify(SINGLE_REGION_TABLE_NAME);
+      // verify map-reduce results
+      verify(SINGLE_REGION_TABLE_NAME);
       } finally {
         mrCluster.shutdown();
       }
@@ -306,7 +307,7 @@
             IdentityTableReduce.class, jobConf);
         LOG.info("Started " + MULTI_REGION_TABLE_NAME);
         JobClient.runJob(jobConf);
-        
+
         // verify map-reduce results
         verify(MULTI_REGION_TABLE_NAME);
       } finally {

Added: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/util/TestMigrate.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/util/TestMigrate.java?rev=611519&view=auto
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/util/TestMigrate.java
 (added)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/util/TestMigrate.java
 Sat Jan 12 17:36:08 2008
@@ -0,0 +1,168 @@
+/**
+ * Copyright 2007 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.util;
+
+import java.io.IOException;
+
+import java.util.zip.ZipInputStream;
+import java.util.zip.ZipEntry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+
+import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.StaticTestEnvironment;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ * 
+ */
+public class TestMigrate extends HBaseTestCase {
+  static final Log LOG = LogFactory.getLog(TestMigrate.class);
+
+  /**
+   * 
+   */
+  public TestMigrate() {
+    super();
+    Logger.getRootLogger().setLevel(Level.WARN);
+    Logger.getLogger(this.getClass().getPackage().getName()).
+      setLevel(Level.DEBUG);
+  }
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+  }
+
+  /** [EMAIL PROTECTED] */
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+  }
+  
+  /**
+   * 
+   */
+  public void testUpgrade() {
+    MiniDFSCluster dfsCluster = null;
+    try {
+      dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
+      FileSystem dfs = dfsCluster.getFileSystem();
+      Path root = dfs.makeQualified(new Path(
+          conf.get(HConstants.HBASE_DIR, HConstants.DEFAULT_HBASE_DIR)));
+      dfs.mkdirs(root);
+
+      /*
+       * First load files from an old style HBase file structure
+       */
+      
+      // Current directory is 
.../workspace/project/build/contrib/hbase/test/data
+      
+      FileSystem localfs = FileSystem.getLocal(conf);
+      
+      // Get path for zip file
+
+      FSDataInputStream hs = localfs.open(new Path(Path.CUR_DIR,
+          
+          // this path is for running test with ant
+          
+          
"../../../../../src/contrib/hbase/src/testdata/HADOOP-2478-testdata.zip")
+      
+          // and this path is for when you want to run inside eclipse
+      
+          /*"src/contrib/hbase/src/testdata/HADOOP-2478-testdata.zip")*/
+      );
+      
+      ZipInputStream zip = new ZipInputStream(hs);
+      
+      unzip(zip, dfs, root);
+      
+      zip.close();
+      hs.close();
+      
+      listPaths(dfs, root, root.toString().length() + 1);
+      
+      Migrate u = new Migrate(conf);
+      u.run((String[]) null);
+
+      listPaths(dfs, root, root.toString().length() + 1);
+      
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      if (dfsCluster != null) {
+        StaticTestEnvironment.shutdownDfs(dfsCluster);
+      }
+    }
+  }
+
+  private void unzip(ZipInputStream zip, FileSystem dfs, Path root)
+  throws IOException {
+
+    ZipEntry e = null;
+    while ((e = zip.getNextEntry()) != null)  {
+      if (e.isDirectory()) {
+        dfs.mkdirs(new Path(root, e.getName()));
+        
+      } else {
+        FSDataOutputStream out = dfs.create(new Path(root, e.getName()));
+        byte[] buffer = new byte[4096];
+        int len;
+        do {
+          len = zip.read(buffer);
+          if (len > 0) {
+            out.write(buffer, 0, len);
+          }
+        } while (len > 0);
+        out.close();
+      }
+      zip.closeEntry();
+    }
+  }
+  
+  private void listPaths(FileSystem fs, Path dir, int rootdirlength)
+  throws IOException {
+    FileStatus[] stats = fs.listStatus(dir);
+    if (stats == null || stats.length == 0) {
+      return;
+    }
+    for (int i = 0; i < stats.length; i++) {
+      String relativePath =
+        stats[i].getPath().toString().substring(rootdirlength);
+      if (stats[i].isDir()) {
+        System.out.println("d " + relativePath);
+        listPaths(fs, stats[i].getPath(), rootdirlength);
+      } else {
+        System.out.println("f " + relativePath + " size=" + stats[i].getLen());
+      }
+    }
+  }
+}


Reply via email to