You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/06/08 02:04:50 UTC

svn commit: r1133210 [3/3] - in /hbase/trunk: ./ bin/ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/catalog/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/io/ src/main/java/org/apache...

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java Wed Jun  8 00:04:47 2011
@@ -33,9 +33,9 @@ public class TestColumnPrefixFilter {
     String family = "Family";
     HTableDescriptor htd = new HTableDescriptor("TestColumnPrefixFilter");
     htd.addFamily(new HColumnDescriptor(family));
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
-        getTestDir(), TEST_UTIL.getConfiguration(), htd);
+        getTestDir(), TEST_UTIL.getConfiguration());
 
     List<String> rows = generateRandomWords(100, "row");
     List<String> columns = generateRandomWords(10000, "column");
@@ -89,9 +89,9 @@ public class TestColumnPrefixFilter {
     String family = "Family";
     HTableDescriptor htd = new HTableDescriptor("TestColumnPrefixFilter");
     htd.addFamily(new HColumnDescriptor(family));
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
-        getTestDir(), TEST_UTIL.getConfiguration(), htd);
+        getTestDir(), TEST_UTIL.getConfiguration());
 
     List<String> rows = generateRandomWords(100, "row");
     List<String> columns = generateRandomWords(10000, "column");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java Wed Jun  8 00:04:47 2011
@@ -75,9 +75,8 @@ public class TestDependentColumnFilter e
     HTableDescriptor htd = new HTableDescriptor(getName());
     htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
     htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-    this.region = HRegion.createHRegion(info, testUtil.getTestDir(),
-        testUtil.getConfiguration(), htd);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
+    this.region = HRegion.createHRegion(info, testUtil.getTestDir(), testUtil.getConfiguration());
     addData();
   }
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java Wed Jun  8 00:04:47 2011
@@ -92,8 +92,8 @@ public class TestFilter extends HBaseTes
     HTableDescriptor htd = new HTableDescriptor(getName());
     htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
     htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-    this.region = HRegion.createHRegion(info, this.testDir, this.conf, htd);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
+    this.region = HRegion.createHRegion(info, this.testDir, this.conf);
 
     // Insert first half
     for(byte [] ROW : ROWS_ONE) {

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Wed Jun  8 00:04:47 2011
@@ -116,14 +116,9 @@ public class TestCatalogJanitor {
    */
   class MockMasterServices implements MasterServices {
     private final MasterFileSystem mfs;
-    private final AssignmentManager asm;
 
     MockMasterServices(final Server server) throws IOException {
       this.mfs = new MasterFileSystem(server, null);
-      HTableDescriptor htd = new HTableDescriptor("table");
-      htd.addFamily(new HColumnDescriptor("family"));
-      this.asm = Mockito.mock(AssignmentManager.class);
-      Mockito.when(asm.getTableDescriptor("table")).thenReturn(htd);
     }
 
     @Override
@@ -133,7 +128,7 @@ public class TestCatalogJanitor {
 
     @Override
     public AssignmentManager getAssignmentManager() {
-      return this.asm;
+      return null;
     }
 
     @Override
@@ -224,14 +219,11 @@ public class TestCatalogJanitor {
     HTableDescriptor htd = new HTableDescriptor("table");
     htd.addFamily(new HColumnDescriptor("family"));
     HRegionInfo parent =
-      new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
-          Bytes.toBytes("eee"));
+      new HRegionInfo(htd, Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
     HRegionInfo splita =
-      new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
-          Bytes.toBytes("ccc"));
+      new HRegionInfo(htd, Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
     HRegionInfo splitb =
-      new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
-          Bytes.toBytes("eee"));
+      new HRegionInfo(htd, Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
     // Test that when both daughter regions are in place, that we do not
     // remove the parent.
     List<KeyValue> kvs = new ArrayList<KeyValue>();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java Wed Jun  8 00:04:47 2011
@@ -301,7 +301,7 @@ public class TestDistributedLogSplitting
     HTable ht = TEST_UTIL.createTable(table, family);
     int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family, nrs);
     assertEquals(nrs, numRegions);
-      LOG.info("Waiting for no more RIT\n");
+    LOG.info("Waiting for no more RIT\n");
     blockUntilNoRIT(zkw, master);
     // disable-enable cycle to get rid of table's dead regions left behind
     // by createMultiRegions
@@ -353,7 +353,6 @@ public class TestDistributedLogSplitting
       int num_edits, int edit_size) throws IOException {
 
     byte[] table = Bytes.toBytes(tname);
-    HTableDescriptor htd = new HTableDescriptor(tname);
     byte[] value = new byte[edit_size];
     for (int i = 0; i < edit_size; i++) {
       value[i] = (byte)('a' + (i % 26));
@@ -370,7 +369,7 @@ public class TestDistributedLogSplitting
           System.currentTimeMillis(), value));
       // LOG.info("Region " + i + ": " + e);
       j++;
-      log.append(hris.get(j % n), table, e, System.currentTimeMillis(), htd);
+      log.append(hris.get(j % n), table, e, System.currentTimeMillis());
       counts[j % n] += 1;
       // if ((i % 8096) == 0) {
         // log.sync();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLoadBalancer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLoadBalancer.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLoadBalancer.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestLoadBalancer.java Wed Jun  8 00:04:47 2011
@@ -486,7 +486,7 @@ public class TestLoadBalancer {
       Bytes.putInt(start, 0, numRegions << 1);
       Bytes.putInt(end, 0, (numRegions << 1) + 1);
       HRegionInfo hri = new HRegionInfo(
-          Bytes.toBytes("table" + i), start, end,
+          new HTableDescriptor(Bytes.toBytes("table" + i)), start, end,
           false, regionId++);
       regions.add(hri);
     }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java Wed Jun  8 00:04:47 2011
@@ -31,14 +31,18 @@ import java.util.TreeSet;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
 import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
 import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
@@ -258,21 +262,12 @@ public class TestMasterFailover {
     byte [] enabledTable = Bytes.toBytes("enabledTable");
     HTableDescriptor htdEnabled = new HTableDescriptor(enabledTable);
     htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
-
-    FileSystem filesystem = FileSystem.get(conf);
-    Path rootdir = filesystem.makeQualified(
-        new Path(conf.get(HConstants.HBASE_DIR)));
-    HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(), null, null);
-    HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
-
     List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
 
     byte [] disabledTable = Bytes.toBytes("disabledTable");
     HTableDescriptor htdDisabled = new HTableDescriptor(disabledTable);
     htdDisabled.addFamily(new HColumnDescriptor(FAMILY));
-    HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
-    HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
     List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
 
@@ -571,23 +566,12 @@ public class TestMasterFailover {
     byte [] enabledTable = Bytes.toBytes("enabledTable");
     HTableDescriptor htdEnabled = new HTableDescriptor(enabledTable);
     htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
-    FileSystem filesystem = FileSystem.get(conf);
-    Path rootdir = filesystem.makeQualified(
-           new Path(conf.get(HConstants.HBASE_DIR)));
-
-    HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(),
-        null, null);
-    HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
-
     List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
 
     byte [] disabledTable = Bytes.toBytes("disabledTable");
     HTableDescriptor htdDisabled = new HTableDescriptor(disabledTable);
     htdDisabled.addFamily(new HColumnDescriptor(FAMILY));
-    HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
-    HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
-
     List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java Wed Jun  8 00:04:47 2011
@@ -25,12 +25,14 @@ import java.util.List;
 import java.util.NavigableMap;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hbase.tmpl.master.MasterStatusTmpl;
@@ -54,20 +56,11 @@ public class TestMasterStatusServlet {
     new ServerName("fakehost", 12345, 1234567890);
   static final HTableDescriptor FAKE_TABLE =
     new HTableDescriptor("mytable");
-  static final HRegionInfo FAKE_HRI =
-      new HRegionInfo(FAKE_TABLE.getName(), Bytes.toBytes("a"), Bytes.toBytes("b"));
-
- // static final HRegionInfo FAKE_REGION = null;
-
+  static final HRegionInfo FAKE_REGION =
+    new HRegionInfo(FAKE_TABLE, Bytes.toBytes("a"), Bytes.toBytes("b"));
+  
   @Before
   public void setupBasicMocks() {
-    try {
-       HRegion.createHRegion(FAKE_HRI, HBaseTestingUtility.getTestDir(),
-          HBaseConfiguration.create(), FAKE_TABLE);
-    } catch(IOException ioe) {
-
-    }
-
     conf = HBaseConfiguration.create();
     
     master = Mockito.mock(HMaster.class);
@@ -84,7 +77,7 @@ public class TestMasterStatusServlet {
     NavigableMap<String, RegionState> regionsInTransition =
       Maps.newTreeMap();
     regionsInTransition.put("r1",
-        new RegionState(FAKE_HRI, RegionState.State.CLOSING, 12345L, FAKE_HOST));
+        new RegionState(FAKE_REGION, RegionState.State.CLOSING, 12345L, FAKE_HOST));        
     Mockito.doReturn(regionsInTransition).when(am).getRegionsInTransition();
     Mockito.doReturn(am).when(master).getAssignmentManager();
     

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java Wed Jun  8 00:04:47 2011
@@ -68,10 +68,10 @@ public class TestColumnSeeking {
             HColumnDescriptor.DEFAULT_BLOOMFILTER);
     HTableDescriptor htd = new HTableDescriptor(table);
     htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(Bytes.toBytes(table), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     HRegion region =
         HRegion.createHRegion(info, HBaseTestingUtility.getTestDir(), TEST_UTIL
-            .getConfiguration(), htd);
+            .getConfiguration());
 
     List<String> rows = generateRandomWords(10, "row");
     List<String> allColumns = generateRandomWords(10, "column");
@@ -172,11 +172,10 @@ public class TestColumnSeeking {
 
     HTableDescriptor htd = new HTableDescriptor(table);
     htd.addFamily(new HColumnDescriptor(family));
-
-    HRegionInfo info = new HRegionInfo(Bytes.toBytes(table), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     HRegion region =
         HRegion.createHRegion(info, HBaseTestingUtility.getTestDir(), TEST_UTIL
-            .getConfiguration(), htd);
+            .getConfiguration());
 
     List<String> rows = generateRandomWords(10, "row");
     List<String> allColumns = generateRandomWords(100, "column");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java Wed Jun  8 00:04:47 2011
@@ -82,12 +82,9 @@ public class TestCompactSelection extend
 
     HTableDescriptor htd = new HTableDescriptor(Bytes.toBytes("table"));
     htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
-
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
-    HRegion.createHRegion(info, basedir, conf, htd);
-    Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
-    HRegion region = new HRegion(tableDir, hlog, fs, conf, info, null);
+    HRegion region = new HRegion(basedir, hlog, fs, conf, info, null);
 
     store = new Store(basedir, region, hcd, fs, conf);
     TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java Wed Jun  8 00:04:47 2011
@@ -72,17 +72,17 @@ public class TestGetClosestAtOrBefore ex
     Path rootdir = filesystem.makeQualified(new Path(conf.get(HConstants.HBASE_DIR)));
     filesystem.mkdirs(rootdir);
     // Up flush size else we bind up when we use default catalog flush of 16k.
-    HTableDescriptor.META_TABLEDESC.setMemStoreFlushSize(64 * 1024 * 1024);
-
+    HRegionInfo.FIRST_META_REGIONINFO.getTableDesc().
+      setMemStoreFlushSize(64 * 1024 * 1024);
     HRegion mr = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO,
-      rootdir, this.conf, HTableDescriptor.META_TABLEDESC);
+      rootdir, this.conf);
     // Write rows for three tables 'A', 'B', and 'C'.
     for (char c = 'A'; c < 'D'; c++) {
       HTableDescriptor htd = new HTableDescriptor("" + c);
       final int last = 128;
       final int interval = 2;
       for (int i = 0; i <= last; i += interval) {
-        HRegionInfo hri = new HRegionInfo(htd.getName(),
+        HRegionInfo hri = new HRegionInfo(htd,
           i == 0? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i),
           i == last? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i + interval));
         Put put = new Put(hri.getRegionName());

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Wed Jun  8 00:04:47 2011
@@ -2794,9 +2794,9 @@ public class TestHRegion extends HBaseTe
     
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     Path path = new Path(DIR + "testBloomFilterSize");
-    region = HRegion.createHRegion(info, path, conf, htd);
+    region = HRegion.createHRegion(info, path, conf);
     
     int num_unique_rows = 10;
     int duplicate_multiplier =2;
@@ -2852,9 +2852,9 @@ public class TestHRegion extends HBaseTe
         HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
     HTableDescriptor htd = new HTableDescriptor(TABLE);
     htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     Path path = new Path(DIR + "testAllColumnsWithBloomFilter");
-    region = HRegion.createHRegion(info, path, conf, htd);
+    region = HRegion.createHRegion(info, path, conf);
 
     // For row:0, col:0: insert versions 1 through 5.
     byte row[] = Bytes.toBytes("row:" + 0);
@@ -2897,9 +2897,9 @@ public class TestHRegion extends HBaseTe
 
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     Path path = new Path(DIR + "TestDeleteRowWithBloomFilter");
-    region = HRegion.createHRegion(info, path, conf, htd);
+    region = HRegion.createHRegion(info, path, conf);
 
     // Insert some data
     byte row[] = Bytes.toBytes("row1");
@@ -3033,14 +3033,14 @@ public class TestHRegion extends HBaseTe
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     Path path = new Path(DIR + callingMethod);
     if (fs.exists(path)) {
       if (!fs.delete(path, true)) {
         throw new IOException("Failed delete of " + path);
       }
     }
-    region = HRegion.createHRegion(info, path, conf, htd);
+    region = HRegion.createHRegion(info, path, conf);
   }
 
   /**

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java Wed Jun  8 00:04:47 2011
@@ -56,7 +56,7 @@ public class TestHRegionInfo {
   public void testContainsRange() {
     HTableDescriptor tableDesc = new HTableDescriptor("testtable");
     HRegionInfo hri = new HRegionInfo(
-        tableDesc.getName(), Bytes.toBytes("a"), Bytes.toBytes("g"));
+        tableDesc, Bytes.toBytes("a"), Bytes.toBytes("g"));
     // Single row range at start of region
     assertTrue(hri.containsRange(Bytes.toBytes("a"), Bytes.toBytes("a")));
     // Fully contained range

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java Wed Jun  8 00:04:47 2011
@@ -82,8 +82,8 @@ public class TestRSStatusServlet {
   public void testWithRegions() throws IOException {
     HTableDescriptor htd = new HTableDescriptor("mytable");
     List<HRegionInfo> regions = Lists.newArrayList(
-        new HRegionInfo(htd.getName(), Bytes.toBytes("a"), Bytes.toBytes("d")),
-        new HRegionInfo(htd.getName(), Bytes.toBytes("d"), Bytes.toBytes("z"))
+        new HRegionInfo(htd, Bytes.toBytes("a"), Bytes.toBytes("d")),
+        new HRegionInfo(htd, Bytes.toBytes("d"), Bytes.toBytes("z"))
         );
     Mockito.doReturn(regions).when(rs).getOnlineRegions();
     

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java Wed Jun  8 00:04:47 2011
@@ -61,7 +61,7 @@ public class TestResettingCounters {
     HTableDescriptor htd = new HTableDescriptor(table);
     for (byte [] family : families) htd.addFamily(new HColumnDescriptor(family));
 
-    HRegionInfo hri = new HRegionInfo(htd.getName(), null, null, false);
+    HRegionInfo hri = new HRegionInfo(htd, null, null, false);
     String testDir = HBaseTestingUtility.getTestDir() + "/TestResettingCounters/";
     Path path = new Path(testDir);
     if (fs.exists(path)) {
@@ -69,7 +69,7 @@ public class TestResettingCounters {
         throw new IOException("Failed delete of " + path);
       }
     }
-    HRegion region = HRegion.createHRegion(hri, path, conf, htd);
+    HRegion region = HRegion.createHRegion(hri, path, conf);
 
     Increment odd = new Increment(rows[0]);
     Increment even = new Increment(rows[0]);

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java Wed Jun  8 00:04:47 2011
@@ -72,7 +72,7 @@ public class TestScanner extends HBaseTe
   }
   /** HRegionInfo for root region */
   public static final HRegionInfo REGION_INFO =
-    new HRegionInfo(TESTTABLEDESC.getName(), HConstants.EMPTY_BYTE_ARRAY,
+    new HRegionInfo(TESTTABLEDESC, HConstants.EMPTY_BYTE_ARRAY,
     HConstants.EMPTY_BYTE_ARRAY);
 
   private static final byte [] ROW_KEY = REGION_INFO.getRegionName();
@@ -101,7 +101,7 @@ public class TestScanner extends HBaseTe
     byte [] startrow = Bytes.toBytes("bbb");
     byte [] stoprow = Bytes.toBytes("ccc");
     try {
-      this.r = createNewHRegion(TESTTABLEDESC, null, null);
+      this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
       addContent(this.r, HConstants.CATALOG_FAMILY);
       List<KeyValue> results = new ArrayList<KeyValue>();
       // Do simple test of getting one row only first.
@@ -175,7 +175,7 @@ public class TestScanner extends HBaseTe
 
   public void testFilters() throws IOException {
     try {
-      this.r = createNewHRegion(TESTTABLEDESC, null, null);
+      this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
       addContent(this.r, HConstants.CATALOG_FAMILY);
       byte [] prefix = Bytes.toBytes("ab");
       Filter newFilter = new PrefixFilter(prefix);
@@ -203,7 +203,7 @@ public class TestScanner extends HBaseTe
    */
   public void testRaceBetweenClientAndTimeout() throws Exception {
     try {
-      this.r = createNewHRegion(TESTTABLEDESC, null, null);
+      this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
       addContent(this.r, HConstants.CATALOG_FAMILY);
       Scan scan = new Scan();
       InternalScanner s = r.getScanner(scan);
@@ -352,7 +352,7 @@ public class TestScanner extends HBaseTe
     assertEquals(0, info.getStartKey().length);
     assertEquals(0, info.getEndKey().length);
     assertEquals(0, Bytes.compareTo(info.getRegionName(), REGION_INFO.getRegionName()));
-    //assertEquals(0, info.getTableDesc().compareTo(REGION_INFO.getTableDesc()));
+    assertEquals(0, info.getTableDesc().compareTo(REGION_INFO.getTableDesc()));
   }
 
   /** Use a scanner to get the region info and then validate the results */
@@ -448,7 +448,7 @@ public class TestScanner extends HBaseTe
    * @throws Exception
    */
   public void testScanAndSyncFlush() throws Exception {
-    this.r = createNewHRegion(TESTTABLEDESC, null, null);
+    this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
     HRegionIncommon hri = new HRegionIncommon(r);
     try {
         LOG.info("Added: " + addContent(hri, Bytes.toString(HConstants.CATALOG_FAMILY),
@@ -472,7 +472,7 @@ public class TestScanner extends HBaseTe
    * @throws Exception
    */
   public void testScanAndRealConcurrentFlush() throws Exception {
-    this.r = createNewHRegion(TESTTABLEDESC, null, null);
+    this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
     HRegionIncommon hri = new HRegionIncommon(r);
     try {
         LOG.info("Added: " + addContent(hri, Bytes.toString(HConstants.CATALOG_FAMILY),

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java Wed Jun  8 00:04:47 2011
@@ -153,8 +153,8 @@ public class TestSplitTransaction {
     int daughtersRowCount = 0;
     for (HRegion r: daughters) {
       // Open so can count its content.
-      HRegion openRegion = HRegion.openHRegion(this.testdir, r.getRegionInfo(),
-         r.getLog(), r.getConf());
+      HRegion openRegion = HRegion.openHRegion(r.getRegionInfo(),
+        r.getLog(), r.getConf());
       try {
         int count = countRows(openRegion);
         assertTrue(count > 0 && count != rowcount);
@@ -208,8 +208,8 @@ public class TestSplitTransaction {
     int daughtersRowCount = 0;
     for (HRegion r: daughters) {
       // Open so can count its content.
-      HRegion openRegion = HRegion.openHRegion(this.testdir, r.getRegionInfo(),
-         r.getLog(), r.getConf());
+      HRegion openRegion = HRegion.openHRegion(r.getRegionInfo(),
+        r.getLog(), r.getConf());
       try {
         int count = countRows(openRegion);
         assertTrue(count > 0 && count != rowcount);
@@ -252,8 +252,7 @@ public class TestSplitTransaction {
     HTableDescriptor htd = new HTableDescriptor("table");
     HColumnDescriptor hcd = new HColumnDescriptor(CF);
     htd.addFamily(hcd);
-    HRegionInfo hri = new HRegionInfo(htd.getName(), STARTROW, ENDROW);
-    HRegion.createHRegion(hri, testdir, TEST_UTIL.getConfiguration(), htd);
-    return HRegion.openHRegion(testdir, hri, wal, TEST_UTIL.getConfiguration());
+    HRegionInfo hri = new HRegionInfo(htd, STARTROW, ENDROW);
+    return HRegion.openHRegion(hri, wal, TEST_UTIL.getConfiguration());
   }
 }
\ No newline at end of file

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java Wed Jun  8 00:04:47 2011
@@ -127,7 +127,7 @@ public class TestStore extends TestCase 
 
     HTableDescriptor htd = new HTableDescriptor(table);
     htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
+    HRegionInfo info = new HRegionInfo(htd, null, null, false);
     HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
     HRegion region = new HRegion(basedir, hlog, fs, conf, info, null);
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java Wed Jun  8 00:04:47 2011
@@ -69,6 +69,10 @@ public class TestWideScanner extends HBa
   }
 
   /** HRegionInfo for root region */
+  public static final HRegionInfo REGION_INFO =
+    new HRegionInfo(TESTTABLEDESC, HConstants.EMPTY_BYTE_ARRAY,
+    HConstants.EMPTY_BYTE_ARRAY);
+
   MiniDFSCluster cluster = null;
   HRegion r;
 
@@ -103,7 +107,7 @@ public class TestWideScanner extends HBa
   public void testWideScanBatching() throws IOException {
     final int batch = 256;
     try {
-      this.r = createNewHRegion(TESTTABLEDESC, null, null);
+      this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
       int inserted = addWideContent(this.r);
       List<KeyValue> results = new ArrayList<KeyValue>();
       Scan scan = new Scan();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java Wed Jun  8 00:04:47 2011
@@ -233,17 +233,11 @@ public class TestOpenRegionHandler {
     HTableDescriptor htd =
       new HTableDescriptor("testOpenRegionHandlerYankingRegionFromUnderIt");
     final HRegionInfo hri =
-      new HRegionInfo(htd.getName(), HConstants.EMPTY_END_ROW,
-          HConstants.EMPTY_END_ROW);
-    HRegion region =
-         HRegion.createHRegion(hri, HBaseTestingUtility.getTestDir(), HTU
-            .getConfiguration(), htd);
+      new HRegionInfo(htd, HConstants.EMPTY_END_ROW, HConstants.EMPTY_END_ROW);
     OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri) {
       HRegion openRegion() {
         // Open region first, then remove znode as though it'd been hijacked.
-        //HRegion region = super.openRegion();
-        HRegion region = super.openRegion(HBaseTestingUtility.getTestDir());
-
+        HRegion region = super.openRegion();
         // Don't actually open region BUT remove the znode as though it'd
         // been hijacked on us.
         ZooKeeperWatcher zkw = this.server.getZooKeeper();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Wed Jun  8 00:04:47 2011
@@ -38,7 +38,11 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -141,14 +145,11 @@ public class TestHLog  {
     Path tabledir = new Path(hbaseDir, getName());
     fs.mkdirs(tabledir);
     for(int i = 0; i < howmany; i++) {
-      infos[i] = new HRegionInfo(tableName,
+      infos[i] = new HRegionInfo(new HTableDescriptor(tableName),
                 Bytes.toBytes("" + i), Bytes.toBytes("" + (i+1)), false);
       fs.mkdirs(new Path(tabledir, infos[i].getEncodedName()));
       LOG.info("allo " + new Path(tabledir, infos[i].getEncodedName()).toString());
     }
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor("column"));
-
     // Add edits for three regions.
     try {
       for (int ii = 0; ii < howmany; ii++) {
@@ -163,7 +164,7 @@ public class TestHLog  {
                 System.currentTimeMillis(), column));
             LOG.info("Region " + i + ": " + edit);
             log.append(infos[i], tableName, edit,
-              System.currentTimeMillis(), htd);
+              System.currentTimeMillis());
           }
         }
         log.rollWriter();
@@ -205,15 +206,13 @@ public class TestHLog  {
     HLog wal = new HLog(fs, subdir, oldLogDir, conf);
     final int total = 20;
 
-    HRegionInfo info = new HRegionInfo(bytes,
+    HRegionInfo info = new HRegionInfo(new HTableDescriptor(bytes),
                 null,null, false);
-    HTableDescriptor htd = new HTableDescriptor();
-    htd.addFamily(new HColumnDescriptor(bytes));
 
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
-      wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
+      wal.append(info, bytes, kvs, System.currentTimeMillis());
     }
     // Now call sync and try reading.  Opening a Reader before you sync just
     // gives you EOFE.
@@ -231,7 +230,7 @@ public class TestHLog  {
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
-      wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
+      wal.append(info, bytes, kvs, System.currentTimeMillis());
     }
     reader = HLog.getReader(fs, walPath, conf);
     count = 0;
@@ -250,7 +249,7 @@ public class TestHLog  {
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(Bytes.toBytes(i), bytes, value));
-      wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
+      wal.append(info, bytes, kvs, System.currentTimeMillis());
     }
     // Now I should have written out lots of blocks.  Sync then read.
     wal.sync();
@@ -335,19 +334,17 @@ public class TestHLog  {
   @Test
   public void testAppendClose() throws Exception {
     byte [] tableName = Bytes.toBytes(getName());
-    HRegionInfo regioninfo = new HRegionInfo(tableName,
-             HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
+    HRegionInfo regioninfo = new HRegionInfo(new HTableDescriptor(tableName),
+        HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
     Path subdir = new Path(dir, "hlogdir");
     Path archdir = new Path(dir, "hlogdir_archive");
     HLog wal = new HLog(fs, subdir, archdir, conf);
     final int total = 20;
-    HTableDescriptor htd = new HTableDescriptor();
-    htd.addFamily(new HColumnDescriptor(tableName));
 
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(Bytes.toBytes(i), tableName, tableName));
-      wal.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd);
+      wal.append(regioninfo, tableName, kvs, System.currentTimeMillis());
     }
     // Now call sync to send the data to HDFS datanodes
     wal.sync();
@@ -463,15 +460,11 @@ public class TestHLog  {
             Bytes.toBytes(Integer.toString(i)),
           timestamp, new byte[] { (byte)(i + '0') }));
       }
-      HRegionInfo info = new HRegionInfo(tableName,
+      HRegionInfo info = new HRegionInfo(new HTableDescriptor(tableName),
         row,Bytes.toBytes(Bytes.toString(row) + "1"), false);
-      HTableDescriptor htd = new HTableDescriptor();
-      htd.addFamily(new HColumnDescriptor("column"));
-
-      log.append(info, tableName, cols, System.currentTimeMillis(), htd);
+      log.append(info, tableName, cols, System.currentTimeMillis());
       long logSeqId = log.startCacheFlush();
-      log.completeCacheFlush(info.getEncodedNameAsBytes(), tableName, logSeqId,
-          info.isMetaRegion());
+      log.completeCacheFlush(info.getEncodedNameAsBytes(), tableName, logSeqId, info.isMetaRegion());
       log.close();
       Path filename = log.computeFilename();
       log = null;
@@ -535,11 +528,9 @@ public class TestHLog  {
           Bytes.toBytes(Integer.toString(i)),
           timestamp, new byte[] { (byte)(i + '0') }));
       }
-      HRegionInfo hri = new HRegionInfo(tableName,
+      HRegionInfo hri = new HRegionInfo(new HTableDescriptor(tableName),
           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      HTableDescriptor htd = new HTableDescriptor();
-      htd.addFamily(new HColumnDescriptor("column"));
-      log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
+      log.append(hri, tableName, cols, System.currentTimeMillis());
       long logSeqId = log.startCacheFlush();
       log.completeCacheFlush(hri.getEncodedNameAsBytes(), tableName, logSeqId, false);
       log.close();
@@ -596,17 +587,14 @@ public class TestHLog  {
     DumbWALObserver visitor = new DumbWALObserver();
     log.registerWALActionsListener(visitor);
     long timestamp = System.currentTimeMillis();
-    HTableDescriptor htd = new HTableDescriptor();
-    htd.addFamily(new HColumnDescriptor("column"));
-
-    HRegionInfo hri = new HRegionInfo(tableName,
+    HRegionInfo hri = new HRegionInfo(new HTableDescriptor(tableName),
         HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
     for (int i = 0; i < COL_COUNT; i++) {
       WALEdit cols = new WALEdit();
       cols.add(new KeyValue(row, Bytes.toBytes("column"),
           Bytes.toBytes(Integer.toString(i)),
           timestamp, new byte[]{(byte) (i + '0')}));
-      log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
+      log.append(hri, tableName, cols, System.currentTimeMillis());
     }
     assertEquals(COL_COUNT, visitor.increments);
     log.unregisterWALActionsListener(visitor);
@@ -614,7 +602,7 @@ public class TestHLog  {
     cols.add(new KeyValue(row, Bytes.toBytes("column"),
         Bytes.toBytes(Integer.toString(11)),
         timestamp, new byte[]{(byte) (11 + '0')}));
-    log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
+    log.append(hri, tableName, cols, System.currentTimeMillis());
     assertEquals(COL_COUNT, visitor.increments);
   }
 
@@ -625,9 +613,9 @@ public class TestHLog  {
     final byte [] tableName2 = Bytes.toBytes("testLogCleaning2");
 
     HLog log = new HLog(fs, dir, oldLogDir, conf);
-    HRegionInfo hri = new HRegionInfo(tableName,
+    HRegionInfo hri = new HRegionInfo(new HTableDescriptor(tableName),
         HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-    HRegionInfo hri2 = new HRegionInfo(tableName2,
+    HRegionInfo hri2 = new HRegionInfo(new HTableDescriptor(tableName2),
         HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
 
     // Add a single edit and make sure that rolling won't remove the file
@@ -679,15 +667,12 @@ public class TestHLog  {
 
   private void addEdits(HLog log, HRegionInfo hri, byte [] tableName,
                         int times) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor();
-    htd.addFamily(new HColumnDescriptor("row"));
-
     final byte [] row = Bytes.toBytes("row");
     for (int i = 0; i < times; i++) {
       long timestamp = System.currentTimeMillis();
       WALEdit cols = new WALEdit();
       cols.add(new KeyValue(row, row, row, timestamp, row));
-      log.append(hri, tableName, cols, timestamp, htd);
+      log.append(hri, tableName, cols, timestamp);
     }
   }
 
@@ -701,12 +686,6 @@ public class TestHLog  {
     }
 
     @Override
-    public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
-      //To change body of implemented methods use File | Settings | File Templates.
-      increments++;
-    }
-
-    @Override
     public void logRolled(Path newFile) {
       // TODO Auto-generated method stub
       

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALObserver.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALObserver.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALObserver.java Wed Jun  8 00:04:47 2011
@@ -27,7 +27,11 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.Before;
@@ -84,19 +88,16 @@ public class TestWALObserver {
     list.add(observer);
     DummyWALObserver laterobserver = new DummyWALObserver();
     HLog hlog = new HLog(fs, logDir, oldLogDir, conf, list, null);
-    HRegionInfo hri = new HRegionInfo(SOME_BYTES,
-             SOME_BYTES, SOME_BYTES, false);
+    HRegionInfo hri = new HRegionInfo(new HTableDescriptor(SOME_BYTES),
+        SOME_BYTES, SOME_BYTES, false);
 
     for (int i = 0; i < 20; i++) {
       byte[] b = Bytes.toBytes(i+"");
       KeyValue kv = new KeyValue(b,b,b);
       WALEdit edit = new WALEdit();
       edit.add(kv);
-      HTableDescriptor htd = new HTableDescriptor();
-      htd.addFamily(new HColumnDescriptor(b));
-
       HLogKey key = new HLogKey(b,b, 0, 0);
-      hlog.append(hri, key, edit, htd);
+      hlog.append(hri, key, edit);
       if (i == 10) {
         hlog.registerWALActionsListener(laterobserver);
       }
@@ -113,7 +114,6 @@ public class TestWALObserver {
     assertEquals(2, observer.closedCount);
   }
 
-
   /**
    * Just counts when methods are called
    */
@@ -142,10 +142,5 @@ public class TestWALObserver {
     public void logCloseRequested() {
       closedCount++;
     }
-
-    public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
   }
 }
\ No newline at end of file

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Wed Jun  8 00:04:47 2011
@@ -133,19 +133,14 @@ public class TestWALReplay {
     deleteDir(basedir);
     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
 
-    HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
-    HRegion region2 = HRegion.createHRegion(hri,
-        hbaseRootDir, this.conf, htd);
-
     final byte [] tableName = Bytes.toBytes(tableNameStr);
     final byte [] rowName = tableName;
 
     HLog wal1 = createWAL(this.conf);
     // Add 1k to each family.
     final int countPerFamily = 1000;
-    for (HColumnDescriptor hcd: htd.getFamilies()) {
-      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee,
-          wal1, htd);
+    for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
+      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal1);
     }
     wal1.close();
     runWALSplit(this.conf);
@@ -154,9 +149,8 @@ public class TestWALReplay {
     // Up the sequenceid so that these edits are after the ones added above.
     wal2.setSequenceNumber(wal1.getSequenceNumber());
     // Add 1k to each family.
-    for (HColumnDescriptor hcd: htd.getFamilies()) {
-      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
-          ee, wal2, htd);
+    for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
+      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal2);
     }
     wal2.close();
     runWALSplit(this.conf);
@@ -193,14 +187,11 @@ public class TestWALReplay {
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
     deleteDir(basedir);
-    HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
-    HRegion region2 = HRegion.createHRegion(hri,
-        hbaseRootDir, this.conf, htd);
     HLog wal = createWAL(this.conf);
     HRegion region = HRegion.openHRegion(hri, wal, this.conf);
     Path f =  new Path(basedir, "hfile");
     HFile.Writer writer = new HFile.Writer(this.fs, f);
-    byte [] family = htd.getFamilies().iterator().next().getName();
+    byte [] family = hri.getTableDesc().getFamilies().iterator().next().getName();
     byte [] row = Bytes.toBytes(tableNameStr);
     writer.append(new KeyValue(row, family, family, row));
     writer.close();
@@ -249,9 +240,6 @@ public class TestWALReplay {
     deleteDir(basedir);
     final byte[] rowName = Bytes.toBytes(tableNameStr);
     final int countPerFamily = 10;
-    final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
-    HRegion region3 = HRegion.createHRegion(hri,
-            hbaseRootDir, this.conf, htd);
 
     // Write countPerFamily edits into the three families.  Do a flush on one
     // of the families during the load of edits so its seqid is not same as
@@ -262,7 +250,7 @@ public class TestWALReplay {
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal.setSequenceNumber(seqid);
     boolean first = true;
-    for (HColumnDescriptor hcd: htd.getFamilies()) {
+    for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
       if (first ) {
         // If first, so we have at least one family w/ different seqid to rest.
@@ -273,7 +261,7 @@ public class TestWALReplay {
     // Now assert edits made it in.
     final Get g = new Get(rowName);
     Result result = region.get(g, null);
-    assertEquals(countPerFamily * htd.getFamilies().size(),
+    assertEquals(countPerFamily * hri.getTableDesc().getFamilies().size(),
       result.size());
     // Now close the region, split the log, reopen the region and assert that
     // replay of log has no effect, that our seqids are calculated correctly so
@@ -297,7 +285,7 @@ public class TestWALReplay {
     // Next test.  Add more edits, then 'crash' this region by stealing its wal
     // out from under it and assert that replay of the log adds the edits back
     // correctly when region is opened again.
-    for (HColumnDescriptor hcd: htd.getFamilies()) {
+    for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
     }
     // Get count of edits.
@@ -331,7 +319,7 @@ public class TestWALReplay {
         Result result3 = region3.get(g, null);
         // Assert that count of cells is same as before crash.
         assertEquals(result2.size(), result3.size());
-        assertEquals(htd.getFamilies().size() * countPerFamily,
+        assertEquals(hri.getTableDesc().getFamilies().size() * countPerFamily,
           countOfRestoredEdits.get());
 
         // I can't close wal1.  Its been appropriated when we split.
@@ -354,10 +342,6 @@ public class TestWALReplay {
     final Path basedir = new Path(hbaseRootDir, tableNameStr);
     deleteDir(basedir);
     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
-    final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
-    HRegion region2 = HRegion.createHRegion(hri,
-            hbaseRootDir, this.conf, htd);
-
     final HLog wal = createWAL(this.conf);
     final byte[] tableName = Bytes.toBytes(tableNameStr);
     final byte[] rowName = tableName;
@@ -365,9 +349,8 @@ public class TestWALReplay {
 
     // Add 1k to each family.
     final int countPerFamily = 1000;
-    for (HColumnDescriptor hcd: htd.getFamilies()) {
-      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
-          ee, wal, htd);
+    for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
+      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal);
     }
 
     // Add a cache flush, shouldn't have any effect
@@ -379,14 +362,14 @@ public class TestWALReplay {
     long now = ee.currentTimeMillis();
     edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
       now, rowName));
-    wal.append(hri, tableName, edit, now, htd);
+    wal.append(hri, tableName, edit, now);
 
     // Delete the c family to verify deletes make it over.
     edit = new WALEdit();
     now = ee.currentTimeMillis();
     edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now,
       KeyValue.Type.DeleteFamily));
-    wal.append(hri, tableName, edit, now, htd);
+    wal.append(hri, tableName, edit, now);
 
     // Sync.
     wal.sync();
@@ -428,7 +411,7 @@ public class TestWALReplay {
           Get get = new Get(rowName);
           Result result = region.get(get, -1);
           // Make sure we only see the good edits
-          assertEquals(countPerFamily * (htd.getFamilies().size() - 1),
+          assertEquals(countPerFamily * (hri.getTableDesc().getFamilies().size() - 1),
             result.size());
           region.close();
         } finally {
@@ -458,7 +441,7 @@ public class TestWALReplay {
 
   private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
       final byte [] rowName, final byte [] family,
-      final int count, EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
+      final int count, EnvironmentEdge ee, final HLog wal)
   throws IOException {
     String familyStr = Bytes.toString(family);
     for (int j = 0; j < count; j++) {
@@ -467,7 +450,7 @@ public class TestWALReplay {
       WALEdit edit = new WALEdit();
       edit.add(new KeyValue(rowName, family, qualifierBytes,
         ee.currentTimeMillis(), columnBytes));
-      wal.append(hri, tableName, edit, ee.currentTimeMillis(), htd);
+      wal.append(hri, tableName, edit, ee.currentTimeMillis());
     }
   }
 
@@ -488,9 +471,17 @@ public class TestWALReplay {
    * column families named 'a','b', and 'c'.
    * @param tableName Name of table to use when we create HTableDescriptor.
    */
-   private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
-    return new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
-   }
+  private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
+    htd.addFamily(a);
+    HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
+    htd.addFamily(b);
+    HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
+    htd.addFamily(c);
+    return new HRegionInfo(htd, null, null, false);
+  }
+
 
   /*
    * Run the split.  Verify only single split file made.
@@ -523,15 +514,4 @@ public class TestWALReplay {
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
     return wal;
   }
-
-  private HTableDescriptor createBasic3FamilyHTD(final String tableName) {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
-    htd.addFamily(a);
-    HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
-    htd.addFamily(b);
-    HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
-    htd.addFamily(c);
-    return htd;
-  }
 }
\ No newline at end of file

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java Wed Jun  8 00:04:47 2011
@@ -125,7 +125,7 @@ public class TestReplicationSourceManage
     col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
     htd.addFamily(col);
 
-    hri = new HRegionInfo(htd.getName(), r1, r2);
+    hri = new HRegionInfo(htd, r1, r2);
 
 
   }
@@ -162,8 +162,7 @@ public class TestReplicationSourceManage
       URLEncoder.encode("regionserver:60020", "UTF8"));
 
     manager.init();
-    HTableDescriptor htd = new HTableDescriptor();
-    htd.addFamily(new HColumnDescriptor(f1));
+
     // Testing normal log rolling every 20
     for(long i = 1; i < 101; i++) {
       if(i > 1 && i % 20 == 0) {
@@ -172,7 +171,7 @@ public class TestReplicationSourceManage
       LOG.info(i);
       HLogKey key = new HLogKey(hri.getRegionName(),
         test, seq++, System.currentTimeMillis());
-      hlog.append(hri, key, edit, htd);
+      hlog.append(hri, key, edit);
     }
 
     // Simulate a rapid insert that's followed
@@ -185,7 +184,7 @@ public class TestReplicationSourceManage
     for (int i = 0; i < 3; i++) {
       HLogKey key = new HLogKey(hri.getRegionName(),
         test, seq++, System.currentTimeMillis());
-      hlog.append(hri, key, edit, htd);
+      hlog.append(hri, key, edit);
     }
 
     assertEquals(6, manager.getHLogs().size());
@@ -197,7 +196,7 @@ public class TestReplicationSourceManage
 
     HLogKey key = new HLogKey(hri.getRegionName(),
           test, seq++, System.currentTimeMillis());
-    hlog.append(hri, key, edit, htd);
+    hlog.append(hri, key, edit);
 
     assertEquals(1, manager.getHLogs().size());
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java Wed Jun  8 00:04:47 2011
@@ -88,7 +88,7 @@ public class TestTableRegionModel extend
   public void testGetName() {
     TableRegionModel model = buildTestModel();
     String modelName = model.getName();
-    HRegionInfo hri = new HRegionInfo(Bytes.toBytes(TABLE),
+    HRegionInfo hri = new HRegionInfo(new HTableDescriptor(TABLE),
       START_KEY, END_KEY, false, ID);
     assertEquals(modelName, hri.getRegionNameAsString());
   }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java Wed Jun  8 00:04:47 2011
@@ -131,7 +131,7 @@ public class TestHBaseFsck {
       htd, byte[] startKey, byte[] endKey)
       throws IOException {
     HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
-    HRegionInfo hri = new HRegionInfo(htd.getName(), startKey, endKey);
+    HRegionInfo hri = new HRegionInfo(htd, startKey, endKey);
     Put put = new Put(hri.getRegionName());
     put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
         Writables.getBytes(hri));

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java Wed Jun  8 00:04:47 2011
@@ -136,8 +136,8 @@ public class TestMergeTable {
   private HRegion createRegion(final HTableDescriptor desc,
       byte [] startKey, byte [] endKey, int firstRow, int nrows, Path rootdir)
   throws IOException {
-    HRegionInfo hri = new HRegionInfo(desc.getName(), startKey, endKey);
-    HRegion region = HRegion.createHRegion(hri, rootdir, UTIL.getConfiguration(), desc);
+    HRegionInfo hri = new HRegionInfo(desc, startKey, endKey);
+    HRegion region = HRegion.createHRegion(hri, rootdir, UTIL.getConfiguration());
     LOG.info("Created region " + region.getRegionNameAsString());
     for(int i = firstRow; i < firstRow + nrows; i++) {
       Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i)));
@@ -156,11 +156,10 @@ public class TestMergeTable {
   protected void setupROOTAndMeta(Path rootdir, final HRegion [] regions)
   throws IOException {
     HRegion root =
-      HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, rootdir,
-          UTIL.getConfiguration(), HTableDescriptor.ROOT_TABLEDESC);
+      HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, rootdir, UTIL.getConfiguration());
     HRegion meta =
       HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, rootdir,
-      UTIL.getConfiguration(), HTableDescriptor.META_TABLEDESC);
+      UTIL.getConfiguration());
     HRegion.addRegionToMETA(root, meta);
     for (HRegion r: regions) {
       HRegion.addRegionToMETA(meta, r);

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java?rev=1133210&r1=1133209&r2=1133210&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java Wed Jun  8 00:04:47 2011
@@ -71,36 +71,31 @@ public class TestMergeTool extends HBase
      * Create the HRegionInfos for the regions.
      */
     // Region 0 will contain the key range [row_0200,row_0300)
-    sourceRegions[0] = new HRegionInfo(this.desc.getName(),
-        Bytes.toBytes("row_0200"),
+    sourceRegions[0] = new HRegionInfo(this.desc, Bytes.toBytes("row_0200"),
       Bytes.toBytes("row_0300"));
 
     // Region 1 will contain the key range [row_0250,row_0400) and overlaps
     // with Region 0
     sourceRegions[1] =
-      new HRegionInfo(this.desc.getName(),
-          Bytes.toBytes("row_0250"),
+      new HRegionInfo(this.desc, Bytes.toBytes("row_0250"),
           Bytes.toBytes("row_0400"));
 
     // Region 2 will contain the key range [row_0100,row_0200) and is adjacent
     // to Region 0 or the region resulting from the merge of Regions 0 and 1
     sourceRegions[2] =
-      new HRegionInfo(this.desc.getName(),
-          Bytes.toBytes("row_0100"),
+      new HRegionInfo(this.desc, Bytes.toBytes("row_0100"),
           Bytes.toBytes("row_0200"));
 
     // Region 3 will contain the key range [row_0500,row_0600) and is not
     // adjacent to any of Regions 0, 1, 2 or the merged result of any or all
     // of those regions
     sourceRegions[3] =
-      new HRegionInfo(this.desc.getName(),
-          Bytes.toBytes("row_0500"),
+      new HRegionInfo(this.desc, Bytes.toBytes("row_0500"),
           Bytes.toBytes("row_0600"));
 
     // Region 4 will have empty start and end keys and overlaps all regions.
     sourceRegions[4] =
-      new HRegionInfo(this.desc.getName(),
-          HConstants.EMPTY_BYTE_ARRAY,
+      new HRegionInfo(this.desc, HConstants.EMPTY_BYTE_ARRAY,
           HConstants.EMPTY_BYTE_ARRAY);
 
     /*
@@ -139,8 +134,7 @@ public class TestMergeTool extends HBase
        */
       for (int i = 0; i < sourceRegions.length; i++) {
         regions[i] =
-          HRegion.createHRegion(this.sourceRegions[i], this.testDir, this.conf,
-              this.desc);
+          HRegion.createHRegion(this.sourceRegions[i], this.testDir, this.conf);
         /*
          * Insert data
          */