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 [23/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/util/TestHBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java Thu Aug  8 04:19:49 2013
@@ -31,6 +31,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -49,6 +50,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -66,6 +68,7 @@ import org.apache.hadoop.hbase.client.HB
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.MetaScanner;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -152,32 +155,37 @@ public class TestHBaseFsck {
 
     // Now let's mess it up and change the assignment in .META. to
     // point to a different region server
-    HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName(), executorService);
-    ResultScanner scanner = meta.getScanner(new Scan());
+    HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName(),
+        executorService);
+    Scan scan = new Scan();
+    scan.setStartRow(Bytes.toBytes(table+",,"));
+    ResultScanner scanner = meta.getScanner(scan);
     HRegionInfo hri = null;
 
-    resforloop:
-    for (Result res : scanner) {
-      long startCode = Bytes.toLong(res.getValue(HConstants.CATALOG_FAMILY,
-          HConstants.STARTCODE_QUALIFIER));
-
-      for (JVMClusterUtil.RegionServerThread rs :
-          TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
-
-        ServerName sn = rs.getRegionServer().getServerName();
-
-        // When we find a diff RS, change the assignment and break
-        if (startCode != sn.getStartcode()) {
-          Put put = new Put(res.getRow());
-          put.setDurability(Durability.SKIP_WAL);
-          put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
-            Bytes.toBytes(sn.getHostAndPort()));
-          put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
-            Bytes.toBytes(sn.getStartcode()));
-          meta.put(put);
-          hri = HRegionInfo.getHRegionInfo(res);
-          break resforloop;
-        }
+    Result res = scanner.next();
+    ServerName currServer =
+      ServerName.parseFrom(res.getValue(HConstants.CATALOG_FAMILY,
+          HConstants.SERVER_QUALIFIER));
+    long startCode = Bytes.toLong(res.getValue(HConstants.CATALOG_FAMILY,
+        HConstants.STARTCODE_QUALIFIER));
+
+    for (JVMClusterUtil.RegionServerThread rs :
+        TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
+
+      ServerName sn = rs.getRegionServer().getServerName();
+
+      // When we find a diff RS, change the assignment and break
+      if (!currServer.getHostAndPort().equals(sn.getHostAndPort()) ||
+          startCode != sn.getStartcode()) {
+        Put put = new Put(res.getRow());
+        put.setDurability(Durability.SKIP_WAL);
+        put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
+          Bytes.toBytes(sn.getHostAndPort()));
+        put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
+          Bytes.toBytes(sn.getStartcode()));
+        meta.put(put);
+        hri = HRegionInfo.getHRegionInfo(res);
+        break;
       }
     }
 
@@ -207,8 +215,8 @@ public class TestHBaseFsck {
   private HRegionInfo createRegion(Configuration conf, final HTableDescriptor
       htd, byte[] startKey, byte[] endKey)
       throws IOException {
-    HTable meta = new HTable(conf, HConstants.META_TABLE_NAME, executorService);
-    HRegionInfo hri = new HRegionInfo(htd.getName(), startKey, endKey);
+    HTable meta = new HTable(conf, TableName.META_TABLE_NAME, executorService);
+    HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKey, endKey);
     MetaEditor.addRegionToMeta(meta, hri);
     meta.close();
     return hri;
@@ -217,7 +225,7 @@ public class TestHBaseFsck {
   /**
    * Debugging method to dump the contents of meta.
    */
-  private void dumpMeta(byte[] tableName) throws IOException {
+  private void dumpMeta(TableName tableName) throws IOException {
     List<byte[]> metaRows = TEST_UTIL.getMetaTableRows(tableName);
     for (byte[] row : metaRows) {
       LOG.info(Bytes.toString(row));
@@ -263,7 +271,7 @@ public class TestHBaseFsck {
       byte[] startKey, byte[] endKey, boolean unassign, boolean metaRow,
       boolean hdfs, boolean regionInfoOnly) throws IOException, InterruptedException {
     LOG.info("** Before delete:");
-    dumpMeta(htd.getName());
+    dumpMeta(htd.getTableName());
 
     Map<HRegionInfo, ServerName> hris = tbl.getRegionLocations();
     for (Entry<HRegionInfo, ServerName> e: hris.entrySet()) {
@@ -284,7 +292,8 @@ public class TestHBaseFsck {
           LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString());
           Path rootDir = FSUtils.getRootDir(conf);
           FileSystem fs = rootDir.getFileSystem(conf);
-          Path p = new Path(rootDir + "/" + htd.getNameAsString(), hri.getEncodedName());
+          Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()),
+              hri.getEncodedName());
           Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE);
           fs.delete(hriPath, true);
         }
@@ -293,7 +302,8 @@ public class TestHBaseFsck {
           LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
           Path rootDir = FSUtils.getRootDir(conf);
           FileSystem fs = rootDir.getFileSystem(conf);
-          Path p = new Path(rootDir + "/" + htd.getNameAsString(), hri.getEncodedName());
+          Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()),
+              hri.getEncodedName());
           HBaseFsck.debugLsr(conf, p);
           boolean success = fs.delete(p, true);
           LOG.info("Deleted " + p + " sucessfully? " + success);
@@ -301,7 +311,7 @@ public class TestHBaseFsck {
         }
 
         if (metaRow) {
-          HTable meta = new HTable(conf, HConstants.META_TABLE_NAME, executorService);
+          HTable meta = new HTable(conf, TableName.META_TABLE_NAME, executorService);
           Delete delete = new Delete(deleteRow);
           meta.delete(delete);
         }
@@ -309,9 +319,9 @@ public class TestHBaseFsck {
       LOG.info(hri.toString() + hsa.toString());
     }
 
-    TEST_UTIL.getMetaTableRows(htd.getName());
+    TEST_UTIL.getMetaTableRows(htd.getTableName());
     LOG.info("*** After delete:");
-    dumpMeta(htd.getName());
+    dumpMeta(htd.getTableName());
   }
 
   /**
@@ -321,12 +331,12 @@ public class TestHBaseFsck {
    * @throws InterruptedException
    * @throws KeeperException
    */
-  HTable setupTable(String tablename) throws Exception {
+  HTable setupTable(TableName tablename) throws Exception {
     HTableDescriptor desc = new HTableDescriptor(tablename);
     HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
     desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
     TEST_UTIL.getHBaseAdmin().createTable(desc, SPLITS);
-    tbl = new HTable(TEST_UTIL.getConfiguration(), Bytes.toBytes(tablename), executorService);
+    tbl = new HTable(TEST_UTIL.getConfiguration(), tablename, executorService);
 
     List<Put> puts = new ArrayList<Put>();
     for (byte[] row : ROWKEYS) {
@@ -358,15 +368,14 @@ public class TestHBaseFsck {
    * @param tablename
    * @throws IOException
    */
-  void deleteTable(String tablename) throws IOException {
+  void deleteTable(TableName tablename) throws IOException {
     HBaseAdmin admin = new HBaseAdmin(conf);
     admin.getConnection().clearRegionCache();
-    byte[] tbytes = Bytes.toBytes(tablename);
-    admin.disableTableAsync(tbytes);
+    admin.disableTableAsync(tablename);
     long totalWait = 0;
     long maxWait = 30*1000;
     long sleepTime = 250;
-    while (!admin.isTableDisabled(tbytes)) {
+    while (!admin.isTableDisabled(tablename)) {
       try {
         Thread.sleep(sleepTime);
         totalWait += sleepTime;
@@ -378,7 +387,7 @@ public class TestHBaseFsck {
         fail("Interrupted when trying to disable table " + tablename);
       }
     }
-    admin.deleteTable(tbytes);
+    admin.deleteTable(tablename);
   }
 
   /**
@@ -387,7 +396,7 @@ public class TestHBaseFsck {
   @Test
   public void testHBaseFsckClean() throws Exception {
     assertNoErrors(doFsck(conf, false));
-    String table = "tableClean";
+    TableName table = TableName.valueOf("tableClean");
     try {
       HBaseFsck hbck = doFsck(conf, false);
       assertNoErrors(hbck);
@@ -410,7 +419,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testHbckThreadpooling() throws Exception {
-    String table = "tableDupeStartKey";
+    TableName table =
+        TableName.valueOf("tableDupeStartKey");
     try {
       // Create table with 4 regions
       setupTable(table);
@@ -428,15 +438,15 @@ public class TestHBaseFsck {
 
   @Test
   public void testHbckFixOrphanTable() throws Exception {
-    String table = "tableInfo";
+    TableName table = TableName.valueOf("tableInfo");
     FileSystem fs = null;
     Path tableinfo = null;
     try {
       setupTable(table);
       HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
 
-      Path hbaseTableDir = HTableDescriptor.getTableDir(
-        FSUtils.getRootDir(conf), Bytes.toBytes(table));
+      Path hbaseTableDir = FSUtils.getTableDir(
+          FSUtils.getRootDir(conf), table);
       fs = hbaseTableDir.getFileSystem(conf);
       FileStatus status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       tableinfo = status.getPath();
@@ -453,21 +463,21 @@ public class TestHBaseFsck {
       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       assertNotNull(status);
 
-      HTableDescriptor htd = admin.getTableDescriptor(table.getBytes());
+      HTableDescriptor htd = admin.getTableDescriptor(table);
       htd.setValue("NOT_DEFAULT", "true");
       admin.disableTable(table);
-      admin.modifyTable(table.getBytes(), htd);
+      admin.modifyTable(table, htd);
       admin.enableTable(table);
       fs.delete(status.getPath(), true);
 
       // fix OrphanTable with cache
-      htd = admin.getTableDescriptor(table.getBytes()); // warms up cached htd on master
+      htd = admin.getTableDescriptor(table); // warms up cached htd on master
       hbck = doFsck(conf, true);
       assertNoErrors(hbck);
       status = null;
       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       assertNotNull(status);
-      htd = admin.getTableDescriptor(table.getBytes());
+      htd = admin.getTableDescriptor(table);
       assertEquals(htd.getValue("NOT_DEFAULT"), "true");
     } finally {
       fs.rename(new Path("/.tableinfo"), tableinfo);
@@ -481,7 +491,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testDupeStartKey() throws Exception {
-    String table = "tableDupeStartKey";
+    TableName table =
+        TableName.valueOf("tableDupeStartKey");
     try {
       setupTable(table);
       assertNoErrors(doFsck(conf, false));
@@ -557,7 +568,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testDupeRegion() throws Exception {
-    String table = "tableDupeRegion";
+    TableName table =
+        TableName.valueOf("tableDupeRegion");
     try {
       setupTable(table);
       assertNoErrors(doFsck(conf, false));
@@ -609,7 +621,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testDegenerateRegions() throws Exception {
-    String table = "tableDegenerateRegions";
+    TableName table =
+        TableName.valueOf("tableDegenerateRegions");
     try {
       setupTable(table);
       assertNoErrors(doFsck(conf,false));
@@ -649,7 +662,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testContainedRegionOverlap() throws Exception {
-    String table = "tableContainedRegionOverlap";
+    TableName table =
+        TableName.valueOf("tableContainedRegionOverlap");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -690,7 +704,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testSidelineOverlapRegion() throws Exception {
-    String table = "testSidelineOverlapRegion";
+    TableName table =
+        TableName.valueOf("testSidelineOverlapRegion");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -742,7 +757,7 @@ public class TestHBaseFsck {
 
       assertNotNull(regionName);
       assertNotNull(serverName);
-      HTable meta = new HTable(conf, HConstants.META_TABLE_NAME, executorService);
+      HTable meta = new HTable(conf, TableName.META_TABLE_NAME, executorService);
       Put put = new Put(regionName);
       put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
         Bytes.toBytes(serverName.getHostAndPort()));
@@ -780,7 +795,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testOverlapAndOrphan() throws Exception {
-    String table = "tableOverlapAndOrphan";
+    TableName table =
+        TableName.valueOf("tableOverlapAndOrphan");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -824,7 +840,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testCoveredStartKey() throws Exception {
-    String table = "tableCoveredStartKey";
+    TableName table =
+        TableName.valueOf("tableCoveredStartKey");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -864,7 +881,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testRegionHole() throws Exception {
-    String table = "tableRegionHole";
+    TableName table =
+        TableName.valueOf("tableRegionHole");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -898,7 +916,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testHDFSRegioninfoMissing() throws Exception {
-    String table = "tableHDFSRegioininfoMissing";
+    TableName table =
+        TableName.valueOf("tableHDFSRegioininfoMissing");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -934,7 +953,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testNotInMetaOrDeployedHole() throws Exception {
-    String table = "tableNotInMetaOrDeployedHole";
+    TableName table =
+        TableName.valueOf("tableNotInMetaOrDeployedHole");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -968,7 +988,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testNotInMetaHole() throws Exception {
-    String table = "tableNotInMetaHole";
+    TableName table =
+        TableName.valueOf("tableNotInMetaHole");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -1003,13 +1024,14 @@ public class TestHBaseFsck {
    */
   @Test
   public void testNotInHdfs() throws Exception {
-    String table = "tableNotInHdfs";
+    TableName table =
+        TableName.valueOf("tableNotInHdfs");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
 
       // make sure data in regions, if in hlog only there is no data loss
-      TEST_UTIL.getHBaseAdmin().flush(table);
+      TEST_UTIL.getHBaseAdmin().flush(table.getName());
 
       // Mess it up by leaving a hole in the hdfs data
       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
@@ -1037,12 +1059,12 @@ public class TestHBaseFsck {
    */
   @Test
   public void testNoHdfsTable() throws Exception {
-    String table = "NoHdfsTable";
+    TableName table = TableName.valueOf("NoHdfsTable");
     setupTable(table);
     assertEquals(ROWKEYS.length, countRows());
 
     // make sure data in regions, if in hlog only there is no data loss
-    TEST_UTIL.getHBaseAdmin().flush(table);
+    TEST_UTIL.getHBaseAdmin().flush(table.getName());
 
     // Mess it up by leaving a giant hole in meta
     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes(""),
@@ -1096,7 +1118,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testRegionShouldNotBeDeployed() throws Exception {
-    String table = "tableRegionShouldNotBeDeployed";
+    TableName table =
+        TableName.valueOf("tableRegionShouldNotBeDeployed");
     try {
       LOG.info("Starting testRegionShouldNotBeDeployed.");
       MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
@@ -1105,7 +1128,7 @@ public class TestHBaseFsck {
 
       byte[][] SPLIT_KEYS = new byte[][] { new byte[0], Bytes.toBytes("aaa"),
           Bytes.toBytes("bbb"), Bytes.toBytes("ccc"), Bytes.toBytes("ddd") };
-      HTableDescriptor htdDisabled = new HTableDescriptor(Bytes.toBytes(table));
+      HTableDescriptor htdDisabled = new HTableDescriptor(table);
       htdDisabled.addFamily(new HColumnDescriptor(FAM));
 
       // Write the .tableinfo
@@ -1156,19 +1179,21 @@ public class TestHBaseFsck {
    */
   @Test
   public void testFixByTable() throws Exception {
-    String table1 = "testFixByTable1";
-    String table2 = "testFixByTable2";
+    TableName table1 =
+        TableName.valueOf("testFixByTable1");
+    TableName table2 =
+        TableName.valueOf("testFixByTable2");
     try {
       setupTable(table1);
       // make sure data in regions, if in hlog only there is no data loss
-      TEST_UTIL.getHBaseAdmin().flush(table1);
+      TEST_UTIL.getHBaseAdmin().flush(table1.getName());
       // Mess them up by leaving a hole in the hdfs data
       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
         Bytes.toBytes("C"), false, false, true); // don't rm meta
 
       setupTable(table2);
       // make sure data in regions, if in hlog only there is no data loss
-      TEST_UTIL.getHBaseAdmin().flush(table2);
+      TEST_UTIL.getHBaseAdmin().flush(table2.getName());
       // Mess them up by leaving a hole in the hdfs data
       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
         Bytes.toBytes("C"), false, false, true); // don't rm meta
@@ -1200,14 +1225,15 @@ public class TestHBaseFsck {
    */
   @Test
   public void testLingeringSplitParent() throws Exception {
-    String table = "testLingeringSplitParent";
+    TableName table =
+        TableName.valueOf("testLingeringSplitParent");
     HTable meta = null;
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
 
       // make sure data in regions, if in hlog only there is no data loss
-      TEST_UTIL.getHBaseAdmin().flush(table);
+      TEST_UTIL.getHBaseAdmin().flush(table.getName());
       HRegionLocation location = tbl.getRegionLocation("B");
 
       // Delete one region from meta, but not hdfs, unassign it.
@@ -1215,12 +1241,13 @@ public class TestHBaseFsck {
         Bytes.toBytes("C"), true, true, false);
 
       // Create a new meta entry to fake it as a split parent.
-      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName(), executorService);
+      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName(),
+          executorService);
       HRegionInfo hri = location.getRegionInfo();
 
-      HRegionInfo a = new HRegionInfo(tbl.getTableName(),
+      HRegionInfo a = new HRegionInfo(tbl.getName(),
         Bytes.toBytes("B"), Bytes.toBytes("BM"));
-      HRegionInfo b = new HRegionInfo(tbl.getTableName(),
+      HRegionInfo b = new HRegionInfo(tbl.getName(),
         Bytes.toBytes("BM"), Bytes.toBytes("C"));
 
       hri.setOffline(true);
@@ -1228,7 +1255,7 @@ public class TestHBaseFsck {
 
       MetaEditor.addRegionToMeta(meta, hri, a, b);
       meta.flushCommits();
-      TEST_UTIL.getHBaseAdmin().flush(HConstants.META_TABLE_NAME);
+      TEST_UTIL.getHBaseAdmin().flush(TableName.META_TABLE_NAME.getName());
 
       HBaseFsck hbck = doFsck(conf, false);
       assertErrors(hbck, new ERROR_CODE[] {
@@ -1258,7 +1285,7 @@ public class TestHBaseFsck {
         HConstants.SPLITA_QUALIFIER).isEmpty());
       assertTrue(result.getColumn(HConstants.CATALOG_FAMILY,
         HConstants.SPLITB_QUALIFIER).isEmpty());
-      TEST_UTIL.getHBaseAdmin().flush(HConstants.META_TABLE_NAME);
+      TEST_UTIL.getHBaseAdmin().flush(TableName.META_TABLE_NAME.getName());
 
       // fix other issues
       doFsck(conf, true);
@@ -1278,17 +1305,18 @@ public class TestHBaseFsck {
    */
   @Test
   public void testValidLingeringSplitParent() throws Exception {
-    String table = "testLingeringSplitParent";
+    TableName table =
+        TableName.valueOf("testLingeringSplitParent");
     HTable meta = null;
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
 
       // make sure data in regions, if in hlog only there is no data loss
-      TEST_UTIL.getHBaseAdmin().flush(table);
+      TEST_UTIL.getHBaseAdmin().flush(table.getName());
       HRegionLocation location = tbl.getRegionLocation("B");
 
-      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
+      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName());
       HRegionInfo hri = location.getRegionInfo();
 
       // do a regular split
@@ -1327,17 +1355,18 @@ public class TestHBaseFsck {
    */
   @Test
   public void testSplitDaughtersNotInMeta() throws Exception {
-    String table = "testSplitdaughtersNotInMeta";
+    TableName table =
+        TableName.valueOf("testSplitdaughtersNotInMeta");
     HTable meta = null;
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
 
       // make sure data in regions, if in hlog only there is no data loss
-      TEST_UTIL.getHBaseAdmin().flush(table);
+      TEST_UTIL.getHBaseAdmin().flush(table.getName());
       HRegionLocation location = tbl.getRegionLocation("B");
 
-      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
+      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName());
       HRegionInfo hri = location.getRegionInfo();
 
       // do a regular split
@@ -1390,7 +1419,8 @@ public class TestHBaseFsck {
    */
   @Test(timeout=120000)
   public void testMissingFirstRegion() throws Exception {
-    String table = "testMissingFirstRegion";
+    TableName table =
+        TableName.valueOf("testMissingFirstRegion");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -1418,7 +1448,8 @@ public class TestHBaseFsck {
    */
   @Test(timeout=120000)
   public void testMissingLastRegion() throws Exception {
-    String table = "testMissingLastRegion";
+    TableName table =
+        TableName.valueOf("testMissingLastRegion");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -1445,7 +1476,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testFixAssignmentsAndNoHdfsChecking() throws Exception {
-    String table = "testFixAssignmentsAndNoHdfsChecking";
+    TableName table =
+        TableName.valueOf("testFixAssignmentsAndNoHdfsChecking");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -1494,7 +1526,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testFixMetaNotWorkingWithNoHdfsChecking() throws Exception {
-    String table = "testFixMetaNotWorkingWithNoHdfsChecking";
+    TableName table =
+        TableName.valueOf("testFixMetaNotWorkingWithNoHdfsChecking");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -1541,7 +1574,8 @@ public class TestHBaseFsck {
    */
   @Test
   public void testFixHdfsHolesNotWorkingWithNoHdfsChecking() throws Exception {
-    String table = "testFixHdfsHolesNotWorkingWithNoHdfsChecking";
+    TableName table =
+        TableName.valueOf("testFixHdfsHolesNotWorkingWithNoHdfsChecking");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -1604,8 +1638,8 @@ public class TestHBaseFsck {
    * @return Path of a flushed hfile.
    * @throws IOException
    */
-  Path getFlushedHFile(FileSystem fs, String table) throws IOException {
-    Path tableDir= FSUtils.getTablePath(FSUtils.getRootDir(conf), table);
+  Path getFlushedHFile(FileSystem fs, TableName table) throws IOException {
+    Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), table);
     Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
     Path famDir = new Path(regionDir, FAM_STR);
 
@@ -1628,11 +1662,11 @@ public class TestHBaseFsck {
    */
   @Test(timeout=120000)
   public void testQuarantineCorruptHFile() throws Exception {
-    String table = name.getMethodName();
+    TableName table = TableName.valueOf(name.getMethodName());
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
-      TEST_UTIL.getHBaseAdmin().flush(table); // flush is async.
+      TEST_UTIL.getHBaseAdmin().flush(table.getName()); // flush is async.
 
       FileSystem fs = FileSystem.get(conf);
       Path hfile = getFlushedHFile(fs, table);
@@ -1666,17 +1700,18 @@ public class TestHBaseFsck {
   /**
   * Test that use this should have a timeout, because this method could potentially wait forever.
   */
-  private void doQuarantineTest(String table, HBaseFsck hbck, int check, int corrupt, int fail,
-      int quar, int missing) throws Exception {
+  private void doQuarantineTest(TableName table, HBaseFsck hbck, int check,
+                                int corrupt, int fail, int quar, int missing) throws Exception {
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
-      TEST_UTIL.getHBaseAdmin().flush(table); // flush is async.
+      TEST_UTIL.getHBaseAdmin().flush(table.getName()); // flush is async.
 
       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
       TEST_UTIL.getHBaseAdmin().disableTable(table);
 
-      String[] args = {"-sidelineCorruptHFiles", "-repairHoles", "-ignorePreCheckPermission", table};
+      String[] args = {"-sidelineCorruptHFiles", "-repairHoles", "-ignorePreCheckPermission",
+          table.getNameAsString()};
       ExecutorService exec = new ScheduledThreadPoolExecutor(10);
       HBaseFsck res = hbck.exec(exec, args);
 
@@ -1709,7 +1744,7 @@ public class TestHBaseFsck {
    */
   @Test(timeout=120000)
   public void testQuarantineMissingHFile() throws Exception {
-    String table = name.getMethodName();
+    TableName table = TableName.valueOf(name.getMethodName());
     ExecutorService exec = new ScheduledThreadPoolExecutor(10);
     // inject a fault in the hfcc created.
     final FileSystem fs = FileSystem.get(conf);
@@ -1738,7 +1773,7 @@ public class TestHBaseFsck {
   // files in a column family on initial creation -- as suggested by Matteo.
   @Ignore @Test(timeout=120000)
   public void testQuarantineMissingFamdir() throws Exception {
-    String table = name.getMethodName();
+    TableName table = TableName.valueOf(name.getMethodName());
     ExecutorService exec = new ScheduledThreadPoolExecutor(10);
     // inject a fault in the hfcc created.
     final FileSystem fs = FileSystem.get(conf);
@@ -1765,7 +1800,7 @@ public class TestHBaseFsck {
    */
   @Test(timeout=120000)
   public void testQuarantineMissingRegionDir() throws Exception {
-    String table = name.getMethodName();
+    TableName table = TableName.valueOf(name.getMethodName());
     ExecutorService exec = new ScheduledThreadPoolExecutor(10);
     // inject a fault in the hfcc created.
     final FileSystem fs = FileSystem.get(conf);
@@ -1791,14 +1826,15 @@ public class TestHBaseFsck {
    */
   @Test
   public void testLingeringReferenceFile() throws Exception {
-    String table = "testLingeringReferenceFile";
+    TableName table =
+        TableName.valueOf("testLingeringReferenceFile");
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
 
       // Mess it up by creating a fake reference file
       FileSystem fs = FileSystem.get(conf);
-      Path tableDir= FSUtils.getTablePath(FSUtils.getRootDir(conf), table);
+      Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), table);
       Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
       Path famDir = new Path(regionDir, FAM_STR);
       Path fakeReferenceFile = new Path(famDir, "fbce357483ceea.12144538");
@@ -1820,17 +1856,32 @@ public class TestHBaseFsck {
    */
   @Test
   public void testMissingRegionInfoQualifier() throws Exception {
-    String table = "testMissingRegionInfoQualifier";
+    TableName table =
+        TableName.valueOf("testMissingRegionInfoQualifier");
     try {
       setupTable(table);
 
       // Mess it up by removing the RegionInfo for one region.
-      HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
-      ResultScanner scanner = meta.getScanner(new Scan());
-      Result result = scanner.next();
-      Delete delete = new Delete (result.getRow());
-      delete.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-      meta.delete(delete);
+      final List<Delete> deletes = new LinkedList<Delete>();
+      HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName());
+      MetaScanner.metaScan(conf, new MetaScanner.MetaScannerVisitor() {
+
+        @Override
+        public boolean processRow(Result rowResult) throws IOException {
+          if(!HTableDescriptor.isSystemTable(MetaScanner.getHRegionInfo(rowResult)
+              .getTableName())) {
+            Delete delete = new Delete(rowResult.getRow());
+            delete.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
+            deletes.add(delete);
+          }
+          return true;
+        }
+
+        @Override
+        public void close() throws IOException {
+        }
+      });
+      meta.delete(deletes);
 
       // Mess it up by creating a fake META entry with no associated RegionInfo
       meta.put(new Put(Bytes.toBytes(table + ",,1361911384013.810e28f59a57da91c66")).add(
@@ -1950,7 +2001,8 @@ public class TestHBaseFsck {
 
     // obtain one lock
     final TableLockManager tableLockManager = TableLockManager.createTableLockManager(conf, TEST_UTIL.getZooKeeperWatcher(), mockName);
-    TableLock writeLock = tableLockManager.writeLock(Bytes.toBytes("foo"), "testCheckTableLocks");
+    TableLock writeLock = tableLockManager.writeLock(TableName.valueOf("foo"),
+        "testCheckTableLocks");
     writeLock.acquire();
     hbck = doFsck(conf, false);
     assertNoErrors(hbck); // should not have expired, no problems
@@ -1964,7 +2016,8 @@ public class TestHBaseFsck {
     final CountDownLatch latch = new CountDownLatch(1);
     new Thread() {
       public void run() {
-        TableLock readLock = tableLockManager.writeLock(Bytes.toBytes("foo"), "testCheckTableLocks");
+        TableLock readLock = tableLockManager.writeLock(TableName.valueOf("foo"),
+            "testCheckTableLocks");
         try {
           latch.countDown();
           readLock.acquire();
@@ -1998,7 +2051,8 @@ public class TestHBaseFsck {
     assertNoErrors(hbck);
 
     // ensure that locks are deleted
-    writeLock = tableLockManager.writeLock(Bytes.toBytes("foo"), "should acquire without blocking");
+    writeLock = tableLockManager.writeLock(TableName.valueOf("foo"),
+        "should acquire without blocking");
     writeLock.acquire(); // this should not block.
     writeLock.release(); // release for clean state
   }
@@ -2022,7 +2076,7 @@ public class TestHBaseFsck {
   private void deleteMetaRegion(Configuration conf, boolean unassign, boolean hdfs,
       boolean regionInfoOnly) throws IOException, InterruptedException {
     HConnection connection = HConnectionManager.getConnection(conf);
-    HRegionLocation metaLocation = connection.locateRegion(HConstants.META_TABLE_NAME,
+    HRegionLocation metaLocation = connection.locateRegion(TableName.META_TABLE_NAME,
         HConstants.EMPTY_START_ROW);
     ServerName hsa = new ServerName(metaLocation.getHostnamePort(), 0L);
     HRegionInfo hri = metaLocation.getRegionInfo();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java Thu Aug  8 04:19:49 2013
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.util;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
@@ -34,15 +35,17 @@ import org.junit.experimental.categories
 @Category(SmallTests.class)
 public class TestHBaseFsckComparator {
 
-  byte[] table = Bytes.toBytes("table1");
-  byte[] table2 = Bytes.toBytes("table2");
+  TableName table =
+      TableName.valueOf("table1");
+  TableName table2 =
+      TableName.valueOf("table2");
   byte[] keyStart = Bytes.toBytes("");
   byte[] keyA = Bytes.toBytes("A");
   byte[] keyB = Bytes.toBytes("B");
   byte[] keyC = Bytes.toBytes("C");
   byte[] keyEnd = Bytes.toBytes("");
 
-  static HbckInfo genHbckInfo(byte[] table, byte[] start, byte[] end, int time) {
+  static HbckInfo genHbckInfo(TableName table, byte[] start, byte[] end, int time) {
     return new HbckInfo(new MetaEntry(new HRegionInfo(table, start, end), null,
         time));
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java Thu Aug  8 04:19:49 2013
@@ -21,24 +21,24 @@ import static org.junit.Assert.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.SmallTests;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
+
+import java.io.IOException;
 
 /**
  * Test that the utility works as expected
  */
 @Category(SmallTests.class)
 public class TestHFileArchiveUtil {
-
+  private Path rootDir = new Path("./");
   @Test
   public void testGetTableArchivePath() {
-    assertNotNull(HFileArchiveUtil.getTableArchivePath(new Path("table")));
-    assertNotNull(HFileArchiveUtil.getTableArchivePath(new Path("root", new Path("table"))));
+    assertNotNull(HFileArchiveUtil.getTableArchivePath(rootDir,
+        TableName.valueOf("table")));
   }
 
   @Test
@@ -50,19 +50,19 @@ public class TestHFileArchiveUtil {
   
   @Test
   public void testRegionArchiveDir() {
-    Path tableDir = new Path("table");
     Path regionDir = new Path("region");
-    assertNotNull(HFileArchiveUtil.getRegionArchiveDir(tableDir, regionDir));
+    assertNotNull(HFileArchiveUtil.getRegionArchiveDir(rootDir,
+        TableName.valueOf("table"), regionDir));
   }
   
   @Test
-  public void testGetStoreArchivePath(){
+  public void testGetStoreArchivePath() throws IOException {
       byte[] family = Bytes.toBytes("Family");
-    Path tabledir = new Path("table");
-    HRegionInfo region = new HRegionInfo(Bytes.toBytes("table"));
-    Configuration conf = null;
-    assertNotNull(HFileArchiveUtil.getStoreArchivePath(conf, region, tabledir, family));
-    conf = new Configuration();
+    Path tabledir = FSUtils.getTableDir(rootDir,
+        TableName.valueOf("table"));
+    HRegionInfo region = new HRegionInfo(TableName.valueOf("table"));
+    Configuration conf = new Configuration();
+    FSUtils.setRootDir(conf, new Path("root"));
     assertNotNull(HFileArchiveUtil.getStoreArchivePath(conf, region, tabledir, family));
   }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java Thu Aug  8 04:19:49 2013
@@ -66,7 +66,7 @@ public class TestMergeTable {
    */
   @Test (timeout=300000) public void testMergeTable() throws Exception {
     // Table we are manually creating offline.
-    HTableDescriptor desc = new HTableDescriptor(Bytes.toBytes("test"));
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test")));
     desc.addFamily(new HColumnDescriptor(COLUMN_NAME));
 
     // Set maximum regionsize down.
@@ -114,14 +114,14 @@ public class TestMergeTable {
       CatalogTracker ct = new CatalogTracker(c);
       ct.start();
       List<HRegionInfo> originalTableRegions =
-        MetaReader.getTableRegions(ct, desc.getName());
+        MetaReader.getTableRegions(ct, desc.getTableName());
       LOG.info("originalTableRegions size=" + originalTableRegions.size() +
         "; " + originalTableRegions);
       HBaseAdmin admin = new HBaseAdmin(c);
-      admin.disableTable(desc.getName());
-      HMerge.merge(c, FileSystem.get(c), desc.getName());
+      admin.disableTable(desc.getTableName());
+      HMerge.merge(c, FileSystem.get(c), desc.getTableName());
       List<HRegionInfo> postMergeTableRegions =
-        MetaReader.getTableRegions(ct, desc.getName());
+        MetaReader.getTableRegions(ct, desc.getTableName());
       LOG.info("postMergeTableRegions size=" + postMergeTableRegions.size() +
         "; " + postMergeTableRegions);
       assertTrue("originalTableRegions=" + originalTableRegions.size() +
@@ -137,7 +137,7 @@ 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);
+    HRegionInfo hri = new HRegionInfo(desc.getTableName(), startKey, endKey);
     HRegion region = HRegion.createHRegion(hri, rootdir, UTIL.getConfiguration(), desc);
     LOG.info("Created region " + region.getRegionNameAsString());
     for(int i = firstRow; i < firstRow + nrows; i++) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java Thu Aug  8 04:19:49 2013
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.experimental.categories.Category;
@@ -68,28 +67,28 @@ public class TestMergeTool extends HBase
     this.conf.set("hbase.hstore.compactionThreshold", "2");
 
     // Create table description
-    this.desc = new HTableDescriptor("TestMergeTool");
+    this.desc = new HTableDescriptor(TableName.valueOf("TestMergeTool"));
     this.desc.addFamily(new HColumnDescriptor(FAMILY));
 
     /*
      * Create the HRegionInfos for the regions.
      */
     // Region 0 will contain the key range [row_0200,row_0300)
-    sourceRegions[0] = new HRegionInfo(this.desc.getName(),
+    sourceRegions[0] = new HRegionInfo(this.desc.getTableName(),
         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(),
+      new HRegionInfo(this.desc.getTableName(),
           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(),
+      new HRegionInfo(this.desc.getTableName(),
           Bytes.toBytes("row_0100"),
           Bytes.toBytes("row_0200"));
 
@@ -97,13 +96,13 @@ public class TestMergeTool extends HBase
     // 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(),
+      new HRegionInfo(this.desc.getTableName(),
           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(),
+      new HRegionInfo(this.desc.getTableName(),
           HConstants.EMPTY_BYTE_ARRAY,
           HConstants.EMPTY_BYTE_ARRAY);
 
@@ -195,7 +194,7 @@ public class TestMergeTool extends HBase
     LOG.info(msg);
     LOG.info("fs2=" + this.conf.get("fs.defaultFS"));
     int errCode = ToolRunner.run(this.conf, merger,
-      new String[] {this.desc.getNameAsString(), regionName1, regionName2}
+      new String[] {this.desc.getTableName().getNameAsString(), regionName1, regionName2}
     );
     assertTrue("'" + msg + "' failed with errCode " + errCode, errCode == 0);
     HRegionInfo mergedInfo = merger.getMergedHRegionInfo();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java Thu Aug  8 04:19:49 2013
@@ -26,6 +26,7 @@ import java.util.List;
 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;
@@ -55,7 +56,8 @@ public class TestMiniClusterLoadSequenti
   private static final Log LOG = LogFactory.getLog(
       TestMiniClusterLoadSequential.class);
 
-  protected static final byte[] TABLE = Bytes.toBytes("load_test_tbl");
+  protected static final TableName TABLE =
+      TableName.valueOf("load_test_tbl");
   protected static final byte[] CF = Bytes.toBytes("load_test_cf");
   protected static final int NUM_THREADS = 8;
   protected static final int NUM_RS = 2;
@@ -108,13 +110,13 @@ public class TestMiniClusterLoadSequenti
   }
 
   protected MultiThreadedReader prepareReaderThreads(LoadTestDataGenerator dataGen,
-      Configuration conf, byte[] tableName, double verifyPercent) {
+      Configuration conf, TableName tableName, double verifyPercent) {
     MultiThreadedReader reader = new MultiThreadedReader(dataGen, conf, tableName, verifyPercent);
     return reader;
   }
 
   protected MultiThreadedWriter prepareWriterThreads(LoadTestDataGenerator dataGen,
-      Configuration conf, byte[] tableName) {
+      Configuration conf, TableName tableName) {
     MultiThreadedWriter writer = new MultiThreadedWriter(dataGen, conf, tableName);
     writer.setMultiPut(isMultiPut);
     return writer;
@@ -141,7 +143,7 @@ public class TestMiniClusterLoadSequenti
   protected void createPreSplitLoadTestTable(HTableDescriptor htd, HColumnDescriptor hcd)
       throws IOException {
     HBaseTestingUtility.createPreSplitLoadTestTable(conf, htd, hcd);
-    TEST_UTIL.waitUntilAllRegionsAssigned(htd.getName());
+    TEST_UTIL.waitUntilAllRegionsAssigned(htd.getTableName());
   }
 
   protected void prepareForLoadTest() throws IOException {

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java?rev=1511577&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java Thu Aug  8 04:19:49 2013
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.TableName;
+import org.junit.rules.TestWatcher;
+import org.junit.runner.Description;
+
+/**
+ * Returns a {@code byte[]} containing the name of the currently running test method.
+ */
+public class TestTableName extends TestWatcher {
+  private TableName tableName;
+
+  /**
+   * Invoked when a test is about to start
+   */
+  @Override
+  protected void starting(Description description) {
+    tableName = TableName.valueOf(description.getMethodName());
+  }
+
+  public TableName getTableName() {
+    return tableName;
+  }
+}

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java Thu Aug  8 04:19:49 2013
@@ -20,12 +20,14 @@ package org.apache.hadoop.hbase.util.hbc
 import static org.junit.Assert.assertEquals;
 
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 
+import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 
@@ -37,14 +39,15 @@ public class HbckTestingUtil {
   }
 
   public static HBaseFsck doFsck(
-      Configuration conf, boolean fix, String table) throws Exception {
+      Configuration conf, boolean fix, TableName table) throws Exception {
     return doFsck(conf, fix, fix, fix, fix,fix, fix, fix, fix, fix, fix, table);
   }
 
   public static HBaseFsck doFsck(Configuration conf, boolean fixAssignments,
       boolean fixMeta, boolean fixHdfsHoles, boolean fixHdfsOverlaps,
       boolean fixHdfsOrphans, boolean fixTableOrphans, boolean fixVersionFile,
-      boolean fixReferenceFiles, boolean fixEmptyMetaRegionInfo, boolean fixTableLocks, String table) throws Exception {
+      boolean fixReferenceFiles, boolean fixEmptyMetaRegionInfo, boolean fixTableLocks,
+      TableName table) throws Exception {
     HBaseFsck fsck = new HBaseFsck(conf, exec);
     fsck.connect();
     fsck.setDisplayFullReport(); // i.e. -details
@@ -73,8 +76,8 @@ public class HbckTestingUtil {
    * @return <returncode, hbckInstance>
    * @throws Exception
    */
-  public static HBaseFsck doHFileQuarantine(Configuration conf, String table) throws Exception {
-    String[] args = {"-sidelineCorruptHFiles", "-ignorePreCheckPermission", table};
+  public static HBaseFsck doHFileQuarantine(Configuration conf, TableName table) throws Exception {
+    String[] args = {"-sidelineCorruptHFiles", "-ignorePreCheckPermission", table.getNameAsString()};
     HBaseFsck hbck = new HBaseFsck(conf, exec);
     hbck.exec(exec, args);
     return hbck;
@@ -87,6 +90,9 @@ public class HbckTestingUtil {
 
   public static void assertErrors(HBaseFsck fsck, ERROR_CODE[] expectedErrors) {
     List<ERROR_CODE> errs = fsck.getErrors().getErrorList();
-    assertEquals(Arrays.asList(expectedErrors), errs);
+    Collections.sort(errs);
+    List<ERROR_CODE> expErrs = Lists.newArrayList(expectedErrors);
+    Collections.sort(expErrs);
+    assertEquals(expErrs, errs);
   }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java Thu Aug  8 04:19:49 2013
@@ -31,12 +31,13 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FSDataOutputStream;
 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.LargeTests;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.client.Delete;
@@ -83,7 +84,7 @@ public class OfflineMetaRebuildTestCore 
 
   private final static String TABLE_BASE = "tableMetaRebuild";
   private static int tableIdx = 0;
-  protected String table = "tableMetaRebuild";
+  protected TableName table = TableName.valueOf("tableMetaRebuild");
 
   @Before
   public void setUpBefore() throws Exception {
@@ -94,11 +95,11 @@ public class OfflineMetaRebuildTestCore 
     assertEquals(0, TEST_UTIL.getHBaseAdmin().listTables().length);
 
     // setup the table
-    table = TABLE_BASE + "-" + tableIdx;
+    table = TableName.valueOf(TABLE_BASE + "-" + tableIdx);
     tableIdx++;
     htbl = setupTable(table);
     populateTable(htbl);
-    assertEquals(4, scanMeta());
+    assertEquals(5, scanMeta());
     LOG.info("Table " + table + " has " + tableRowCount(conf, table)
         + " entries.");
     assertEquals(16, tableRowCount(conf, table));
@@ -119,7 +120,7 @@ public class OfflineMetaRebuildTestCore 
    * @throws InterruptedException
    * @throws KeeperException
    */
-  private HTable setupTable(String tablename) throws Exception {
+  private HTable setupTable(TableName tablename) throws Exception {
     HTableDescriptor desc = new HTableDescriptor(tablename);
     HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
     desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
@@ -128,7 +129,7 @@ public class OfflineMetaRebuildTestCore 
   }
 
   private void dumpMeta(HTableDescriptor htd) throws IOException {
-    List<byte[]> metaRows = TEST_UTIL.getMetaTableRows(htd.getName());
+    List<byte[]> metaRows = TEST_UTIL.getMetaTableRows(htd.getTableName());
     for (byte[] row : metaRows) {
       LOG.info(Bytes.toString(row));
     }
@@ -184,11 +185,11 @@ public class OfflineMetaRebuildTestCore 
         LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
         Path rootDir = FSUtils.getRootDir(conf);
         FileSystem fs = rootDir.getFileSystem(conf);
-        Path p = new Path(rootDir + "/" + htd.getNameAsString(),
+        Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()),
             hri.getEncodedName());
         fs.delete(p, true);
 
-        HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+        HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
         Delete delete = new Delete(deleteRow);
         meta.delete(delete);
         meta.close();
@@ -196,21 +197,21 @@ public class OfflineMetaRebuildTestCore 
       LOG.info(hri.toString() + hsa.toString());
     }
 
-    TEST_UTIL.getMetaTableRows(htd.getName());
+    TEST_UTIL.getMetaTableRows(htd.getTableName());
     LOG.info("After delete:");
     dumpMeta(htd);
   }
 
   protected HRegionInfo createRegion(Configuration conf, final HTable htbl,
       byte[] startKey, byte[] endKey) throws IOException {
-    HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+    HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
     HTableDescriptor htd = htbl.getTableDescriptor();
-    HRegionInfo hri = new HRegionInfo(htbl.getTableName(), startKey, endKey);
+    HRegionInfo hri = new HRegionInfo(htbl.getName(), startKey, endKey);
 
     LOG.info("manually adding regioninfo and hdfs data: " + hri.toString());
     Path rootDir = FSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
-    Path p = new Path(rootDir + "/" + htd.getNameAsString(),
+    Path p = new Path(FSUtils.getTableDir(rootDir, htbl.getName()),
         hri.getEncodedName());
     fs.mkdirs(p);
     Path riPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE);
@@ -228,13 +229,19 @@ public class OfflineMetaRebuildTestCore 
     // Mess it up by blowing up meta.
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     Scan s = new Scan();
-    HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+    HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
     ResultScanner scanner = meta.getScanner(s);
     List<Delete> dels = new ArrayList<Delete>();
     for (Result r : scanner) {
-      Delete d = new Delete(r.getRow());
-      dels.add(d);
-      admin.unassign(r.getRow(), true);
+      HRegionInfo info =
+          HRegionInfo.getHRegionInfo(r);
+      if(info != null &&
+          !info.getTableName().getNamespaceAsString()
+          .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
+        Delete d = new Delete(r.getRow());
+        dels.add(d);
+        admin.unassign(r.getRow(), true);
+      }
     }
     meta.delete(dels);
     meta.flushCommits();
@@ -248,7 +255,7 @@ public class OfflineMetaRebuildTestCore 
    *
    * @return # of rows in the specified table
    */
-  protected int tableRowCount(Configuration conf, String table)
+  protected int tableRowCount(Configuration conf, TableName table)
       throws IOException {
     HTable t = new HTable(conf, table);
     Scan st = new Scan();
@@ -270,7 +277,7 @@ public class OfflineMetaRebuildTestCore 
    */
   protected int scanMeta() throws IOException {
     int count = 0;
-    HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
+    HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName());
     ResultScanner scanner = meta.getScanner(new Scan());
     LOG.info("Table: " + Bytes.toString(meta.getTableName()));
     for (Result res : scanner) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java Thu Aug  8 04:19:49 2013
@@ -45,12 +45,13 @@ public class TestOfflineMetaRebuildBase 
     wipeOutMeta();
 
     // is meta really messed up?
-    assertEquals(0, scanMeta());
+    assertEquals(1, scanMeta());
     assertErrors(doFsck(conf, false),
-        new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+        new ERROR_CODE[] {
             ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
             ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-            ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED});
     // Note, would like to check # of tables, but this takes a while to time
     // out.
 
@@ -74,7 +75,7 @@ public class TestOfflineMetaRebuildBase 
     LOG.info("No more RIT in ZK, now doing final test verification");
 
     // everything is good again.
-    assertEquals(4, scanMeta());
+    assertEquals(5, scanMeta());
     HTableDescriptor[] htbls = TEST_UTIL.getHBaseAdmin().listTables();
     LOG.info("Tables present after restart: " + Arrays.toString(htbls));
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java Thu Aug  8 04:19:49 2013
@@ -52,10 +52,11 @@ public class TestOfflineMetaRebuildHole 
     wipeOutMeta();
 
     // is meta really messed up?
-    assertEquals(0, scanMeta());
+    assertEquals(1, scanMeta());
     assertErrors(doFsck(conf, false), new ERROR_CODE[] {
-        ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-        ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
+        ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+        ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+        ERROR_CODE.NOT_IN_META_OR_DEPLOYED});
     // Note, would like to check # of tables, but this takes a while to time
     // out.
 
@@ -76,9 +77,17 @@ public class TestOfflineMetaRebuildHole 
     LOG.info("Waiting for no more RIT");
     ZKAssign.blockUntilNoRIT(zkw);
     LOG.info("No more RIT in ZK, now doing final test verification");
+    int tries = 60;
+    while(TEST_UTIL.getHBaseCluster()
+        .getMaster().getAssignmentManager().getRegionStates().getRegionsInTransition().size() > 0 &&
+        tries-- > 0) {
+      LOG.info("Waiting for RIT: "+TEST_UTIL.getHBaseCluster()
+              .getMaster().getAssignmentManager().getRegionStates().getRegionsInTransition());
+      Thread.sleep(1000);
+    }
 
     // Meta still messed up.
-    assertEquals(0, scanMeta());
+    assertEquals(1, scanMeta());
     HTableDescriptor[] htbls = TEST_UTIL.getHBaseAdmin().listTables();
     LOG.info("Tables present after restart: " + Arrays.toString(htbls));
 
@@ -86,8 +95,9 @@ public class TestOfflineMetaRebuildHole 
     // so the table is still present and this should be 1.
     assertEquals(1, htbls.length);
     assertErrors(doFsck(conf, false), new ERROR_CODE[] {
-        ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-        ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
+        ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+        ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+        ERROR_CODE.NOT_IN_META_OR_DEPLOYED});
   }
 
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java Thu Aug  8 04:19:49 2013
@@ -54,12 +54,13 @@ public class TestOfflineMetaRebuildOverl
     wipeOutMeta();
 
     // is meta really messed up?
-    assertEquals(0, scanMeta());
+    assertEquals(1, scanMeta());
     assertErrors(doFsck(conf, false),
-        new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+        new ERROR_CODE[] {
             ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
             ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-            ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED});
     // Note, would like to check # of tables, but this takes a while to time
     // out.
 
@@ -84,9 +85,17 @@ public class TestOfflineMetaRebuildOverl
     LOG.info("Waiting for no more RIT");
     ZKAssign.blockUntilNoRIT(zkw);
     LOG.info("No more RIT in ZK, now doing final test verification");
+    int tries = 60;
+    while(TEST_UTIL.getHBaseCluster()
+        .getMaster().getAssignmentManager().getRegionStates().getRegionsInTransition().size() > 0 &&
+        tries-- > 0) {
+      LOG.info("Waiting for RIT: "+TEST_UTIL.getHBaseCluster()
+              .getMaster().getAssignmentManager().getRegionStates().getRegionsInTransition());
+      Thread.sleep(1000);
+    }
     
     // Meta still messed up.
-    assertEquals(0, scanMeta());
+    assertEquals(1, scanMeta());
     HTableDescriptor[] htbls = TEST_UTIL.getHBaseAdmin().listTables();
     LOG.info("Tables present after restart: " + Arrays.toString(htbls));
 
@@ -94,10 +103,11 @@ public class TestOfflineMetaRebuildOverl
     // so the table is still present and this should be 1.
     assertEquals(1, htbls.length);
     assertErrors(doFsck(conf, false),
-        new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+        new ERROR_CODE[] {
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
             ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
             ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-            ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED});
   }
 
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKTable.java Thu Aug  8 04:19:49 2013
@@ -51,7 +51,8 @@ public class TestZKTable {
   @Test
   public void testTableStates()
   throws ZooKeeperConnectionException, IOException, KeeperException {
-    final String name = "testDisabled";
+    final TableName name =
+        TableName.valueOf("testDisabled");
     Abortable abortable = new Abortable() {
       @Override
       public void abort(String why, Throwable e) {
@@ -65,7 +66,7 @@ public class TestZKTable {
       
     };
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
-      name, abortable, true);
+      name.getNameAsString(), abortable, true);
     ZKTable zkt = new ZKTable(zkw);
     assertFalse(zkt.isEnabledTable(name));
     assertFalse(zkt.isDisablingTable(name));

Modified: hbase/trunk/hbase-server/src/test/ruby/hbase/admin_test.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/ruby/hbase/admin_test.rb?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/ruby/hbase/admin_test.rb (original)
+++ hbase/trunk/hbase-server/src/test/ruby/hbase/admin_test.rb Thu Aug  8 04:19:49 2013
@@ -37,11 +37,11 @@ module Hbase
     end
 
     define_test "exists? should return true when a table exists" do
-      assert(admin.exists?('.META.'))
+      assert(admin.exists?('hbase:meta'))
     end
 
     define_test "exists? should return false when a table exists" do
-      assert(!admin.exists?('.NOT.EXISTS.'))
+      assert(!admin.exists?('NOT.EXISTS'))
     end
 
     define_test "enabled? should return true for enabled tables" do
@@ -74,38 +74,38 @@ module Hbase
     end
 
     define_test "list should not return meta tables" do
-      assert(!admin.list.member?('.META.'))
+      assert(!admin.list.member?('hbase:meta'))
     end
 
     #-------------------------------------------------------------------------------
 
     define_test "flush should work" do
-      admin.flush('.META.')
+      admin.flush('hbase:meta')
     end
 
     #-------------------------------------------------------------------------------
 
     define_test "compact should work" do
-      admin.compact('.META.')
+      admin.compact('hbase:meta')
     end
 
     #-------------------------------------------------------------------------------
 
     define_test "major_compact should work" do
-      admin.major_compact('.META.')
+      admin.major_compact('hbase:meta')
     end
 
     #-------------------------------------------------------------------------------
 
     define_test "split should work" do
-      admin.split('.META.', nil)
+      admin.split('hbase:meta', nil)
     end
 
     #-------------------------------------------------------------------------------
 
     define_test "drop should fail on non-existent tables" do
       assert_raise(ArgumentError) do
-        admin.drop('.NOT.EXISTS.')
+        admin.drop('NOT.EXISTS')
       end
     end
 
@@ -243,7 +243,7 @@ module Hbase
 
     define_test "alter should fail with non-existing tables" do
       assert_raise(ArgumentError) do
-        admin.alter('.NOT.EXISTS.', true, METHOD => 'delete', NAME => 'y')
+        admin.alter('NOT.EXISTS', true, METHOD => 'delete', NAME => 'y')
       end
     end
 

Modified: hbase/trunk/hbase-server/src/test/ruby/hbase/hbase_test.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/ruby/hbase/hbase_test.rb?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/ruby/hbase/hbase_test.rb (original)
+++ hbase/trunk/hbase-server/src/test/ruby/hbase/hbase_test.rb Thu Aug  8 04:19:49 2013
@@ -39,11 +39,11 @@ module Hbase
     end
 
     define_test "Hbase::Hbase#table should create a new table object when called the first time" do
-      assert_kind_of(::Hbase::Table, @hbase.table('.META.', @formatter))
+      assert_kind_of(::Hbase::Table, @hbase.table('hbase:meta', @formatter))
     end
 
     define_test "Hbase::Hbase#table should create a new table object every call" do
-      assert_not_same(@hbase.table('.META.', @formatter), @hbase.table('.META.', @formatter))
+      assert_not_same(@hbase.table('hbase:meta', @formatter), @hbase.table('hbase:meta', @formatter))
     end
   end
 end

Modified: hbase/trunk/hbase-server/src/test/ruby/hbase/table_test.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/ruby/hbase/table_test.rb?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/ruby/hbase/table_test.rb (original)
+++ hbase/trunk/hbase-server/src/test/ruby/hbase/table_test.rb Thu Aug  8 04:19:49 2013
@@ -37,7 +37,7 @@ module Hbase
 
     define_test "Hbase::Table constructor should not fail for existent tables" do
       assert_nothing_raised do
-        table('.META.')
+        table('hbase:meta')
       end
     end
   end
@@ -55,7 +55,7 @@ module Hbase
     end
 
     define_test "is_meta_table? method should return true for the meta table" do
-      assert(table('.META.').is_meta_table?)
+      assert(table('hbase:meta').is_meta_table?)
     end
 
     define_test "is_meta_table? method should return false for a normal table" do
@@ -65,7 +65,7 @@ module Hbase
     #-------------------------------------------------------------------------------
 
     define_test "get_all_columns should return columns list" do
-      cols = table('.META.').get_all_columns
+      cols = table('hbase:meta').get_all_columns
       assert_kind_of(Array, cols)
       assert(cols.length > 0)
     end
@@ -73,19 +73,19 @@ module Hbase
     #-------------------------------------------------------------------------------
 
     define_test "parse_column_name should not return a qualifier for name-only column specifiers" do
-      col, qual = table('.META.').parse_column_name('foo')
+      col, qual = table('hbase:meta').parse_column_name('foo')
       assert_not_nil(col)
       assert_nil(qual)
     end
 
     define_test "parse_column_name should not return a qualifier for family-only column specifiers" do
-      col, qual = table('.META.').parse_column_name('foo:')
+      col, qual = table('hbase:meta').parse_column_name('foo:')
       assert_not_nil(col)
       assert_nil(qual)
     end
 
     define_test "parse_column_name should return a qualifier for family:qualifier column specifiers" do
-      col, qual = table('.META.').parse_column_name('foo:bar')
+      col, qual = table('hbase:meta').parse_column_name('foo:bar')
       assert_not_nil(col)
       assert_not_nil(qual)
     end

Modified: hbase/trunk/hbase-server/src/test/ruby/shell/shell_test.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/ruby/shell/shell_test.rb?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/ruby/shell/shell_test.rb (original)
+++ hbase/trunk/hbase-server/src/test/ruby/shell/shell_test.rb Thu Aug  8 04:19:49 2013
@@ -39,11 +39,11 @@ class ShellTest < Test::Unit::TestCase
   #-------------------------------------------------------------------------------
 
   define_test "Shell::Shell#hbase_table should return a table instance" do
-    assert_kind_of(Hbase::Table, @shell.hbase_table('.META.'))
+    assert_kind_of(Hbase::Table, @shell.hbase_table('hbase:meta'))
   end
 
   define_test "Shell::Shell#hbase_table should not cache table instances" do
-    assert_not_same(@shell.hbase_table('.META.'), @shell.hbase_table('.META.'))
+    assert_not_same(@shell.hbase_table('hbase:meta'), @shell.hbase_table('hbase:meta'))
   end
 
   #-------------------------------------------------------------------------------