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 2013/08/08 06:19:56 UTC

svn commit: r1511577 [18/23] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/ap...

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java Thu Aug  8 04:19:49 2013
@@ -28,6 +28,7 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
@@ -134,7 +135,7 @@ public class TestWALPlayer {
     configuration.set(WALPlayer.TABLES_KEY, "table");
     HLogKeyValueMapper mapper = new HLogKeyValueMapper();
     HLogKey key = mock(HLogKey.class);
-    when(key.getTablename()).thenReturn(Bytes.toBytes("table"));
+    when(key.getTablename()).thenReturn(TableName.valueOf("table"));
     @SuppressWarnings("unchecked")
     Mapper<HLogKey, WALEdit, ImmutableBytesWritable, KeyValue>.Context context =
         mock(Context.class);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java Thu Aug  8 04:19:49 2013
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -512,7 +513,7 @@ ClientProtos.ClientService.BlockingInter
   }
 
   @Override
-  public List<HRegion> getOnlineRegions(byte[] tableName) throws IOException {
+  public List<HRegion> getOnlineRegions(TableName tableName) throws IOException {
     // TODO Auto-generated method stub
     return null;
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java Thu Aug  8 04:19:49 2013
@@ -32,6 +32,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -68,7 +69,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@@ -103,7 +103,7 @@ public class TestAssignmentManager {
   private static final ServerName SERVERNAME_B =
     new ServerName("example.org", 0, 5678);
   private static final HRegionInfo REGIONINFO =
-    new HRegionInfo(Bytes.toBytes("t"),
+    new HRegionInfo(TableName.valueOf("t"),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
   private static int assignmentCount;
   private static boolean enabling = false;
@@ -500,7 +500,7 @@ public class TestAssignmentManager {
     // adding region in pending close.
     am.getRegionStates().updateRegionState(
       REGIONINFO, State.SPLITTING, SERVERNAME_A);
-    am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
+    am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
     RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
         REGIONINFO.getRegionName(), SERVERNAME_A);
     String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
@@ -549,9 +549,9 @@ public class TestAssignmentManager {
     // adding region in pending close.
     am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE);
     if (state == Table.State.DISABLING) {
-      am.getZKTable().setDisablingTable(REGIONINFO.getTableNameAsString());
+      am.getZKTable().setDisablingTable(REGIONINFO.getTableName());
     } else {
-      am.getZKTable().setDisabledTable(REGIONINFO.getTableNameAsString());
+      am.getZKTable().setDisabledTable(REGIONINFO.getTableName());
     }
     RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
         REGIONINFO.getRegionName(), SERVERNAME_A);
@@ -575,7 +575,7 @@ public class TestAssignmentManager {
             am.getRegionStates().isRegionsInTransition());
       }
     } finally {
-      am.setEnabledTable(REGIONINFO.getTableNameAsString());
+      am.setEnabledTable(REGIONINFO.getTableName());
       executor.shutdown();
       am.shutdown();
       // Clean up all znodes
@@ -889,7 +889,7 @@ public class TestAssignmentManager {
     }
     try{
       // set table in disabling state.
-      am.getZKTable().setDisablingTable(REGIONINFO.getTableNameAsString());
+      am.getZKTable().setDisablingTable(REGIONINFO.getTableName());
       am.joinCluster();
       // should not call retainAssignment if we get empty regions in assignAllUserRegions.
       assertFalse(
@@ -897,12 +897,12 @@ public class TestAssignmentManager {
           gate.get());
       // need to change table state from disabling to disabled.
       assertTrue("Table should be disabled.",
-          am.getZKTable().isDisabledTable(REGIONINFO.getTableNameAsString()));
+          am.getZKTable().isDisabledTable(REGIONINFO.getTableName()));
     } finally {
       this.server.getConfiguration().setClass(
         HConstants.HBASE_MASTER_LOADBALANCER_CLASS, DefaultLoadBalancer.class,
         LoadBalancer.class);
-      am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
+      am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
       am.shutdown();
     }
   }
@@ -928,17 +928,17 @@ public class TestAssignmentManager {
         this.serverManager);
     try {
       // set table in enabling state.
-      am.getZKTable().setEnablingTable(REGIONINFO.getTableNameAsString());
-      new EnableTableHandler(server, REGIONINFO.getTableName(), am.getCatalogTracker(),
-          am, new NullTableLockManager(), true).prepare()
+      am.getZKTable().setEnablingTable(REGIONINFO.getTableName());
+      new EnableTableHandler(server, REGIONINFO.getTableName(),
+          am.getCatalogTracker(), am, new NullTableLockManager(), true).prepare()
           .process();
       assertEquals("Number of assignments should be 1.", 1, assignmentCount);
       assertTrue("Table should be enabled.",
-          am.getZKTable().isEnabledTable(REGIONINFO.getTableNameAsString()));
+          am.getZKTable().isEnabledTable(REGIONINFO.getTableName()));
     } finally {
       enabling = false;
       assignmentCount = 0;
-      am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
+      am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
       am.shutdown();
       ZKAssign.deleteAllNodes(this.watcher);
     }
@@ -965,7 +965,7 @@ public class TestAssignmentManager {
     // adding region plan
     am.regionPlans.put(REGIONINFO.getEncodedName(),
       new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A));
-    am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
+    am.getZKTable().setEnabledTable(REGIONINFO.getTableName());
 
     try {
       am.assignInvoked = false;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java Thu Aug  8 04:19:49 2013
@@ -31,6 +31,7 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -94,13 +95,13 @@ public class TestAssignmentManagerOnClus
   public void testAssignRegion() throws Exception {
     String table = "testAssignRegion";
     try {
-      HTableDescriptor desc = new HTableDescriptor(table);
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
-        desc.getName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaEditor.addRegionToMeta(meta, hri);
 
       HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
@@ -124,13 +125,13 @@ public class TestAssignmentManagerOnClus
     ServerName deadServer = null;
     HMaster master = null;
     try {
-      HTableDescriptor desc = new HTableDescriptor(table);
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
-        desc.getName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaEditor.addRegionToMeta(meta, hri);
 
       master = TEST_UTIL.getHBaseCluster().getMaster();
@@ -178,7 +179,8 @@ public class TestAssignmentManagerOnClus
    */
   @Test (timeout=60000)
   public void testOfflineRegion() throws Exception {
-    String table = "testOfflineRegion";
+    TableName table =
+        TableName.valueOf("testOfflineRegion");
     try {
       HRegionInfo hri = createTableAndGetOneRegion(table);
 
@@ -191,7 +193,7 @@ public class TestAssignmentManagerOnClus
       long timeoutTime = System.currentTimeMillis() + 800;
       while (true) {
         List<HRegionInfo> regions =
-          regionStates.getRegionsOfTable(Bytes.toBytes(table));
+          regionStates.getRegionsOfTable(table);
         if (!regions.contains(hri)) break;
         long now = System.currentTimeMillis();
         if (now > timeoutTime) {
@@ -203,7 +205,7 @@ public class TestAssignmentManagerOnClus
       RegionState regionState = regionStates.getRegionState(hri);
       assertTrue(regionState.isOffline());
     } finally {
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
@@ -212,7 +214,8 @@ public class TestAssignmentManagerOnClus
    */
   @Test (timeout=50000)
   public void testMoveRegion() throws Exception {
-    String table = "testMoveRegion";
+    TableName table =
+        TableName.valueOf("testMoveRegion");
     try {
       HRegionInfo hri = createTableAndGetOneRegion(table);
 
@@ -248,12 +251,12 @@ public class TestAssignmentManagerOnClus
       }
 
     } finally {
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
   HRegionInfo createTableAndGetOneRegion(
-      final String tableName) throws IOException, InterruptedException {
+      final TableName tableName) throws IOException, InterruptedException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(FAMILY));
     admin.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), 5);
@@ -263,7 +266,7 @@ public class TestAssignmentManagerOnClus
     long timeoutTime = System.currentTimeMillis() + 1000;
     while (true) {
       List<HRegionInfo> regions = master.getAssignmentManager().
-        getRegionStates().getRegionsOfTable(Bytes.toBytes(tableName));
+        getRegionStates().getRegionsOfTable(tableName);
       if (regions.size() > 3) {
         return regions.get(2);
       }
@@ -286,13 +289,13 @@ public class TestAssignmentManagerOnClus
   public void testForceAssignWhileClosing() throws Exception {
     String table = "testForceAssignWhileClosing";
     try {
-      HTableDescriptor desc = new HTableDescriptor(table);
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
-        desc.getName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaEditor.addRegionToMeta(meta, hri);
 
       HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
@@ -332,13 +335,13 @@ public class TestAssignmentManagerOnClus
   public void testCloseFailed() throws Exception {
     String table = "testCloseFailed";
     try {
-      HTableDescriptor desc = new HTableDescriptor(table);
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
-        desc.getName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaEditor.addRegionToMeta(meta, hri);
 
       HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
@@ -376,13 +379,13 @@ public class TestAssignmentManagerOnClus
   public void testOpenFailed() throws Exception {
     String table = "testOpenFailed";
     try {
-      HTableDescriptor desc = new HTableDescriptor(table);
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
-        desc.getName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaEditor.addRegionToMeta(meta, hri);
 
       MockLoadBalancer.controledRegion = hri.getEncodedName();
@@ -415,19 +418,20 @@ public class TestAssignmentManagerOnClus
    */
   @Test (timeout=60000)
   public void testOpenFailedUnrecoverable() throws Exception {
-    String table = "testOpenFailedUnrecoverable";
+    TableName table =
+        TableName.valueOf("testOpenFailedUnrecoverable");
     try {
       HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
-        desc.getName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaEditor.addRegionToMeta(meta, hri);
 
       FileSystem fs = FileSystem.get(conf);
-      Path tableDir= FSUtils.getTablePath(FSUtils.getRootDir(conf), table);
+      Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), table);
       Path regionDir = new Path(tableDir, hri.getEncodedName());
       // create a file named the same as the region dir to
       // mess up with region opening
@@ -454,13 +458,15 @@ public class TestAssignmentManagerOnClus
         getRegionStates().getRegionServerOfRegion(hri);
       TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
     } finally {
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
   @Test (timeout=60000)
   public void testSSHWhenDisablingTableRegionsInOpeningOrPendingOpenState() throws Exception {
-    final String table = "testSSHWhenDisablingTableRegionsInOpeningOrPendingOpenState";
+    final TableName table =
+        TableName.valueOf
+            ("testSSHWhenDisablingTableRegionsInOpeningOrPendingOpenState");
     AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
     HRegionInfo hri = null;
     ServerName serverName = null;
@@ -491,7 +497,7 @@ public class TestAssignmentManagerOnClus
         am.regionOnline(hri, serverName);
       }
       am.getZKTable().setDisabledTable(table);
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Thu Aug  8 04:19:49 2013
@@ -26,6 +26,7 @@ import static org.mockito.Mockito.doRetu
 import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -37,11 +38,13 @@ 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.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
@@ -180,7 +183,7 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public void checkTableModifiable(byte[] tableName) throws IOException {
+    public void checkTableModifiable(TableName tableName) throws IOException {
       //no-op
     }
 
@@ -261,7 +264,7 @@ public class TestCatalogJanitor {
     public TableDescriptors getTableDescriptors() {
       return new TableDescriptors() {
         @Override
-        public HTableDescriptor remove(String tablename) throws IOException {
+        public HTableDescriptor remove(TableName tablename) throws IOException {
           // TODO Auto-generated method stub
           return null;
         }
@@ -273,15 +276,14 @@ public class TestCatalogJanitor {
         }
 
         @Override
-        public HTableDescriptor get(byte[] tablename)
+        public HTableDescriptor get(TableName tablename)
         throws IOException {
-          return get(Bytes.toString(tablename));
+          return createHTableDescriptor();
         }
 
         @Override
-        public HTableDescriptor get(String tablename)
-        throws IOException {
-          return createHTableDescriptor();
+        public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
+          return null;
         }
 
         @Override
@@ -303,25 +305,59 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public void deleteTable(byte[] tableName) throws IOException { }
+    public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
 
     @Override
-    public void modifyTable(byte[] tableName, HTableDescriptor descriptor) throws IOException { }
+    public void deleteNamespace(String name) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
 
     @Override
-    public void enableTable(byte[] tableName) throws IOException { }
+    public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
+      return null;  //To change body of implemented methods use File | Settings | File Templates.
+    }
 
     @Override
-    public void disableTable(byte[] tableName) throws IOException { }
+    public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
+      return null;  //To change body of implemented methods use File | Settings | File Templates.
+    }
 
     @Override
-    public void addColumn(byte[] tableName, HColumnDescriptor column) throws IOException { }
+    public List<HTableDescriptor> getTableDescriptorsByNamespace(String name) throws IOException {
+      return null;  //To change body of implemented methods use File | Settings | File Templates.
+    }
 
     @Override
-    public void modifyColumn(byte[] tableName, HColumnDescriptor descriptor) throws IOException { }
+    public void deleteTable(TableName tableName) throws IOException { }
 
     @Override
-    public void deleteColumn(byte[] tableName, byte[] columnName) throws IOException { }
+    public void modifyTable(TableName tableName, HTableDescriptor descriptor)
+        throws IOException { }
+
+    @Override
+    public void enableTable(TableName tableName) throws IOException { }
+
+    @Override
+    public void disableTable(TableName tableName) throws IOException { }
+
+    @Override
+    public void addColumn(TableName tableName, HColumnDescriptor column)
+        throws IOException { }
+
+    @Override
+    public void modifyColumn(TableName tableName, HColumnDescriptor descriptor)
+        throws IOException { }
+
+    @Override
+    public void deleteColumn(TableName tableName, byte[] columnName)
+        throws IOException { }
 
     @Override
     public TableLockManager getTableLockManager() {
@@ -349,16 +385,16 @@ public class TestCatalogJanitor {
       MasterServices services = new MockMasterServices(server);
       CatalogJanitor janitor = new CatalogJanitor(server, services);
       // Create regions.
-      HTableDescriptor htd = new HTableDescriptor("table");
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("table"));
       htd.addFamily(new HColumnDescriptor("f"));
       HRegionInfo parent =
-        new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+        new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
             Bytes.toBytes("eee"));
       HRegionInfo splita =
-        new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+        new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
             Bytes.toBytes("ccc"));
       HRegionInfo splitb =
-        new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
+        new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
             Bytes.toBytes("eee"));
       // Test that when both daughter regions are in place, that we do not
       // remove the parent.
@@ -366,7 +402,7 @@ public class TestCatalogJanitor {
       // Add a reference under splitA directory so we don't clear out the parent.
       Path rootdir = services.getMasterFileSystem().getRootDir();
       Path tabledir =
-        HTableDescriptor.getTableDir(rootdir, htd.getName());
+        FSUtils.getTableDir(rootdir, htd.getTableName());
       Path storedir = HStore.getStoreHomedir(tabledir, splita,
           htd.getColumnFamilies()[0].getName());
       Reference ref = Reference.createTopReference(Bytes.toBytes("ccc"));
@@ -430,30 +466,30 @@ public class TestCatalogJanitor {
     // Create regions: aaa->{lastEndKey}, aaa->ccc, aaa->bbb, bbb->ccc, etc.
 
     // Parent
-    HRegionInfo parent = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+    HRegionInfo parent = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
       lastEndKey);
     // Sleep a second else the encoded name on these regions comes out
     // same for all with same start key and made in same second.
     Thread.sleep(1001);
 
     // Daughter a
-    HRegionInfo splita = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+    HRegionInfo splita = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
       Bytes.toBytes("ccc"));
     Thread.sleep(1001);
     // Make daughters of daughter a; splitaa and splitab.
-    HRegionInfo splitaa = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+    HRegionInfo splitaa = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
       Bytes.toBytes("bbb"));
-    HRegionInfo splitab = new HRegionInfo(htd.getName(), Bytes.toBytes("bbb"),
+    HRegionInfo splitab = new HRegionInfo(htd.getTableName(), Bytes.toBytes("bbb"),
       Bytes.toBytes("ccc"));
 
     // Daughter b
-    HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
+    HRegionInfo splitb = new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
       lastEndKey);
     Thread.sleep(1001);
     // Make Daughters of daughterb; splitba and splitbb.
-    HRegionInfo splitba = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
+    HRegionInfo splitba = new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
       Bytes.toBytes("ddd"));
-    HRegionInfo splitbb = new HRegionInfo(htd.getName(), Bytes.toBytes("ddd"),
+    HRegionInfo splitbb = new HRegionInfo(htd.getTableName(), Bytes.toBytes("ddd"),
     lastEndKey);
 
     // First test that our Comparator works right up in CatalogJanitor.
@@ -532,24 +568,24 @@ public class TestCatalogJanitor {
     // Create regions: aaa->{lastEndKey}, aaa->ccc, aaa->bbb, bbb->ccc, etc.
 
     // Parent
-    HRegionInfo parent = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+    HRegionInfo parent = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
       new byte[0], true);
     // Sleep a second else the encoded name on these regions comes out
     // same for all with same start key and made in same second.
     Thread.sleep(1001);
 
     // Daughter a
-    HRegionInfo splita = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+    HRegionInfo splita = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
       Bytes.toBytes("ccc"), true);
     Thread.sleep(1001);
     // Make daughters of daughter a; splitaa and splitab.
-    HRegionInfo splitaa = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+    HRegionInfo splitaa = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
       Bytes.toBytes("bbb"), false);
-    HRegionInfo splitab = new HRegionInfo(htd.getName(), Bytes.toBytes("bbb"),
+    HRegionInfo splitab = new HRegionInfo(htd.getTableName(), Bytes.toBytes("bbb"),
       Bytes.toBytes("ccc"), false);
 
     // Daughter b
-    HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
+    HRegionInfo splitb = new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
         new byte[0]);
     Thread.sleep(1001);
 
@@ -611,11 +647,14 @@ public class TestCatalogJanitor {
      */
 
     // root region
-    HRegionInfo rootRegion = new HRegionInfo(htd.getName(), HConstants.EMPTY_START_ROW,
+    HRegionInfo rootRegion = new HRegionInfo(htd.getTableName(),
+      HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW, true);
-    HRegionInfo firstRegion = new HRegionInfo(htd.getName(), HConstants.EMPTY_START_ROW,
+    HRegionInfo firstRegion = new HRegionInfo(htd.getTableName(),
+      HConstants.EMPTY_START_ROW,
       Bytes.toBytes("bbb"), true);
-    HRegionInfo lastRegion = new HRegionInfo(htd.getName(), Bytes.toBytes("bbb"),
+    HRegionInfo lastRegion = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("bbb"),
       HConstants.EMPTY_END_ROW, true);
 
     assertTrue(comp.compare(rootRegion, rootRegion) == 0);
@@ -626,14 +665,18 @@ public class TestCatalogJanitor {
     assertTrue(comp.compare(firstRegion, lastRegion) < 0);
 
     //first region split into a, b
-    HRegionInfo firstRegiona = new HRegionInfo(htd.getName(), HConstants.EMPTY_START_ROW,
+    HRegionInfo firstRegiona = new HRegionInfo(htd.getTableName(),
+      HConstants.EMPTY_START_ROW,
       Bytes.toBytes("aaa"), true);
-    HRegionInfo firstRegionb = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
+    HRegionInfo firstRegionb = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("aaa"),
       Bytes.toBytes("bbb"), true);
     //last region split into a, b
-    HRegionInfo lastRegiona = new HRegionInfo(htd.getName(), Bytes.toBytes("bbb"),
+    HRegionInfo lastRegiona = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("bbb"),
       Bytes.toBytes("ddd"), true);
-    HRegionInfo lastRegionb = new HRegionInfo(htd.getName(), Bytes.toBytes("ddd"),
+    HRegionInfo lastRegionb = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("ddd"),
       HConstants.EMPTY_END_ROW, true);
 
     assertTrue(comp.compare(firstRegiona, firstRegiona) == 0);
@@ -657,9 +700,11 @@ public class TestCatalogJanitor {
     assertTrue(comp.compare(firstRegionb, lastRegiona) < 0);
     assertTrue(comp.compare(firstRegionb, lastRegionb) < 0);
 
-    HRegionInfo lastRegionaa = new HRegionInfo(htd.getName(), Bytes.toBytes("bbb"),
+    HRegionInfo lastRegionaa = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("bbb"),
       Bytes.toBytes("ccc"), false);
-    HRegionInfo lastRegionab = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
+    HRegionInfo lastRegionab = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("ccc"),
       Bytes.toBytes("ddd"), false);
 
     assertTrue(comp.compare(lastRegiona, lastRegionaa) < 0);
@@ -680,11 +725,15 @@ public class TestCatalogJanitor {
     CatalogJanitor janitor = new CatalogJanitor(server, services);
 
     // Create regions.
-    HTableDescriptor htd = new HTableDescriptor(table);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
     htd.addFamily(new HColumnDescriptor("f"));
-    HRegionInfo parent = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
-    HRegionInfo splita = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
-    HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
+    HRegionInfo parent = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
+    HRegionInfo splita = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
+    HRegionInfo splitb = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("ccc"),
+        Bytes.toBytes("eee"));
 
     // Test that when both daughter regions are in place, that we do not
     // remove the parent.
@@ -695,7 +744,7 @@ public class TestCatalogJanitor {
     // archive directory. Otherwise, it just seems to pick the first root directory it can find (so
     // the single test passes, but when the full suite is run, things get borked).
     FSUtils.setRootDir(fs.getConf(), rootdir);
-    Path tabledir = HTableDescriptor.getTableDir(rootdir, htd.getName());
+    Path tabledir = FSUtils.getTableDir(rootdir, htd.getTableName());
     Path storedir = HStore.getStoreHomedir(tabledir, parent, htd.getColumnFamilies()[0].getName());
     Path storeArchive = HFileArchiveUtil.getStoreArchivePath(services.getConfiguration(), parent,
       tabledir, htd.getColumnFamilies()[0].getName());
@@ -760,11 +809,14 @@ public class TestCatalogJanitor {
     CatalogJanitor janitor = new CatalogJanitor(server, services);
 
     // Create regions.
-    HTableDescriptor htd = new HTableDescriptor(table);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
     htd.addFamily(new HColumnDescriptor("f"));
-    HRegionInfo parent = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
-    HRegionInfo splita = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
-    HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
+    HRegionInfo parent = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
+    HRegionInfo splita = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
+    HRegionInfo splitb = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
     // Test that when both daughter regions are in place, that we do not
     // remove the parent.
     Result r = createResult(parent, splita, splitb);
@@ -776,7 +828,7 @@ public class TestCatalogJanitor {
     // archive directory. Otherwise, it just seems to pick the first root directory it can find (so
     // the single test passes, but when the full suite is run, things get borked).
     FSUtils.setRootDir(fs.getConf(), rootdir);
-    Path tabledir = HTableDescriptor.getTableDir(rootdir, parent.getTableName());
+    Path tabledir = FSUtils.getTableDir(rootdir, parent.getTableName());
     Path storedir = HStore.getStoreHomedir(tabledir, parent, htd.getColumnFamilies()[0].getName());
     System.out.println("Old root:" + rootdir);
     System.out.println("Old table:" + tabledir);
@@ -790,7 +842,6 @@ public class TestCatalogJanitor {
     addMockStoreFiles(2, services, storedir);
     // get the current store files for comparison
     FileStatus[] storeFiles = fs.listStatus(storedir);
-
     // do the cleaning of the parent
     assertTrue(janitor.cleanParent(parent, r));
 
@@ -859,7 +910,7 @@ public class TestCatalogJanitor {
       final HRegionInfo daughter, final byte [] midkey, final boolean top)
   throws IOException {
     Path rootdir = services.getMasterFileSystem().getRootDir();
-    Path tabledir = HTableDescriptor.getTableDir(rootdir, parent.getTableName());
+    Path tabledir = FSUtils.getTableDir(rootdir, parent.getTableName());
     Path storedir = HStore.getStoreHomedir(tabledir, daughter,
       htd.getColumnFamilies()[0].getName());
     Reference ref =
@@ -879,7 +930,7 @@ public class TestCatalogJanitor {
   }
 
   private HTableDescriptor createHTableDescriptor() {
-    HTableDescriptor htd = new HTableDescriptor("t");
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("t"));
     htd.addFamily(new HColumnDescriptor("f"));
     return htd;
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java Thu Aug  8 04:19:49 2013
@@ -53,6 +53,7 @@ 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.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -61,6 +62,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.SplitLogCounters;
 import org.apache.hadoop.hbase.Waiter;
@@ -83,14 +85,18 @@ import org.apache.hadoop.hbase.util.FSUt
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -107,13 +113,28 @@ public class TestDistributedLogSplitting
   }
 
   // Start a cluster with 2 masters and 6 regionservers
-  final int NUM_MASTERS = 2;
-  final int NUM_RS = 6;
+  static final int NUM_MASTERS = 2;
+  static final int NUM_RS = 6;
 
   MiniHBaseCluster cluster;
   HMaster master;
   Configuration conf;
-  HBaseTestingUtility TEST_UTIL;
+  static HBaseTestingUtility TEST_UTIL;
+  static MiniDFSCluster dfsCluster;
+  static MiniZooKeeperCluster zkCluster;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility(HBaseConfiguration.create());
+    dfsCluster = TEST_UTIL.startMiniDFSCluster(1);
+    zkCluster = TEST_UTIL.startMiniZKCluster();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    TEST_UTIL.shutdownMiniZKCluster();
+    TEST_UTIL.shutdownMiniDFSCluster();
+  }
 
   private void startCluster(int num_rs) throws Exception{
     conf = HBaseConfiguration.create();
@@ -130,7 +151,9 @@ public class TestDistributedLogSplitting
     conf.setInt(HConstants.REGIONSERVER_INFO_PORT, -1);
     conf.setFloat(HConstants.LOAD_BALANCER_SLOP_KEY, (float) 100.0); // no load balancing
     TEST_UTIL = new HBaseTestingUtility(conf);
-    TEST_UTIL.startMiniCluster(NUM_MASTERS, num_rs);
+    TEST_UTIL.setDFSCluster(dfsCluster);
+    TEST_UTIL.setZkCluster(zkCluster);
+    TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, num_rs);
     cluster = TEST_UTIL.getHBaseCluster();
     LOG.info("Waiting for active/ready master");
     cluster.waitForActiveAndReadyMaster();
@@ -146,7 +169,9 @@ public class TestDistributedLogSplitting
       mt.getMaster().abort("closing...", new Exception("Trace info"));
     }
 
-    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniHBaseCluster();
+    TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
+    ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
   }
 
   @Test (timeout=300000)
@@ -169,7 +194,7 @@ public class TestDistributedLogSplitting
 
     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
         "table", "family", 40);
-    byte[] table = Bytes.toBytes("table");
+    TableName table = TableName.valueOf("table");
     List<HRegionInfo> regions = null;
     HRegionServer hrs = null;
     for (int i = 0; i < NUM_RS; i++) {
@@ -177,7 +202,7 @@ public class TestDistributedLogSplitting
       hrs = rsts.get(i).getRegionServer();
       regions = ProtobufUtil.getOnlineRegions(hrs);
       for (HRegionInfo region : regions) {
-        if (region.getTableNameAsString().equalsIgnoreCase("table")) {
+        if (region.getTableName().getNameAsString().equalsIgnoreCase("table")) {
           foundRs = true;
           break;
         }
@@ -191,7 +216,8 @@ public class TestDistributedLogSplitting
     Iterator<HRegionInfo> it = regions.iterator();
     while (it.hasNext()) {
       HRegionInfo region = it.next();
-      if (region.isMetaTable()) {
+      if (region.getTableName().getNamespaceAsString()
+          .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
         it.remove();
       }
     }
@@ -202,7 +228,7 @@ public class TestDistributedLogSplitting
     int count = 0;
     for (HRegionInfo hri : regions) {
 
-      Path tdir = HTableDescriptor.getTableDir(rootdir, table);
+      Path tdir = FSUtils.getTableDir(rootdir, table);
       @SuppressWarnings("deprecation")
       Path editsdir =
         HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName()));
@@ -639,12 +665,13 @@ public class TestDistributedLogSplitting
           isCarryingMeta = true;
           break;
         }
-        if (tableName != null && !tableName.equalsIgnoreCase(region.getTableNameAsString())) {
+        if (tableName != null &&
+            !tableName.equalsIgnoreCase(region.getTableName().getNameAsString())) {
           // make sure that we find a RS has online regions for both "table" and "disableTable"
           hasRegionsForBothTables = true;
           break;
         } else if (tableName == null) {
-          tableName = region.getTableNameAsString();
+          tableName = region.getTableName().getNameAsString();
         }
       }
       if (isCarryingMeta) {
@@ -705,7 +732,7 @@ public class TestDistributedLogSplitting
     int count = 0;
     FileSystem fs = master.getMasterFileSystem().getFileSystem();
     Path rootdir = FSUtils.getRootDir(conf);
-    Path tdir = HTableDescriptor.getTableDir(rootdir, Bytes.toBytes("disableTable"));
+    Path tdir = FSUtils.getTableDir(rootdir, TableName.valueOf("disableTable"));
     for (HRegionInfo hri : regions) {
       @SuppressWarnings("deprecation")
       Path editsdir =
@@ -1059,19 +1086,19 @@ public class TestDistributedLogSplitting
     LOG.debug("Waiting for no more RIT\n");
     blockUntilNoRIT(zkw, master);
     NavigableSet<String> regions = getAllOnlineRegions(cluster);
-    LOG.debug("Verifying only catalog regions are assigned\n");
-    if (regions.size() != 1) {
+    LOG.debug("Verifying only catalog and namespace regions are assigned\n");
+    if (regions.size() != 2) {
       for (String oregion : regions)
         LOG.debug("Region still online: " + oregion);
     }
-    assertEquals(1 + existingRegions, regions.size());
+    assertEquals(2 + existingRegions, regions.size());
     LOG.debug("Enabling table\n");
     TEST_UTIL.getHBaseAdmin().enableTable(table);
     LOG.debug("Waiting for no more RIT\n");
     blockUntilNoRIT(zkw, master);
     LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
     regions = getAllOnlineRegions(cluster);
-    assertEquals(numRegions + 1 + existingRegions, regions.size());
+    assertEquals(numRegions + 2 + existingRegions, regions.size());
     return ht;
   }
 
@@ -1085,7 +1112,7 @@ public class TestDistributedLogSplitting
       HRegionServer hrs = rst.getRegionServer();
       List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs);
       for (HRegionInfo hri : hris) {
-        if (hri.isMetaTable()) {
+        if (HTableDescriptor.isSystemTable(hri.getTableName())) {
           continue;
         }
         LOG.debug("adding data to rs = " + rst.getName() +
@@ -1104,16 +1131,22 @@ public class TestDistributedLogSplitting
 
   public void makeHLog(HLog log, List<HRegionInfo> regions, String tname, String fname,
       int num_edits, int edit_size, boolean closeLog) throws IOException {
-
+    TableName fullTName = TableName.valueOf(tname);
     // remove root and meta region
     regions.remove(HRegionInfo.FIRST_META_REGIONINFO);
-    byte[] table = Bytes.toBytes(tname);
-    HTableDescriptor htd = new HTableDescriptor(tname);
+
+    for(Iterator<HRegionInfo> iter = regions.iterator(); iter.hasNext(); ) {
+      HRegionInfo regionInfo = iter.next();
+      if(HTableDescriptor.isSystemTable(regionInfo.getTableName())) {
+         iter.remove();
+      }
+    }
+    HTableDescriptor htd = new HTableDescriptor(fullTName);
     byte[] value = new byte[edit_size];
 
     List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
     for (HRegionInfo region : regions) {
-      if (!region.getTableNameAsString().equalsIgnoreCase(tname)) {
+      if (!region.getTableName().getNameAsString().equalsIgnoreCase(tname)) {
         continue;
       }
       hris.add(region);
@@ -1139,7 +1172,7 @@ public class TestDistributedLogSplitting
         byte[] family = Bytes.toBytes(fname);
         byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
         e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
-        log.append(curRegionInfo, table, e, System.currentTimeMillis(), htd);
+        log.append(curRegionInfo, fullTName, e, System.currentTimeMillis(), htd);
         counts[i % n] += 1;
       }
     }
@@ -1286,7 +1319,7 @@ public class TestDistributedLogSplitting
         if (region.isMetaRegion()) {
           isCarryingMeta = true;
         }
-        if (tableName == null || region.getTableNameAsString().equalsIgnoreCase(tableName)) {
+        if (tableName == null || region.getTableName().getNameAsString().equals(tableName)) {
           foundTableRegion = true;
         }
         if (foundTableRegion && (isCarryingMeta || !hasMetaRegion)) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java Thu Aug  8 04:19:49 2013
@@ -43,7 +43,8 @@ import static org.junit.Assert.*;
 public class TestMaster {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final Log LOG = LogFactory.getLog(TestMaster.class);
-  private static final byte[] TABLENAME = Bytes.toBytes("TestMaster");
+  private static final TableName TABLENAME =
+      TableName.valueOf("TestMaster");
   private static final byte[] FAMILYNAME = Bytes.toBytes("fam");
   private static HBaseAdmin admin;
 
@@ -65,14 +66,12 @@ public class TestMaster {
     HMaster m = cluster.getMaster();
 
     HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME);
-    assertTrue(m.assignmentManager.getZKTable().isEnabledTable
-        (Bytes.toString(TABLENAME)));
+    assertTrue(m.assignmentManager.getZKTable().isEnabledTable(TABLENAME));
     TEST_UTIL.loadTable(ht, FAMILYNAME);
     ht.close();
 
     List<Pair<HRegionInfo, ServerName>> tableRegions =
-      MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(),
-          Bytes.toString(TABLENAME));
+      MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(), TABLENAME);
     LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions));
     assertEquals(1, tableRegions.size());
     assertArrayEquals(HConstants.EMPTY_START_ROW,
@@ -82,7 +81,7 @@ public class TestMaster {
 
     // Now trigger a split and stop when the split is in progress
     LOG.info("Splitting table");
-    TEST_UTIL.getHBaseAdmin().split(TABLENAME);
+    TEST_UTIL.getHBaseAdmin().split(TABLENAME.getName());
     LOG.info("Waiting for split result to be about to open");
     while (!m.assignmentManager.wasSplitHandlerCalled()) {
       Thread.sleep(100);
@@ -122,7 +121,8 @@ public class TestMaster {
 
   @Test
   public void testMoveThrowsUnknownRegionException() throws IOException {
-    byte[] tableName = Bytes.toBytes("testMoveThrowsUnknownRegionException");
+    TableName tableName =
+        TableName.valueOf("testMoveThrowsUnknownRegionException");
     HTableDescriptor htd = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor("value");
     htd.addFamily(hcd);
@@ -144,7 +144,7 @@ public class TestMaster {
   public void testMoveThrowsPleaseHoldException() throws IOException {
     byte[] tableName = Bytes.toBytes("testMoveThrowsPleaseHoldException");
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
-    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
     HColumnDescriptor hcd = new HColumnDescriptor("value");
     htd.addFamily(hcd);
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java Thu Aug  8 04:19:49 2013
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -188,7 +189,7 @@ public class TestMasterFailover {
     };
 
     byte [] enabledTable = Bytes.toBytes("enabledTable");
-    HTableDescriptor htdEnabled = new HTableDescriptor(enabledTable);
+    HTableDescriptor htdEnabled = new HTableDescriptor(TableName.valueOf(enabledTable));
     htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
 
     FileSystem filesystem = FileSystem.get(conf);
@@ -197,26 +198,26 @@ public class TestMasterFailover {
     // Write the .tableinfo
     fstd.createTableDescriptor(htdEnabled);
 
-    HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(), null, null);
+    HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getTableName(), null, null);
     createRegion(hriEnabled, rootdir, conf, htdEnabled);
 
     List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
 
-    byte [] disabledTable = Bytes.toBytes("disabledTable");
+    TableName disabledTable = TableName.valueOf("disabledTable");
     HTableDescriptor htdDisabled = new HTableDescriptor(disabledTable);
     htdDisabled.addFamily(new HColumnDescriptor(FAMILY));
     // Write the .tableinfo
     fstd.createTableDescriptor(htdDisabled);
-    HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
+    HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getTableName(), null, null);
     createRegion(hriDisabled, rootdir, conf, htdDisabled);
     List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
 
-    log("Regions in META have been created");
+    log("Regions in META and namespace have been created");
 
-    // at this point we only expect 2 regions to be assigned out (catalogs)
-    assertEquals(1, cluster.countServedRegions());
+    // at this point we only expect 3 regions to be assigned out (catalogs and namespace)
+    assertEquals(2, cluster.countServedRegions());
 
     // Let's just assign everything to first RS
     HRegionServer hrs = cluster.getRegionServer(0);
@@ -267,7 +268,7 @@ public class TestMasterFailover {
 
     // Disable the disabledTable in ZK
     ZKTable zktable = new ZKTable(zkw);
-    zktable.setDisabledTable(Bytes.toString(disabledTable));
+    zktable.setDisabledTable(disabledTable);
 
     /*
      *  ZK = OFFLINE
@@ -492,35 +493,36 @@ public class TestMasterFailover {
         TEST_UTIL.getRegionSplitStartKeys(Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), 30);
 
     byte [] enabledTable = Bytes.toBytes("enabledTable");
-    HTableDescriptor htdEnabled = new HTableDescriptor(enabledTable);
+    HTableDescriptor htdEnabled = new HTableDescriptor(TableName.valueOf(enabledTable));
     htdEnabled.addFamily(new HColumnDescriptor(FAMILY));
     FileSystem filesystem = FileSystem.get(conf);
     Path rootdir = FSUtils.getRootDir(conf);
     FSTableDescriptors fstd = new FSTableDescriptors(filesystem, rootdir);
     // Write the .tableinfo
     fstd.createTableDescriptor(htdEnabled);
-    HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(),
+    HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getTableName(),
         null, null);
     createRegion(hriEnabled, rootdir, conf, htdEnabled);
 
     List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
 
-    byte [] disabledTable = Bytes.toBytes("disabledTable");
+    TableName disabledTable =
+        TableName.valueOf("disabledTable");
     HTableDescriptor htdDisabled = new HTableDescriptor(disabledTable);
     htdDisabled.addFamily(new HColumnDescriptor(FAMILY));
     // Write the .tableinfo
     fstd.createTableDescriptor(htdDisabled);
-    HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
+    HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getTableName(), null, null);
     createRegion(hriDisabled, rootdir, conf, htdDisabled);
 
     List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
 
-    log("Regions in META have been created");
+    log("Regions in META and Namespace have been created");
 
-    // at this point we only expect 2 regions to be assigned out (catalogs)
-    assertEquals(1, cluster.countServedRegions());
+    // at this point we only expect 2 regions to be assigned out (catalogs and namespace  )
+    assertEquals(2, cluster.countServedRegions());
 
     // The first RS will stay online
     List<RegionServerThread> regionservers =
@@ -558,7 +560,7 @@ public class TestMasterFailover {
     log("Assignment completed");
 
     assertTrue(" Table must be enabled.", master.getAssignmentManager()
-        .getZKTable().isEnabledTable("enabledTable"));
+        .getZKTable().isEnabledTable(TableName.valueOf("enabledTable")));
     // we also need regions assigned out on the dead server
     List<HRegionInfo> enabledAndOnDeadRegions = new ArrayList<HRegionInfo>();
     enabledAndOnDeadRegions.addAll(enabledRegions.subList(0, 6));
@@ -619,10 +621,10 @@ public class TestMasterFailover {
 
     // Disable the disabledTable in ZK
     ZKTable zktable = new ZKTable(zkw);
-    zktable.setDisabledTable(Bytes.toString(disabledTable));
+    zktable.setDisabledTable(disabledTable);
 
     assertTrue(" The enabled table should be identified on master fail over.",
-        zktable.isEnabledTable("enabledTable"));
+        zktable.isEnabledTable(TableName.valueOf("enabledTable")));
 
     /*
      * ZK = CLOSING

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java Thu Aug  8 04:19:49 2013
@@ -88,7 +88,7 @@ public class TestMasterMetrics {
   public void testDefaultMasterMetrics() throws Exception {
     MetricsMasterSource masterSource = master.getMetrics().getMetricsSource();
     metricsHelper.assertGauge( "numRegionServers", 1, masterSource);
-    metricsHelper.assertGauge( "averageLoad", 1, masterSource);
+    metricsHelper.assertGauge( "averageLoad", 2, masterSource);
     metricsHelper.assertGauge( "numDeadRegionServers", 0, masterSource);
 
     metricsHelper.assertGauge("masterStartTime", master.getMasterStartTime(), masterSource);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java Thu Aug  8 04:19:49 2013
@@ -30,6 +30,7 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -43,8 +44,8 @@ import org.apache.hadoop.hbase.catalog.M
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
@@ -52,7 +53,6 @@ import org.apache.hadoop.hbase.zookeeper
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.zookeeper.KeeperException;
-import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -156,7 +156,7 @@ public class TestMasterNoCluster {
     // Put some data into the servers.  Make it look like sn0 has the metaH
     // Put data into sn2 so it looks like it has a few regions for a table named 't'.
     MetaRegionTracker.setMetaLocation(rs0.getZooKeeper(), rs0.getServerName());
-    final byte [] tableName = Bytes.toBytes("t");
+    final TableName tableName = TableName.valueOf("t");
     Result [] results = new Result [] {
       MetaMockingUtil.getMetaTableRowResult(
         new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HBaseTestingUtility.KEYS[1]),
@@ -209,6 +209,11 @@ public class TestMasterNoCluster {
             rs0, rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
         return new CatalogTracker(zk, conf, connection, abortable);
       }
+
+      @Override
+      void assignSystemTables(MonitoredTask status)
+          throws IOException, InterruptedException, KeeperException {
+      }
     };
     master.start();
 
@@ -243,9 +248,9 @@ public class TestMasterNoCluster {
    * @throws DeserializationException
    * @throws ServiceException
    */
-  @Test (timeout=30000)
+  @Test (timeout=60000)
   public void testCatalogDeploys()
-  throws IOException, KeeperException, InterruptedException, DeserializationException, ServiceException {
+      throws Exception {
     final Configuration conf = TESTUTIL.getConfiguration();
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 1);
@@ -295,6 +300,11 @@ public class TestMasterNoCluster {
             rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
         return new CatalogTracker(zk, conf, connection, abortable);
       }
+
+      @Override
+      void assignSystemTables(MonitoredTask status)
+          throws IOException, InterruptedException, KeeperException {
+      }
     };
     master.start();
     LOG.info("Master has started");
@@ -388,6 +398,11 @@ public class TestMasterNoCluster {
             rs0, rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
         return new CatalogTracker(zk, conf, connection, abortable);
       }
+
+      @Override
+      void assignSystemTables(MonitoredTask status)
+          throws IOException, InterruptedException, KeeperException {
+      }
     };
     master.start();
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java Thu Aug  8 04:19:49 2013
@@ -83,7 +83,7 @@ public class TestMasterRestartAfterDisab
     NavigableSet<String> regions = getAllOnlineRegions(cluster);
     assertEquals(
         "The number of regions for the table tableRestart should be 0 and only"
-            + "the catalog tables should be present.", 1, regions.size());
+            + "the catalog and namespace tables should be present.", 2, regions.size());
 
     List<MasterThread> masterThreads = cluster.getMasterThreads();
     MasterThread activeMaster = null;
@@ -99,7 +99,7 @@ public class TestMasterRestartAfterDisab
 
     assertTrue("The table should not be in enabled state", cluster.getMaster()
         .getAssignmentManager().getZKTable().isDisablingOrDisabledTable(
-            "tableRestart"));
+            TableName.valueOf("tableRestart")));
     log("Enabling table\n");
     // Need a new Admin, the previous one is on the old master
     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
@@ -110,10 +110,11 @@ public class TestMasterRestartAfterDisab
     log("Verifying there are " + numRegions + " assigned on cluster\n");
     regions = getAllOnlineRegions(cluster);
     assertEquals(
-        "The assigned regions were not onlined after master switch except for the catalog tables.",
-        5, regions.size());
+        "The assigned regions were not onlined after master switch except for the catalog and namespace tables.",
+        6, regions.size());
     assertTrue("The table should be in enabled state", cluster.getMaster()
-        .getAssignmentManager().getZKTable().isEnabledTable("tableRestart"));
+        .getAssignmentManager().getZKTable()
+        .isEnabledTable(TableName.valueOf("tableRestart")));
     ht.close();
     TEST_UTIL.shutdownMiniCluster();
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java Thu Aug  8 04:19:49 2013
@@ -58,9 +58,10 @@ public class TestMasterStatusServlet {
   static final ServerName FAKE_HOST = 
     new ServerName("fakehost", 12345, 1234567890);
   static final HTableDescriptor FAKE_TABLE =
-    new HTableDescriptor("mytable");
+    new HTableDescriptor(TableName.valueOf("mytable"));
   static final HRegionInfo FAKE_HRI =
-      new HRegionInfo(FAKE_TABLE.getName(), Bytes.toBytes("a"), Bytes.toBytes("b"));
+      new HRegionInfo(FAKE_TABLE.getTableName(),
+          Bytes.toBytes("a"), Bytes.toBytes("b"));
 
   @Before
   public void setupBasicMocks() {
@@ -106,8 +107,8 @@ public class TestMasterStatusServlet {
 
   private void setupMockTables() throws IOException {
     HTableDescriptor tables[] = new HTableDescriptor[] {
-        new HTableDescriptor("foo"),
-        new HTableDescriptor("bar")
+        new HTableDescriptor(TableName.valueOf("foo")),
+        new HTableDescriptor(TableName.valueOf("bar"))
     };
     Mockito.doReturn(tables).when(admin).listTables();
   }
@@ -158,7 +159,7 @@ public class TestMasterStatusServlet {
     NavigableMap<String, RegionState> regionsInTransition =
       Maps.newTreeMap();
     for (byte i = 0; i < 100; i++) {
-      HRegionInfo hri = new HRegionInfo(FAKE_TABLE.getName(),
+      HRegionInfo hri = new HRegionInfo(FAKE_TABLE.getTableName(),
           new byte[]{i}, new byte[]{(byte) (i+1)});
       regionsInTransition.put(hri.getEncodedName(),
         new RegionState(hri, RegionState.State.CLOSING, 12345L, FAKE_HOST));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java Thu Aug  8 04:19:49 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -61,7 +62,7 @@ public class TestMasterTransitions {
     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
     TEST_UTIL.startMiniCluster(2);
     // Create a table of three families.  This will assign a region.
-    byte[] tableName = Bytes.toBytes(TABLENAME);
+    TableName tableName = TableName.valueOf(TABLENAME);
     TEST_UTIL.createTable(tableName, FAMILIES);
     HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
     int countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily());
@@ -480,7 +481,7 @@ public class TestMasterTransitions {
   private static int addToEachStartKey(final int expected) throws IOException {
     HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
     HTable meta = new HTable(TEST_UTIL.getConfiguration(),
-        HConstants.META_TABLE_NAME);
+        TableName.META_TABLE_NAME);
     int rows = 0;
     Scan scan = new Scan();
     scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
@@ -488,8 +489,14 @@ public class TestMasterTransitions {
     for (Result r = null; (r = s.next()) != null;) {
       HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
       if (hri == null) break;
+      if (!hri.getTableName().getNameAsString().equals(TABLENAME)) {
+        continue;
+      }
 
       // If start key, add 'aaa'.
+      if(!hri.getTableName().getNameAsString().equals(TABLENAME)) {
+        continue;
+      }
       byte [] row = getStartKey(hri);
       Put p = new Put(row);
       p.setDurability(Durability.SKIP_WAL);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java Thu Aug  8 04:19:49 2013
@@ -117,8 +117,8 @@ public class TestOpenedRegionHandler {
       TEST_UTIL.startMiniZKCluster();
       final Server server = new MockServer(TEST_UTIL);
       HTableDescriptor htd = new HTableDescriptor(
-          "testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches");
-      HRegionInfo hri = new HRegionInfo(htd.getName(),
+          TableName.valueOf("testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches"));
+      HRegionInfo hri = new HRegionInfo(htd.getTableName(),
           Bytes.toBytes(testIndex), Bytes.toBytes(testIndex + 1));
       region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
       assertNotNull(region);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java Thu Aug  8 04:19:49 2013
@@ -35,6 +35,7 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -100,7 +102,7 @@ public class TestRegionPlacement {
       servers.add(server);
     }
     List<HRegionInfo> regions = new ArrayList<HRegionInfo>(1);
-    HRegionInfo region = new HRegionInfo(("foobar").getBytes());
+    HRegionInfo region = new HRegionInfo(TableName.valueOf("foobar"));
     regions.add(region);
     Map<ServerName,List<HRegionInfo>> assignmentMap = balancer.roundRobinAssignment(regions,
         servers);
@@ -160,7 +162,7 @@ public class TestRegionPlacement {
       servers.add(server);
     }
     List<HRegionInfo> regions = new ArrayList<HRegionInfo>(1);
-    HRegionInfo region = new HRegionInfo(("foobar").getBytes());
+    HRegionInfo region = new HRegionInfo(TableName.valueOf("foobar"));
     regions.add(region);
     ServerName serverBefore = balancer.randomAssignment(region, servers);
     List<ServerName> favoredNodesBefore =
@@ -256,7 +258,8 @@ public class TestRegionPlacement {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     for (int i = 0; i < SLAVES; i++) {
       HRegionServer rs = cluster.getRegionServer(i);
-      for (HRegion region: rs.getOnlineRegions(Bytes.toBytes("testRegionAssignment"))) {
+      for (HRegion region: rs.getOnlineRegions(
+          TableName.valueOf("testRegionAssignment"))) {
         InetSocketAddress[] favoredSocketAddress = rs.getFavoredNodesForRegion(
             region.getRegionInfo().getEncodedName());
         ServerName[] favoredServerList = favoredNodesAssignmentPlan.get(region.getRegionInfo());
@@ -268,7 +271,7 @@ public class TestRegionPlacement {
           // Verify they are ROOT and META regions since no favored nodes
           assertNull(favoredSocketAddress);
           assertTrue("User region " +
-              region.getTableDesc().getNameAsString() +
+              region.getTableDesc().getTableName() +
               " should have favored nodes",
               (desc.isRootRegion() || desc.isMetaRegion()));
         } else {
@@ -310,6 +313,10 @@ public class TestRegionPlacement {
       public boolean processRow(Result result) throws IOException {
         try {
           HRegionInfo info = MetaScanner.getHRegionInfo(result);
+          if(info.getTableName().getNamespaceAsString()
+              .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
+            return true;
+          }
           byte[] server = result.getValue(HConstants.CATALOG_FAMILY,
               HConstants.SERVER_QUALIFIER);
           byte[] favoredNodes = result.getValue(HConstants.CATALOG_FAMILY,
@@ -381,7 +388,7 @@ public class TestRegionPlacement {
       splitKeys[i - 1] = new byte[] { splitKey, splitKey, splitKey };
     }
 
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, splitKeys);
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java Thu Aug  8 04:19:49 2013
@@ -102,7 +102,7 @@ public class TestRestartCluster {
 
     List<HRegionInfo> allRegions =
       MetaScanner.listAllRegions(UTIL.getConfiguration(), true);
-    assertEquals(3, allRegions.size());
+    assertEquals(4, allRegions.size());
 
     LOG.info("\n\nShutting down cluster");
     UTIL.shutdownMiniHBaseCluster();
@@ -116,10 +116,8 @@ public class TestRestartCluster {
     // Need to use a new 'Configuration' so we make a new HConnection.
     // Otherwise we're reusing an HConnection that has gone stale because
     // the shutdown of the cluster also called shut of the connection.
-    allRegions = MetaScanner.
-      listAllRegions(new Configuration(UTIL.getConfiguration()), true);
-    assertEquals(3, allRegions.size());
-
+    allRegions = MetaScanner.listAllRegions(new Configuration(UTIL.getConfiguration()), true);
+    assertEquals(4, allRegions.size());
     LOG.info("\n\nWaiting for tables to be available");
     for(byte [] TABLE: TABLES) {
       try {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java Thu Aug  8 04:19:49 2013
@@ -87,11 +87,11 @@ public class  TestRollingRestart {
     log("Waiting for no more RIT\n");
     blockUntilNoRIT(zkw, master);
     NavigableSet<String> regions = getAllOnlineRegions(cluster);
-    log("Verifying only catalog regions are assigned\n");
-    if (regions.size() != 1) {
+    log("Verifying only catalog and namespace regions are assigned\n");
+    if (regions.size() != 2) {
       for (String oregion : regions) log("Region still online: " + oregion);
     }
-    assertEquals(1, regions.size());
+    assertEquals(2, regions.size());
     log("Enabling table\n");
     TEST_UTIL.getHBaseAdmin().enableTable(table);
     log("Waiting for no more RIT\n");

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java Thu Aug  8 04:19:49 2013
@@ -38,6 +38,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Chore;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -72,7 +73,8 @@ public class TestTableLockManager {
   private static final Log LOG =
     LogFactory.getLog(TestTableLockManager.class);
 
-  private static final byte[] TABLE_NAME = Bytes.toBytes("TestTableLevelLocks");
+  private static final TableName TABLE_NAME =
+      TableName.valueOf("TestTableLevelLocks");
 
   private static final byte[] FAMILY = Bytes.toBytes("f1");
 
@@ -133,18 +135,18 @@ public class TestTableLockManager {
   public static class TestLockTimeoutExceptionMasterObserver extends BaseMasterObserver {
     @Override
     public void preDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        byte[] tableName, byte[] c) throws IOException {
+        TableName tableName, byte[] c) throws IOException {
       deleteColumn.countDown();
     }
     @Override
     public void postDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        byte[] tableName, byte[] c) throws IOException {
+        TableName tableName, byte[] c) throws IOException {
       Threads.sleep(10000);
     }
 
     @Override
     public void preAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        byte[] tableName, HColumnDescriptor column) throws IOException {
+        TableName tableName, HColumnDescriptor column) throws IOException {
       fail("Add column should have timeouted out for acquiring the table lock");
     }
   }
@@ -198,14 +200,14 @@ public class TestTableLockManager {
   public static class TestAlterAndDisableMasterObserver extends BaseMasterObserver {
     @Override
     public void preAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        byte[] tableName, HColumnDescriptor column) throws IOException {
+        TableName tableName, HColumnDescriptor column) throws IOException {
       LOG.debug("addColumn called");
       addColumn.countDown();
     }
 
     @Override
     public void postAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        byte[] tableName, HColumnDescriptor column) throws IOException {
+        TableName tableName, HColumnDescriptor column) throws IOException {
       Threads.sleep(6000);
       try {
         ctx.getEnvironment().getMasterServices().checkTableModifiable(tableName);
@@ -219,7 +221,7 @@ public class TestTableLockManager {
 
     @Override
     public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        byte[] tableName) throws IOException {
+                                TableName tableName) throws IOException {
       try {
         LOG.debug("Waiting for addColumn to be processed first");
         //wait for addColumn to be processed first
@@ -232,7 +234,7 @@ public class TestTableLockManager {
 
     @Override
     public void postDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        byte[] tableName) throws IOException {
+                                        TableName tableName) throws IOException {
       Threads.sleep(3000);
     }
   }
@@ -247,7 +249,7 @@ public class TestTableLockManager {
 
     //ensure that znode for the table node has been deleted
     final ZooKeeperWatcher zkWatcher = TEST_UTIL.getZooKeeperWatcher();
-    final String znode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, Bytes.toString(TABLE_NAME));
+    final String znode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, TABLE_NAME.getNameAsString());
     
     TEST_UTIL.waitFor(5000, new Waiter.Predicate<Exception>() {
       @Override
@@ -257,7 +259,7 @@ public class TestTableLockManager {
       }
     });
     int ver = ZKUtil.checkExists(zkWatcher,
-      ZKUtil.joinZNode(zkWatcher.tableLockZNode, Bytes.toString(TABLE_NAME)));
+      ZKUtil.joinZNode(zkWatcher.tableLockZNode, TABLE_NAME.getNameAsString()));
     assertTrue("Unexpected znode version " + ver, ver < 0);
 
   }
@@ -285,7 +287,8 @@ public class TestTableLockManager {
           @Override
           public Void call() throws Exception {
             writeLocksAttempted.countDown();
-            lockManager.writeLock(Bytes.toBytes(table), "testReapAllTableLocks").acquire();
+            lockManager.writeLock(TableName.valueOf(table),
+                "testReapAllTableLocks").acquire();
             writeLocksObtained.countDown();
             return null;
           }
@@ -304,7 +307,9 @@ public class TestTableLockManager {
           TEST_UTIL.getConfiguration(), TEST_UTIL.getZooKeeperWatcher(), serverName);
 
     //should not throw table lock timeout exception
-    zeroTimeoutLockManager.writeLock(Bytes.toBytes(tables[tables.length -1]), "zero timeout")
+    zeroTimeoutLockManager.writeLock(
+        TableName.valueOf(tables[tables.length - 1]),
+        "zero timeout")
       .acquire();
 
     executor.shutdownNow();
@@ -321,7 +326,7 @@ public class TestTableLockManager {
     LoadTestTool loadTool = new LoadTestTool();
     loadTool.setConf(TEST_UTIL.getConfiguration());
     int numKeys = 10000;
-    final byte[] tableName = Bytes.toBytes("testTableReadLock");
+    final TableName tableName = TableName.valueOf("testTableReadLock");
     final HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     final HTableDescriptor desc = new HTableDescriptor(tableName);
     final byte[] family = Bytes.toBytes("test_cf");
@@ -329,7 +334,7 @@ public class TestTableLockManager {
     admin.createTable(desc); // create with one region
 
     // write some data, not much
-    int ret = loadTool.run(new String[] { "-tn", Bytes.toString(tableName), "-write",
+    int ret = loadTool.run(new String[] { "-tn", tableName.getNameAsString(), "-write",
         String.format("%d:%d:%d", 1, 10, 10), "-num_keys", String.valueOf(numKeys), "-skip_init" });
     if (0 != ret) {
       String errorMsg = "Load failed with error code " + ret;
@@ -406,7 +411,7 @@ public class TestTableLockManager {
     assertTrue(newFamilyValues > familyValues); // at least one alter went
                                                 // through
 
-    ret = loadTool.run(new String[] { "-tn", Bytes.toString(tableName), "-read", "100:10",
+    ret = loadTool.run(new String[] { "-tn", tableName.getNameAsString(), "-read", "100:10",
         "-num_keys", String.valueOf(numKeys), "-skip_init" });
     if (0 != ret) {
       String errorMsg = "Verify failed with error code " + ret;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestZKBasedOpenCloseRegion.java Thu Aug  8 04:19:49 2013
@@ -30,6 +30,7 @@ import java.util.Collection;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -62,7 +63,8 @@ import org.mockito.internal.util.reflect
 public class TestZKBasedOpenCloseRegion {
   private static final Log LOG = LogFactory.getLog(TestZKBasedOpenCloseRegion.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final String TABLENAME = "TestZKBasedOpenCloseRegion";
+  private static final TableName TABLENAME =
+      TableName.valueOf("TestZKBasedOpenCloseRegion");
   private static final byte [][] FAMILIES = new byte [][] {Bytes.toBytes("a"),
     Bytes.toBytes("b"), Bytes.toBytes("c")};
   private static int countOfRegions;
@@ -72,7 +74,7 @@ public class TestZKBasedOpenCloseRegion 
     c.setBoolean("dfs.support.append", true);
     c.setInt("hbase.regionserver.info.port", 0);
     TEST_UTIL.startMiniCluster(2);
-    TEST_UTIL.createTable(Bytes.toBytes(TABLENAME), FAMILIES);
+    TEST_UTIL.createTable(TABLENAME, FAMILIES);
     HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
     countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily());
     waitUntilAllRegionsAssigned();
@@ -219,13 +221,13 @@ public class TestZKBasedOpenCloseRegion 
    */
   @Test
   public void testRegionOpenFailsDueToIOException() throws Exception {
-    HRegionInfo REGIONINFO = new HRegionInfo(Bytes.toBytes("t"),
+    HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("t"),
         HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
     HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(0);
     TableDescriptors htd = Mockito.mock(TableDescriptors.class);
     Object orizinalState = Whitebox.getInternalState(regionServer,"tableDescriptors");
     Whitebox.setInternalState(regionServer, "tableDescriptors", htd);
-    Mockito.doThrow(new IOException()).when(htd).get((byte[]) Mockito.any());
+    Mockito.doThrow(new IOException()).when(htd).get((TableName) Mockito.any());
     try {
       ProtobufUtil.openRegion(regionServer, REGIONINFO);
       fail("It should throw IOException ");
@@ -238,7 +240,7 @@ public class TestZKBasedOpenCloseRegion 
 
   private static void waitUntilAllRegionsAssigned()
   throws IOException {
-    HTable meta = new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
+    HTable meta = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
     while (true) {
       int rows = 0;
       Scan scan = new Scan();
@@ -273,7 +275,7 @@ public class TestZKBasedOpenCloseRegion 
   private static int addToEachStartKey(final int expected) throws IOException {
     HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
     HTable meta = new HTable(TEST_UTIL.getConfiguration(),
-        HConstants.META_TABLE_NAME);
+        TableName.META_TABLE_NAME);
     int rows = 0;
     Scan scan = new Scan();
     scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
@@ -281,7 +283,9 @@ public class TestZKBasedOpenCloseRegion 
     for (Result r = null; (r = s.next()) != null;) {
       HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
       if (hri == null) break;
-
+      if(!hri.getTableName().equals(TABLENAME)) {
+        continue;
+      }
       // If start key, add 'aaa'.
       byte [] row = getStartKey(hri);
       Put p = new Put(row);