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 [16/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/client/HConnectionTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java Thu Aug  8 04:19:49 2013
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
@@ -83,7 +84,7 @@ public class HConnectionTestingUtility {
    * getRegionLocation is called on the mocked connection
    * @return Mock up a connection that returns a {@link Configuration} when
    * {@link HConnection#getConfiguration()} is called, a 'location' when
-   * {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
+   * {@link HConnection#getRegionLocation(org.apache.hadoop.hbase.TableName, byte[], boolean)} is called,
    * and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} instance when
    * {@link HConnection#getAdmin(ServerName)} is called, returns the passed
    * {@link ClientProtos.ClientService.BlockingInterface} instance when
@@ -101,10 +102,10 @@ public class HConnectionTestingUtility {
     Mockito.doNothing().when(c).close();
     // Make it so we return a particular location when asked.
     final HRegionLocation loc = new HRegionLocation(hri, sn);
-    Mockito.when(c.getRegionLocation((byte[]) Mockito.any(),
+    Mockito.when(c.getRegionLocation((TableName) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(loc);
-    Mockito.when(c.locateRegion((byte[]) Mockito.any(), (byte[]) Mockito.any())).
+    Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any())).
       thenReturn(loc);
     if (admin != null) {
       // If a call to getAdmin, return this implementation.

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Thu Aug  8 04:19:49 2013
@@ -52,12 +52,13 @@ import org.apache.hadoop.hbase.MiniHBase
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -194,8 +195,8 @@ public class TestAdmin {
 
     exception = null;
     try {
-      HTableDescriptor htd = new HTableDescriptor(nonexistent);
-      this.admin.modifyTable(htd.getName(), htd);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(nonexistent));
+      this.admin.modifyTable(htd.getTableName(), htd);
     } catch (IOException e) {
       exception = e;
     }
@@ -205,13 +206,13 @@ public class TestAdmin {
     // nonexistent column family -- see if we get right exceptions.
     final String tableName =
         "testDeleteEditUnknownColumnFamilyAndOrTable" + System.currentTimeMillis();
-    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
     htd.addFamily(new HColumnDescriptor("cf"));
     this.admin.createTable(htd);
     try {
       exception = null;
       try {
-        this.admin.deleteColumn(htd.getName(), nonexistentHcd.getName());
+        this.admin.deleteColumn(htd.getTableName(), nonexistentHcd.getName());
       } catch (IOException e) {
         exception = e;
       }
@@ -220,7 +221,7 @@ public class TestAdmin {
 
       exception = null;
       try {
-        this.admin.modifyColumn(htd.getName(), nonexistentHcd);
+        this.admin.modifyColumn(htd.getTableName(), nonexistentHcd);
       } catch (IOException e) {
         exception = e;
       }
@@ -246,10 +247,10 @@ public class TestAdmin {
     get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
     ht.get(get);
 
-    this.admin.disableTable(table);
+    this.admin.disableTable(ht.getName());
     assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getAssignmentManager().getZKTable().isDisabledTable(
-            Bytes.toString(table)));
+            ht.getName()));
 
     // Test that table is disabled
     get = new Get(row);
@@ -264,7 +265,7 @@ public class TestAdmin {
     this.admin.enableTable(table);
     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getAssignmentManager().getZKTable().isEnabledTable(
-            Bytes.toString(table)));
+            ht.getName()));
 
     // Test that table is enabled
     try {
@@ -337,7 +338,7 @@ public class TestAdmin {
     assertEquals(numTables + 1, tables.length);
     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getAssignmentManager().getZKTable().isEnabledTable(
-            "testCreateTable"));
+            TableName.valueOf("testCreateTable")));
   }
 
   @Test
@@ -345,7 +346,7 @@ public class TestAdmin {
     HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
     HColumnDescriptor fam2 = new HColumnDescriptor("fam2");
     HColumnDescriptor fam3 = new HColumnDescriptor("fam3");
-    HTableDescriptor htd = new HTableDescriptor("myTestTable");
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("myTestTable"));
     htd.addFamily(fam1);
     htd.addFamily(fam2);
     htd.addFamily(fam3);
@@ -382,7 +383,8 @@ public class TestAdmin {
    */
   @Test
   public void testOnlineChangeTableSchema() throws IOException, InterruptedException {
-    final byte [] tableName = Bytes.toBytes("changeTableSchemaOnline");
+    final TableName tableName =
+        TableName.valueOf("changeTableSchemaOnline");
     TEST_UTIL.getMiniHBaseCluster().getMaster().getConfiguration().setBoolean(
         "hbase.online.schema.update.enable", true);
     HTableDescriptor [] tables = admin.listTables();
@@ -558,7 +560,7 @@ public class TestAdmin {
   @Test
   public void testCreateTableNumberOfRegions() throws IOException, InterruptedException {
     byte[] tableName = Bytes.toBytes("testCreateTableNumberOfRegions");
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc);
     HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
@@ -567,7 +569,7 @@ public class TestAdmin {
     ht.close();
 
     byte [] TABLE_2 = Bytes.add(tableName, Bytes.toBytes("_2"));
-    desc = new HTableDescriptor(TABLE_2);
+    desc = new HTableDescriptor(TableName.valueOf(TABLE_2));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, new byte[][]{new byte[]{42}});
     HTable ht2 = new HTable(TEST_UTIL.getConfiguration(), TABLE_2);
@@ -576,7 +578,7 @@ public class TestAdmin {
     ht2.close();
 
     byte [] TABLE_3 = Bytes.add(tableName, Bytes.toBytes("_3"));
-    desc = new HTableDescriptor(TABLE_3);
+    desc = new HTableDescriptor(TableName.valueOf(TABLE_3));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3);
     HTable ht3 = new HTable(TEST_UTIL.getConfiguration(), TABLE_3);
@@ -585,7 +587,7 @@ public class TestAdmin {
     ht3.close();
 
     byte [] TABLE_4 = Bytes.add(tableName, Bytes.toBytes("_4"));
-    desc = new HTableDescriptor(TABLE_4);
+    desc = new HTableDescriptor(TableName.valueOf(TABLE_4));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     try {
       admin.createTable(desc, "a".getBytes(), "z".getBytes(), 2);
@@ -595,7 +597,7 @@ public class TestAdmin {
     }
 
     byte [] TABLE_5 = Bytes.add(tableName, Bytes.toBytes("_5"));
-    desc = new HTableDescriptor(TABLE_5);
+    desc = new HTableDescriptor(TableName.valueOf(TABLE_5));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, new byte[] {1}, new byte[] {127}, 16);
     HTable ht5 = new HTable(TEST_UTIL.getConfiguration(), TABLE_5);
@@ -622,7 +624,7 @@ public class TestAdmin {
     };
     int expectedRegions = splitKeys.length + 1;
 
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, splitKeys);
     
@@ -684,7 +686,7 @@ public class TestAdmin {
 
     byte [] TABLE_2 = Bytes.add(tableName, Bytes.toBytes("_2"));
 
-    desc = new HTableDescriptor(TABLE_2);
+    desc = new HTableDescriptor(TableName.valueOf(TABLE_2));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
     admin.createTable(desc, startKey, endKey, expectedRegions);
@@ -740,7 +742,7 @@ public class TestAdmin {
 
     byte [] TABLE_3 = Bytes.add(tableName, Bytes.toBytes("_3"));
 
-    desc = new HTableDescriptor(TABLE_3);
+    desc = new HTableDescriptor(TableName.valueOf(TABLE_3));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
     admin.createTable(desc, startKey, endKey, expectedRegions);
@@ -766,7 +768,7 @@ public class TestAdmin {
     };
 
     byte [] TABLE_4 = Bytes.add(tableName, Bytes.toBytes("_4"));
-    desc = new HTableDescriptor(TABLE_4);
+    desc = new HTableDescriptor(TableName.valueOf(TABLE_4));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     HBaseAdmin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
     try {
@@ -782,7 +784,7 @@ public class TestAdmin {
   @Test
   public void testTableAvailableWithRandomSplitKeys() throws Exception {
     byte[] tableName = Bytes.toBytes("testTableAvailableWithRandomSplitKeys");
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("col"));
     byte[][] splitKeys = new byte[1][];
     splitKeys = new byte [][] {
@@ -799,7 +801,7 @@ public class TestAdmin {
     byte[] tableName = Bytes.toBytes("testCreateTableWithOnlyEmptyStartRow");
     byte[][] splitKeys = new byte[1][];
     splitKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("col"));
     try {
       admin.createTable(desc, splitKeys);
@@ -815,7 +817,7 @@ public class TestAdmin {
     splitKeys[0] = "region1".getBytes();
     splitKeys[1] = HConstants.EMPTY_BYTE_ARRAY;
     splitKeys[2] = "region2".getBytes();
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("col"));
     try {
       admin.createTable(desc, splitKeys);
@@ -865,7 +867,7 @@ public class TestAdmin {
         new byte[] { 6, 6, 6 }, new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 },
         new byte[] { 9, 9, 9 } };
     int expectedRegions = splitKeys.length + 1;
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, splitKeys);
     HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
@@ -880,7 +882,7 @@ public class TestAdmin {
 
     // Check the assignment.
     HTable metaTable = new HTable(TEST_UTIL.getConfiguration(),
-        HConstants.META_TABLE_NAME);
+        TableName.META_TABLE_NAME);
     List<HRegionInfo> regionInfos = admin.getTableRegions(tableName);
     Map<String, Integer> serverMap = new HashMap<String, Integer>();
     for (HRegionInfo hri : regionInfos) {
@@ -944,12 +946,12 @@ public class TestAdmin {
 
   void splitTest(byte[] splitPoint, byte[][] familyNames, int[] rowCounts,
     int numVersions, int blockSize) throws Exception {
+    TableName tableName = TableName.valueOf("testForceSplit");
     StringBuilder sb = new StringBuilder();
     // Add tail to String so can see better in logs where a test is running.
     for (int i = 0; i < rowCounts.length; i++) {
       sb.append("_").append(Integer.toString(rowCounts[i]));
     }
-    byte [] tableName = Bytes.toBytes("testForceSplit" + sb.toString());
     assertFalse(admin.tableExists(tableName));
     final HTable table = TEST_UTIL.createTable(tableName, familyNames,
       numVersions, blockSize);
@@ -996,7 +998,7 @@ public class TestAdmin {
     scanner.next();
 
     // Split the table
-    this.admin.split(tableName, splitPoint);
+    this.admin.split(tableName.getName(), splitPoint);
 
     final AtomicInteger count = new AtomicInteger(0);
     Thread t = new Thread("CheckForSplit") {
@@ -1097,9 +1099,10 @@ public class TestAdmin {
   @Test(timeout=300000)
   public void testEnableDisableAddColumnDeleteColumn() throws Exception {
     ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
-    byte [] tableName = Bytes.toBytes("testMasterAdmin");
+    TableName tableName = TableName.valueOf("testMasterAdmin");
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    while (!ZKTableReadOnly.isEnabledTable(zkw, "testMasterAdmin")) {
+    while (!ZKTableReadOnly.isEnabledTable(zkw,
+        TableName.valueOf("testMasterAdmin"))) {
       Thread.sleep(10);
     }
     this.admin.disableTable(tableName);
@@ -1124,26 +1127,17 @@ public class TestAdmin {
   public void testCreateBadTables() throws IOException {
     String msg = null;
     try {
-      this.admin.createTable(HTableDescriptor.ROOT_TABLEDESC);
-    } catch (IllegalArgumentException e) {
-      msg = e.toString();
-    }
-    assertTrue("Unexcepted exception message " + msg, msg != null &&
-      msg.startsWith(IllegalArgumentException.class.getName()) &&
-      msg.contains(HTableDescriptor.ROOT_TABLEDESC.getNameAsString()));
-    msg = null;
-    try {
       this.admin.createTable(HTableDescriptor.META_TABLEDESC);
-    } catch(IllegalArgumentException e) {
+    } catch(TableExistsException e) {
       msg = e.toString();
     }
     assertTrue("Unexcepted exception message " + msg, msg != null &&
-      msg.startsWith(IllegalArgumentException.class.getName()) &&
-      msg.contains(HTableDescriptor.META_TABLEDESC.getNameAsString()));
+      msg.startsWith(TableExistsException.class.getName()) &&
+      msg.contains(HTableDescriptor.META_TABLEDESC.getTableName().getNameAsString()));
 
     // Now try and do concurrent creation with a bunch of threads.
     final HTableDescriptor threadDesc =
-      new HTableDescriptor("threaded_testCreateBadTables");
+      new HTableDescriptor(TableName.valueOf("threaded_testCreateBadTables"));
     threadDesc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     int count = 10;
     Thread [] threads = new Thread [count];
@@ -1190,8 +1184,8 @@ public class TestAdmin {
   @Test
   public void testTableNameClash() throws Exception {
     String name = "testTableNameClash";
-    admin.createTable(new HTableDescriptor(name + "SOMEUPPERCASE"));
-    admin.createTable(new HTableDescriptor(name));
+    admin.createTable(new HTableDescriptor(TableName.valueOf(name + "SOMEUPPERCASE")));
+    admin.createTable(new HTableDescriptor(TableName.valueOf(name)));
     // Before fix, below would fail throwing a NoServerForRegionException.
     new HTable(TEST_UTIL.getConfiguration(), name).close();
   }
@@ -1215,7 +1209,7 @@ public class TestAdmin {
       byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
       byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
       HBaseAdmin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
-      hbaseadmin.createTable(new HTableDescriptor(name), startKey, endKey,
+      hbaseadmin.createTable(new HTableDescriptor(TableName.valueOf(name)), startKey, endKey,
         expectedRegions);
       hbaseadmin.close();
     } finally {
@@ -1248,13 +1242,11 @@ public class TestAdmin {
   public void testTableNames() throws IOException {
     byte[][] illegalNames = new byte[][] {
         Bytes.toBytes("-bad"),
-        Bytes.toBytes(".bad"),
-        HConstants.ROOT_TABLE_NAME,
-        HConstants.META_TABLE_NAME
+        Bytes.toBytes(".bad")
     };
     for (byte[] illegalName : illegalNames) {
       try {
-        new HTableDescriptor(illegalName);
+        new HTableDescriptor(TableName.valueOf(illegalName));
         throw new IOException("Did not detect '" +
             Bytes.toString(illegalName) + "' as an illegal user table name");
       } catch (IllegalArgumentException e) {
@@ -1263,7 +1255,7 @@ public class TestAdmin {
     }
     byte[] legalName = Bytes.toBytes("g-oo.d");
     try {
-      new HTableDescriptor(legalName);
+      new HTableDescriptor(TableName.valueOf(legalName));
     } catch (IllegalArgumentException e) {
       throw new IOException("Legal user table name: '" +
         Bytes.toString(legalName) + "' caused IllegalArgumentException: " +
@@ -1324,7 +1316,8 @@ public class TestAdmin {
   @Test
   public void testShouldCloseTheRegionBasedOnTheEncodedRegionName()
       throws Exception {
-    byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion");
+    TableName TABLENAME =
+        TableName.valueOf("TestHBACloseRegion");
     createTableWithDefaultConf(TABLENAME);
 
     HRegionInfo info = null;
@@ -1376,7 +1369,8 @@ public class TestAdmin {
 
   @Test
   public void testCloseRegionThatFetchesTheHRIFromMeta() throws Exception {
-    byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion2");
+    TableName TABLENAME =
+        TableName.valueOf("TestHBACloseRegion2");
     createTableWithDefaultConf(TABLENAME);
 
     HRegionInfo info = null;
@@ -1483,7 +1477,7 @@ public class TestAdmin {
     Configuration config = TEST_UTIL.getConfiguration();
     HBaseAdmin admin = new HBaseAdmin(config);
 
-    HTableDescriptor htd = new HTableDescriptor(TABLENAME);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLENAME));
     HColumnDescriptor hcd = new HColumnDescriptor("value");
 
     htd.addFamily(hcd);
@@ -1492,6 +1486,10 @@ public class TestAdmin {
   }
 
   private void createTableWithDefaultConf(byte[] TABLENAME) throws IOException {
+    createTableWithDefaultConf(TableName.valueOf(TABLENAME));
+  }
+
+  private void createTableWithDefaultConf(TableName TABLENAME) throws IOException {
     HTableDescriptor htd = new HTableDescriptor(TABLENAME);
     HColumnDescriptor hcd = new HColumnDescriptor("value");
     htd.addFamily(hcd);
@@ -1515,7 +1513,7 @@ public class TestAdmin {
     byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
 
 
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, startKey, endKey, expectedRegions);
 
@@ -1622,10 +1620,10 @@ public class TestAdmin {
   throws IOException, InterruptedException {
     // When the META table can be opened, the region servers are running
     new HTable(
-      TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME).close();
+      TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME).close();
 
     // Create the test table and open it
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc);
     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
@@ -1698,13 +1696,13 @@ public class TestAdmin {
   @Test
   public void testDisableCatalogTable() throws Exception {
     try {
-      this.admin.disableTable(".META.");
-      fail("Expected to throw IllegalArgumentException");
-    } catch (IllegalArgumentException e) {
+      this.admin.disableTable(TableName.META_TABLE_NAME);
+      fail("Expected to throw ConstraintException");
+    } catch (ConstraintException e) {
     }
     // Before the fix for HBASE-6146, the below table creation was failing as the META table
     // actually getting disabled by the disableTable() call.
-    HTableDescriptor htd = new HTableDescriptor("testDisableCatalogTable".getBytes());
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testDisableCatalogTable".getBytes()));
     HColumnDescriptor hcd = new HColumnDescriptor("cf1".getBytes());
     htd.addFamily(hcd);
     TEST_UTIL.getHBaseAdmin().createTable(htd);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java Thu Aug  8 04:19:49 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
@@ -159,7 +160,8 @@ public class TestClientScannerRPCTimeout
       } else {
         ScanResponse scanRes = super.scan(controller, request);
         String regionName = Bytes.toString(request.getRegion().getValue().toByteArray());
-        if (!regionName.contains("-ROOT-") && !regionName.contains(".META.")) {
+        if (!regionName.contains(TableName.ROOT_TABLE_NAME.getNameAsString()) &&
+            !regionName.contains(TableName.META_TABLE_NAME.getNameAsString())) {
           tableScannerId = scanRes.getScannerId();
         }
         return scanRes;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java Thu Aug  8 04:19:49 2013
@@ -29,6 +29,8 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
@@ -58,7 +60,7 @@ public class TestCloneSnapshotFromClient
   private byte[] snapshotName2;
   private int snapshot0Rows;
   private int snapshot1Rows;
-  private byte[] tableName;
+  private TableName tableName;
   private HBaseAdmin admin;
 
   @BeforeClass
@@ -89,7 +91,7 @@ public class TestCloneSnapshotFromClient
     this.admin = TEST_UTIL.getHBaseAdmin();
 
     long tid = System.currentTimeMillis();
-    tableName = Bytes.toBytes("testtb-" + tid);
+    tableName = TableName.valueOf("testtb-" + tid);
     emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
     snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
     snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
@@ -147,13 +149,13 @@ public class TestCloneSnapshotFromClient
 
   @Test
   public void testCloneSnapshot() throws IOException, InterruptedException {
-    byte[] clonedTableName = Bytes.toBytes("clonedtb-" + System.currentTimeMillis());
+    TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis());
     testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
     testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
     testCloneSnapshot(clonedTableName, emptySnapshot, 0);
   }
 
-  private void testCloneSnapshot(final byte[] tableName, final byte[] snapshotName,
+  private void testCloneSnapshot(final TableName tableName, final byte[] snapshotName,
       int snapshotRows) throws IOException, InterruptedException {
     // create a new table from snapshot
     admin.cloneSnapshot(snapshotName, tableName);
@@ -162,13 +164,24 @@ public class TestCloneSnapshotFromClient
     TEST_UTIL.deleteTable(tableName);
   }
 
+  @Test
+  public void testCloneSnapshotCrossNamespace() throws IOException, InterruptedException {
+    String nsName = "testCloneSnapshotCrossNamespace";
+    admin.createNamespace(NamespaceDescriptor.create(nsName).build());
+    TableName clonedTableName =
+        TableName.valueOf(nsName, "clonedtb-" + System.currentTimeMillis());
+    testCloneSnapshot(clonedTableName, snapshotName0, snapshot0Rows);
+    testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
+    testCloneSnapshot(clonedTableName, emptySnapshot, 0);
+  }
+
   /**
    * Verify that tables created from the snapshot are still alive after source table deletion.
    */
   @Test
   public void testCloneLinksAfterDelete() throws IOException, InterruptedException {
     // Clone a table from the first snapshot
-    byte[] clonedTableName = Bytes.toBytes("clonedtb1-" + System.currentTimeMillis());
+    TableName clonedTableName = TableName.valueOf("clonedtb1-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName0, clonedTableName);
     SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
 
@@ -177,7 +190,7 @@ public class TestCloneSnapshotFromClient
     admin.snapshot(snapshotName2, clonedTableName);
 
     // Clone the snapshot of the cloned table
-    byte[] clonedTableName2 = Bytes.toBytes("clonedtb2-" + System.currentTimeMillis());
+    TableName clonedTableName2 = TableName.valueOf("clonedtb2-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName2, clonedTableName2);
     SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
     admin.disableTable(clonedTableName2);
@@ -204,7 +217,7 @@ public class TestCloneSnapshotFromClient
     SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
 
     // Clone a new table from cloned
-    byte[] clonedTableName3 = Bytes.toBytes("clonedtb3-" + System.currentTimeMillis());
+    TableName clonedTableName3 = TableName.valueOf("clonedtb3-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName2, clonedTableName3);
     SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName3, snapshot0Rows);
 
@@ -217,6 +230,7 @@ public class TestCloneSnapshotFromClient
   // ==========================================================================
   //  Helpers
   // ==========================================================================
+
   private void waitCleanerRun() throws InterruptedException {
     TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Thu Aug  8 04:19:49 2013
@@ -51,6 +51,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -174,7 +175,7 @@ public class TestFromClientSide {
      HColumnDescriptor hcd = new HColumnDescriptor(FAMILY)
          .setKeepDeletedCells(true).setMaxVersions(3);
 
-     HTableDescriptor desc = new HTableDescriptor(TABLENAME);
+     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TABLENAME));
      desc.addFamily(hcd);
      TEST_UTIL.getHBaseAdmin().createTable(desc);
      Configuration c = TEST_UTIL.getConfiguration();
@@ -317,7 +318,8 @@ public class TestFromClientSide {
   @Test
   public void testRegionCachePreWarm() throws Exception {
     LOG.info("Starting testRegionCachePreWarm");
-    final byte [] TABLENAME = Bytes.toBytes("testCachePrewarm");
+    final TableName TABLENAME =
+        TableName.valueOf("testCachePrewarm");
     Configuration conf = TEST_UTIL.getConfiguration();
 
     // Set up test table:
@@ -1212,7 +1214,7 @@ public class TestFromClientSide {
 
     // Null table name (should NOT work)
     try {
-      TEST_UTIL.createTable(null, FAMILY);
+      TEST_UTIL.createTable((TableName)null, FAMILY);
       fail("Creating a table with null name passed, should have failed");
     } catch(Exception e) {}
 
@@ -4081,7 +4083,7 @@ public class TestFromClientSide {
     for (int i = 0; i < tables.length && i < size; i++) {
       boolean found = false;
       for (int j = 0; j < ts.length; j++) {
-        if (Bytes.equals(ts[j].getName(), tables[i])) {
+        if (Bytes.equals(ts[j].getTableName().getName(), tables[i])) {
           found = true;
           break;
         }
@@ -4212,7 +4214,7 @@ public class TestFromClientSide {
     // Test that attribute changes were applied
     desc = a.getTableDescriptor();
     assertTrue("wrong table descriptor returned",
-      Bytes.compareTo(desc.getName(), tableAname) == 0);
+      Bytes.compareTo(desc.getTableName().getName(), tableAname) == 0);
     // check HTD attribute
     value = desc.getValue(attrName);
     assertFalse("missing HTD attribute value", value == null);
@@ -4387,7 +4389,8 @@ public class TestFromClientSide {
   @Test
   public void testIncrementWithDeletes() throws Exception {
     LOG.info("Starting testIncrementWithDeletes");
-    final byte [] TABLENAME = Bytes.toBytes("testIncrementWithDeletes");
+    final TableName TABLENAME =
+        TableName.valueOf("testIncrementWithDeletes");
     HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
     final byte[] COLUMN = Bytes.toBytes("column");
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java Thu Aug  8 04:19:49 2013
@@ -31,6 +31,7 @@ import java.util.Random;
 
 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.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -150,7 +151,8 @@ public class TestFromClientSide3 {
     TEST_UTIL.getConfiguration().setInt("hbase.hstore.compaction.min", 3);
 
     String tableName = "testAdvancedConfigOverride";
-    byte[] TABLE = Bytes.toBytes(tableName);
+    TableName TABLE =
+        TableName.valueOf(tableName);
     HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
     HConnection connection = HConnectionManager.getConnection(TEST_UTIL
@@ -169,7 +171,7 @@ public class TestFromClientSide3 {
       server, regionName, FAMILY).size() > 1);
 
     // Issue a compaction request
-    admin.compact(TABLE);
+    admin.compact(TABLE.getName());
 
     // poll wait for the compactions to happen
     for (int i = 0; i < 10 * 1000 / 40; ++i) {
@@ -205,7 +207,7 @@ public class TestFromClientSide3 {
     performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 10);
 
     // Issue a compaction request
-    admin.compact(TABLE);
+    admin.compact(TABLE.getName());
 
     // This time, the compaction request should not happen
     Thread.sleep(10 * 1000);
@@ -229,7 +231,7 @@ public class TestFromClientSide3 {
     LOG.info("alter status finished");
 
     // Issue a compaction request
-    admin.compact(TABLE);
+    admin.compact(TABLE.getName());
 
     // poll wait for the compactions to happen
     for (int i = 0; i < 10 * 1000 / 40; ++i) {
@@ -380,7 +382,7 @@ public class TestFromClientSide3 {
   public void testGetEmptyRow() throws Exception {
     //Create a table and put in 1 row
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    HTableDescriptor desc = new HTableDescriptor(Bytes.toBytes("test"));
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test")));
     desc.addFamily(new HColumnDescriptor(FAMILY));
     admin.createTable(desc);
     HTable table = new HTable(TEST_UTIL.getConfiguration(), "test");

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java Thu Aug  8 04:19:49 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos;
 import org.junit.Test;
@@ -71,7 +72,8 @@ public class TestHBaseAdminNoCluster {
     // Mock up our admin Interfaces
     HBaseAdmin admin = new HBaseAdmin(configuration);
     try {
-      HTableDescriptor htd = new HTableDescriptor("testMasterMonitorCollableRetries");
+      HTableDescriptor htd =
+          new HTableDescriptor(TableName.valueOf("testMasterMonitorCollableRetries"));
       // Pass any old htable descriptor; not important
       try {
         admin.createTable(htd, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Thu Aug  8 04:19:49 2013
@@ -43,6 +43,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.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -82,11 +83,16 @@ import com.google.common.collect.Lists;
 public class TestHCM {
   private static final Log LOG = LogFactory.getLog(TestHCM.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final byte[] TABLE_NAME = Bytes.toBytes("test");
-  private static final byte[] TABLE_NAME1 = Bytes.toBytes("test1");
-  private static final byte[] TABLE_NAME2 = Bytes.toBytes("test2");
-  private static final byte[] TABLE_NAME3 = Bytes.toBytes("test3");
-  private static final byte[] TABLE_NAME4 = Bytes.toBytes("test4");
+  private static final TableName TABLE_NAME =
+      TableName.valueOf("test");
+  private static final TableName TABLE_NAME1 =
+      TableName.valueOf("test1");
+  private static final TableName TABLE_NAME2 =
+      TableName.valueOf("test2");
+  private static final TableName TABLE_NAME3 =
+      TableName.valueOf("test3");
+  private static final TableName TABLE_NAME4 =
+      TableName.valueOf("test4");
   private static final byte[] FAM_NAM = Bytes.toBytes("f");
   private static final byte[] ROW = Bytes.toBytes("bbb");
   private static final byte[] ROW_X = Bytes.toBytes("xxx");
@@ -149,7 +155,7 @@ public class TestHCM {
     // and that the table is using it
     assertTrue(t.getPool() == pool);
     t.close();
-    
+
     t = (HTable)con1.getTable(tableName);
     // still using the *same* internal pool
     assertTrue(t.getPool() == pool);
@@ -167,7 +173,8 @@ public class TestHCM {
 
   @Ignore ("Fails in IDEs: HBASE-9042") @Test(expected = RegionServerStoppedException.class)
   public void testClusterStatus() throws Exception {
-    byte[] tn = "testClusterStatus".getBytes();
+    TableName tn =
+        TableName.valueOf("testClusterStatus");
     byte[] cf = "cf".getBytes();
     byte[] rk = "rk1".getBytes();
 
@@ -176,7 +183,7 @@ public class TestHCM {
     final ServerName sn = rs.getRegionServer().getServerName();
 
     HTable t = TEST_UTIL.createTable(tn, cf);
-    TEST_UTIL.waitTableAvailable(tn);
+    TEST_UTIL.waitTableAvailable(tn.getName());
 
     while(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
         getRegionStates().isRegionsInTransition()){
@@ -342,7 +349,7 @@ public class TestHCM {
     HTable table = new HTable(conf, TABLE_NAME);
 
     TEST_UTIL.createMultiRegions(table, FAM_NAM);
-    TEST_UTIL.waitUntilAllRegionsAssigned(table.getTableName());
+    TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
     Put put = new Put(ROW);
     put.add(FAM_NAM, ROW, ROW);
     table.put(put);
@@ -350,9 +357,6 @@ public class TestHCM {
       (HConnectionManager.HConnectionImplementation)table.getConnection();
 
     assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
-    assertNotNull(conn.getCachedLocation(TABLE_NAME.clone(), ROW.clone()));
-    assertNotNull(conn.getCachedLocation(
-      Bytes.toString(TABLE_NAME).getBytes() , Bytes.toString(ROW).getBytes()));
 
     final int nextPort = conn.getCachedLocation(TABLE_NAME, ROW).getPort() + 1;
     HRegionLocation loc = conn.getCachedLocation(TABLE_NAME, ROW);
@@ -360,7 +364,7 @@ public class TestHCM {
       HConstants.LATEST_TIMESTAMP), HConstants.LATEST_TIMESTAMP);
     Assert.assertEquals(conn.getCachedLocation(TABLE_NAME, ROW).getPort(), nextPort);
 
-    conn.forceDeleteCachedLocation(TABLE_NAME.clone(), ROW.clone());
+    conn.forceDeleteCachedLocation(TABLE_NAME, ROW.clone());
     HRegionLocation rl = conn.getCachedLocation(TABLE_NAME, ROW);
     assertNull("What is this location?? " + rl, rl);
 
@@ -372,6 +376,7 @@ public class TestHCM {
     put2.add(FAM_NAM, ROW, ROW);
     table.put(put2);
     assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
+    assertNotNull(conn.getCachedLocation(TableName.valueOf(TABLE_NAME.getName()), ROW.clone()));
 
     TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, false);
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
@@ -514,12 +519,11 @@ public class TestHCM {
   public void testConnectionManagement() throws Exception{
     TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM);
     HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
-    HTableInterface table = conn.getTable(TABLE_NAME1);
-          //new HTable(TABLE_NAME1, conn, pool);
+    HTableInterface table = conn.getTable(TABLE_NAME1.getName());
     table.close();
     assertFalse(conn.isClosed());
     assertFalse(((HTable)table).getPool().isShutdown());
-    table = conn.getTable(TABLE_NAME1);
+    table = conn.getTable(TABLE_NAME1.getName());
     table.close();
     assertFalse(((HTable)table).getPool().isShutdown());
     conn.close();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java Thu Aug  8 04:19:49 2013
@@ -27,6 +27,7 @@ import java.util.List;
 
 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.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -64,7 +65,8 @@ public class TestHTableMultiplexer {
 
   @Test
   public void testHTableMultiplexer() throws Exception {
-    byte[] TABLE = Bytes.toBytes("testHTableMultiplexer");
+    TableName TABLE =
+        TableName.valueOf("testHTableMultiplexer");
     final int NUM_REGIONS = 10;
     final int VERSION = 3;
     List<Put> failedPuts;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java Thu Aug  8 04:19:49 2013
@@ -261,7 +261,7 @@ public class TestHTablePool {
 				admin.deleteTable(TABLENAME);
 			}
 
-			HTableDescriptor tableDescriptor = new HTableDescriptor(TABLENAME);
+			HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(TABLENAME));
 			tableDescriptor.addFamily(new HColumnDescriptor("randomFamily"));
 			admin.createTable(tableDescriptor);
 
@@ -337,7 +337,7 @@ public class TestHTablePool {
 				admin.deleteTable(TABLENAME);
 			}
 
-			HTableDescriptor tableDescriptor = new HTableDescriptor(TABLENAME);
+			HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(TABLENAME));
 			tableDescriptor.addFamily(new HColumnDescriptor("randomFamily"));
 			admin.createTable(tableDescriptor);
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java Thu Aug  8 04:19:49 2013
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.HTestConst;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.junit.Test;
@@ -51,8 +52,8 @@ public class TestIntraRowPagination {
     byte [][] FAMILIES = HTestConst.makeNAscii(HTestConst.DEFAULT_CF_BYTES, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(HTestConst.DEFAULT_QUALIFIER_BYTES, 10);
 
-    HTableDescriptor htd = new HTableDescriptor(HTestConst.DEFAULT_TABLE_BYTES);
-    HRegionInfo info = new HRegionInfo(HTestConst.DEFAULT_TABLE_BYTES, null, null, false);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(HTestConst.DEFAULT_TABLE_BYTES));
+    HRegionInfo info = new HRegionInfo(HTestConst.DEFAULT_TABLE, null, null, false);
     for (byte[] family : FAMILIES) {
       HColumnDescriptor hcd = new HColumnDescriptor(family);
       htd.addFamily(hcd);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java Thu Aug  8 04:19:49 2013
@@ -34,6 +34,7 @@ import java.util.Random;
 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;
@@ -67,7 +68,8 @@ public class TestMetaScanner {
   public void testMetaScanner() throws Exception {
     LOG.info("Starting testMetaScanner");
     setUp();
-    final byte[] TABLENAME = Bytes.toBytes("testMetaScanner");
+    final TableName TABLENAME =
+        TableName.valueOf("testMetaScanner");
     final byte[] FAMILY = Bytes.toBytes("family");
     TEST_UTIL.createTable(TABLENAME, FAMILY);
     Configuration conf = TEST_UTIL.getConfiguration();
@@ -121,7 +123,8 @@ public class TestMetaScanner {
 
     final long runtime = 30 * 1000; //30 sec
     LOG.info("Starting testConcurrentMetaScannerAndCatalogJanitor");
-    final byte[] TABLENAME = Bytes.toBytes("testConcurrentMetaScannerAndCatalogJanitor");
+    final TableName TABLENAME =
+        TableName.valueOf("testConcurrentMetaScannerAndCatalogJanitor");
     final byte[] FAMILY = Bytes.toBytes("family");
     TEST_UTIL.createTable(TABLENAME, FAMILY);
     final CatalogTracker catalogTracker = mock(CatalogTracker.class);
@@ -139,7 +142,7 @@ public class TestMetaScanner {
 
             //select a random region
             HRegionInfo parent = regions.get(random.nextInt(regions.size()));
-            if (parent == null || !Bytes.equals(TABLENAME, parent.getTableName())) {
+            if (parent == null || !TABLENAME.equals(parent.getTableName())) {
               continue;
             }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java Thu Aug  8 04:19:49 2013
@@ -80,8 +80,9 @@ public class TestMultipleTimestamps {
 
   @Test
   public void testReseeksWithOneColumnMiltipleTimestamp() throws IOException {
-    byte [] TABLE = Bytes.toBytes("testReseeksWithOne" +
-    "ColumnMiltipleTimestamps");
+    TableName TABLE =
+        TableName.valueOf("testReseeksWithOne" +
+            "ColumnMiltipleTimestamps");
     byte [] FAMILY = Bytes.toBytes("event_log");
     byte [][] FAMILIES = new byte[][] { FAMILY };
 
@@ -121,8 +122,9 @@ public class TestMultipleTimestamps {
   @Test
   public void testReseeksWithMultipleColumnOneTimestamp() throws IOException {
     LOG.info("testReseeksWithMultipleColumnOneTimestamp");
-    byte [] TABLE = Bytes.toBytes("testReseeksWithMultiple" +
-    "ColumnOneTimestamps");
+    TableName TABLE =
+        TableName.valueOf("testReseeksWithMultiple" +
+            "ColumnOneTimestamps");
     byte [] FAMILY = Bytes.toBytes("event_log");
     byte [][] FAMILIES = new byte[][] { FAMILY };
 
@@ -162,7 +164,8 @@ public class TestMultipleTimestamps {
   IOException {
     LOG.info("testReseeksWithMultipleColumnMultipleTimestamp");
 
-    byte [] TABLE = Bytes.toBytes("testReseeksWithMultipleColumnMiltipleTimestamps");
+    TableName TABLE =
+        TableName.valueOf("testReseeksWithMultipleColumnMiltipleTimestamps");
     byte [] FAMILY = Bytes.toBytes("event_log");
     byte [][] FAMILIES = new byte[][] { FAMILY };
 
@@ -214,7 +217,8 @@ public class TestMultipleTimestamps {
   @Test
   public void testReseeksWithMultipleFiles() throws IOException {
     LOG.info("testReseeksWithMultipleFiles");
-    byte [] TABLE = Bytes.toBytes("testReseeksWithMultipleFiles");
+    TableName TABLE =
+        TableName.valueOf("testReseeksWithMultipleFiles");
     byte [] FAMILY = Bytes.toBytes("event_log");
     byte [][] FAMILIES = new byte[][] { FAMILY };
 
@@ -282,7 +286,9 @@ public class TestMultipleTimestamps {
 
   public void testWithVersionDeletes(boolean flushTables) throws IOException {
     LOG.info("testWithVersionDeletes_"+ (flushTables ? "flush" : "noflush"));
-    byte [] TABLE = Bytes.toBytes("testWithVersionDeletes_" + (flushTables ? "flush" : "noflush"));
+    TableName TABLE =
+        TableName.valueOf("testWithVersionDeletes_" + (flushTables ?
+            "flush" : "noflush"));
     byte [] FAMILY = Bytes.toBytes("event_log");
     byte [][] FAMILIES = new byte[][] { FAMILY };
 
@@ -315,7 +321,8 @@ public class TestMultipleTimestamps {
   public void testWithMultipleVersionDeletes() throws IOException {
     LOG.info("testWithMultipleVersionDeletes");
 
-    byte [] TABLE = Bytes.toBytes("testWithMultipleVersionDeletes");
+    TableName TABLE =
+        TableName.valueOf("testWithMultipleVersionDeletes");
     byte [] FAMILY = Bytes.toBytes("event_log");
     byte [][] FAMILIES = new byte[][] { FAMILY };
 
@@ -340,7 +347,8 @@ public class TestMultipleTimestamps {
 
   @Test
   public void testWithColumnDeletes() throws IOException {
-    byte [] TABLE = Bytes.toBytes("testWithColumnDeletes");
+    TableName TABLE =
+        TableName.valueOf("testWithColumnDeletes");
     byte [] FAMILY = Bytes.toBytes("event_log");
     byte [][] FAMILIES = new byte[][] { FAMILY };
 
@@ -365,7 +373,8 @@ public class TestMultipleTimestamps {
 
   @Test
   public void testWithFamilyDeletes() throws IOException {
-    byte [] TABLE = Bytes.toBytes("testWithFamilyDeletes");
+    TableName TABLE =
+        TableName.valueOf("testWithFamilyDeletes");
     byte [] FAMILY = Bytes.toBytes("event_log");
     byte [][] FAMILIES = new byte[][] { FAMILY };
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java Thu Aug  8 04:19:49 2013
@@ -27,6 +27,7 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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;
@@ -62,7 +63,7 @@ public class TestRestoreSnapshotFromClie
   private byte[] snapshotName2;
   private int snapshot0Rows;
   private int snapshot1Rows;
-  private byte[] tableName;
+  private TableName tableName;
   private HBaseAdmin admin;
 
   @BeforeClass
@@ -93,7 +94,8 @@ public class TestRestoreSnapshotFromClie
     this.admin = TEST_UTIL.getHBaseAdmin();
 
     long tid = System.currentTimeMillis();
-    tableName = Bytes.toBytes("testtb-" + tid);
+    tableName =
+        TableName.valueOf("testtb-" + tid);
     emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
     snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
     snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
@@ -221,7 +223,8 @@ public class TestRestoreSnapshotFromClie
 
   @Test
   public void testRestoreSnapshotOfCloned() throws IOException, InterruptedException {
-    byte[] clonedTableName = Bytes.toBytes("clonedtb-" + System.currentTimeMillis());
+    TableName clonedTableName =
+        TableName.valueOf("clonedtb-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName0, clonedTableName);
     SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
     admin.disableTable(clonedTableName);
@@ -241,10 +244,10 @@ public class TestRestoreSnapshotFromClie
     TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
   }
 
-  private Set<String> getFamiliesFromFS(final byte[] tableName) throws IOException {
+  private Set<String> getFamiliesFromFS(final TableName tableName) throws IOException {
     MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     Set<String> families = new HashSet<String>();
-    Path tableDir = HTableDescriptor.getTableDir(mfs.getRootDir(), tableName);
+    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
     for (Path regionDir: FSUtils.getRegionDirs(mfs.getFileSystem(), tableDir)) {
       for (Path familyDir: FSUtils.getFamilyDirs(mfs.getFileSystem(), regionDir)) {
         families.add(familyDir.getName());

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java Thu Aug  8 04:19:49 2013
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -190,25 +191,26 @@ public class TestSnapshotCloneIndependen
 
     HBaseAdmin admin = UTIL.getHBaseAdmin();
     final long startTime = System.currentTimeMillis();
-    final String localTableNameAsString = STRING_TABLE_NAME + startTime;
+    final TableName localTableName =
+        TableName.valueOf(STRING_TABLE_NAME + startTime);
 
-    HTable original = UTIL.createTable(Bytes.toBytes(localTableNameAsString), TEST_FAM);
+    HTable original = UTIL.createTable(localTableName, TEST_FAM);
     try {
 
       UTIL.loadTable(original, TEST_FAM);
       final int origTableRowCount = UTIL.countRows(original);
 
       // Take a snapshot
-      final String snapshotNameAsString = "snapshot_" + localTableNameAsString;
+      final String snapshotNameAsString = "snapshot_" + localTableName;
       byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
 
-      SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableNameAsString, TEST_FAM_STR,
+      SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
         snapshotNameAsString, rootDir, fs, online);
 
       if (!online) {
-        admin.enableTable(localTableNameAsString);
+        admin.enableTable(localTableName);
       }
-      byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableNameAsString);
+      byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableName);
       admin.cloneSnapshot(snapshotName, cloneTableName);
 
       HTable clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName);
@@ -267,23 +269,24 @@ public class TestSnapshotCloneIndependen
     // Create a table
     HBaseAdmin admin = UTIL.getHBaseAdmin();
     final long startTime = System.currentTimeMillis();
-    final String localTableNameAsString = STRING_TABLE_NAME + startTime;
-    HTable original = UTIL.createTable(Bytes.toBytes(localTableNameAsString), TEST_FAM);
+    final TableName localTableName =
+        TableName.valueOf(STRING_TABLE_NAME + startTime);
+    HTable original = UTIL.createTable(localTableName, TEST_FAM);
     UTIL.loadTable(original, TEST_FAM);
     final int loadedTableCount = UTIL.countRows(original);
     System.out.println("Original table has: " + loadedTableCount + " rows");
 
-    final String snapshotNameAsString = "snapshot_" + localTableNameAsString;
+    final String snapshotNameAsString = "snapshot_" + localTableName;
 
     // Create a snapshot
-    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableNameAsString, TEST_FAM_STR,
+    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
       snapshotNameAsString, rootDir, fs, online);
 
     if (!online) {
-      admin.enableTable(localTableNameAsString);
+      admin.enableTable(localTableName);
     }
 
-    byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableNameAsString);
+    byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableName);
 
     // Clone the snapshot
     byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
@@ -291,8 +294,7 @@ public class TestSnapshotCloneIndependen
 
     // Verify that region information is the same pre-split
     original.clearRegionCache();
-    List<HRegionInfo> originalTableHRegions = admin.getTableRegions(Bytes
-        .toBytes(localTableNameAsString));
+    List<HRegionInfo> originalTableHRegions = admin.getTableRegions(localTableName);
 
     final int originalRegionCount = originalTableHRegions.size();
     final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
@@ -323,20 +325,21 @@ public class TestSnapshotCloneIndependen
     // Create a table
     HBaseAdmin admin = UTIL.getHBaseAdmin();
     final long startTime = System.currentTimeMillis();
-    final String localTableNameAsString = STRING_TABLE_NAME + startTime;
-    HTable original = UTIL.createTable(Bytes.toBytes(localTableNameAsString), TEST_FAM);
+    final TableName localTableName =
+        TableName.valueOf(STRING_TABLE_NAME + startTime);
+    HTable original = UTIL.createTable(localTableName, TEST_FAM);
     UTIL.loadTable(original, TEST_FAM);
 
-    final String snapshotNameAsString = "snapshot_" + localTableNameAsString;
+    final String snapshotNameAsString = "snapshot_" + localTableName;
 
     // Create a snapshot
-    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableNameAsString, TEST_FAM_STR,
+    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
       snapshotNameAsString, rootDir, fs, online);
 
     if (!online) {
-      admin.enableTable(localTableNameAsString);
+      admin.enableTable(localTableName);
     }
-    byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableNameAsString);
+    byte[] cloneTableName = Bytes.toBytes("test-clone-" + localTableName);
 
     // Clone the snapshot
     byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
@@ -346,11 +349,11 @@ public class TestSnapshotCloneIndependen
     byte[] TEST_FAM_2 = Bytes.toBytes("fam2");
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM_2);
 
-    admin.disableTable(localTableNameAsString);
-    admin.addColumn(localTableNameAsString, hcd);
+    admin.disableTable(localTableName);
+    admin.addColumn(localTableName, hcd);
 
     // Verify that it is not in the snapshot
-    admin.enableTable(localTableNameAsString);
+    admin.enableTable(localTableName);
 
     // get a description of the cloned table
     // get a list of its families

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java Thu Aug  8 04:19:49 2013
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import java.io.IOException;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -30,6 +29,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LargeTests;
@@ -61,7 +61,8 @@ public class TestSnapshotFromClient {
   private static final int NUM_RS = 2;
   private static final String STRING_TABLE_NAME = "test";
   private static final byte[] TEST_FAM = Bytes.toBytes("fam");
-  private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
+  private static final TableName TABLE_NAME =
+      TableName.valueOf(STRING_TABLE_NAME);
 
   /**
    * Setup the config for the cluster
@@ -121,14 +122,14 @@ public class TestSnapshotFromClient {
     byte[] snapshotName = Bytes.toBytes("metaSnapshot");
 
     try {
-      admin.snapshot(snapshotName, HConstants.META_TABLE_NAME);
+      admin.snapshot(snapshotName, TableName.META_TABLE_NAME);
       fail("taking a snapshot of .META. should not be allowed");
     } catch (IllegalArgumentException e) {
       // expected
     }
 
     try {
-      admin.snapshot(snapshotName, HConstants.ROOT_TABLE_NAME);
+      admin.snapshot(snapshotName, TableName.ROOT_TABLE_NAME);
       fail("taking a snapshot of -ROOT- should not be allowed");
     } catch (IllegalArgumentException e) {
       // expected

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java Thu Aug  8 04:19:49 2013
@@ -27,6 +27,7 @@ import static org.junit.Assert.assertTru
 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;
@@ -82,8 +83,7 @@ public class TestSnapshotMetadata {
   private HBaseAdmin admin;
   private String originalTableDescription;
   private HTableDescriptor originalTableDescriptor;
-  private byte[] originalTableName;
-  private String originalTableNameAsString;
+  TableName originalTableName;
 
   private static FileSystem fs;
   private static Path rootDir;
@@ -136,7 +136,7 @@ public class TestSnapshotMetadata {
 
     final long startTime = System.currentTimeMillis();
     final String sourceTableNameAsString = STRING_TABLE_NAME + startTime;
-    originalTableName = Bytes.toBytes(sourceTableNameAsString);
+    originalTableName = TableName.valueOf(sourceTableNameAsString);
 
     // enable replication on a column family
     HColumnDescriptor maxVersionsColumn = new HColumnDescriptor(MAX_VERSIONS_FAM);
@@ -149,7 +149,7 @@ public class TestSnapshotMetadata {
     dataBlockColumn.setDataBlockEncoding(DATA_BLOCK_ENCODING_TYPE);
     blockSizeColumn.setBlocksize(BLOCK_SIZE);
 
-    HTableDescriptor htd = new HTableDescriptor(sourceTableNameAsString);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(sourceTableNameAsString));
     htd.addFamily(maxVersionsColumn);
     htd.addFamily(bloomFilterColumn);
     htd.addFamily(dataBlockColumn);
@@ -160,8 +160,7 @@ public class TestSnapshotMetadata {
 
     admin.createTable(htd);
     HTable original = new HTable(UTIL.getConfiguration(), originalTableName);
-
-    originalTableNameAsString = sourceTableNameAsString;
+    originalTableName = TableName.valueOf(sourceTableNameAsString);
     originalTableDescriptor = admin.getTableDescriptor(originalTableName);
     originalTableDescription = originalTableDescriptor.toStringCustomizedValues();
 
@@ -175,9 +174,9 @@ public class TestSnapshotMetadata {
   @Test (timeout=300000)
   public void testDescribeMatchesAfterClone() throws Exception {
     // Clone the original table
-    final String clonedTableNameAsString = "clone" + originalTableNameAsString;
+    final String clonedTableNameAsString = "clone" + originalTableName;
     final byte[] clonedTableName = Bytes.toBytes(clonedTableNameAsString);
-    final String snapshotNameAsString = "snapshot" + originalTableNameAsString
+    final String snapshotNameAsString = "snapshot" + originalTableName
         + System.currentTimeMillis();
     final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
 
@@ -188,14 +187,14 @@ public class TestSnapshotMetadata {
     }
 
     // Create a snapshot in which all families are empty
-    SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableNameAsString, null,
+    SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableName, null,
       familiesList, snapshotNameAsString, rootDir, fs);
 
     admin.cloneSnapshot(snapshotName, clonedTableName);
     HTable clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName);
     HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName);
     assertEquals(
-      originalTableDescription.replace(originalTableNameAsString, clonedTableNameAsString),
+      originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
       cloneHtd.toStringCustomizedValues());
 
     // Verify the custom fields
@@ -273,11 +272,11 @@ public class TestSnapshotMetadata {
     }
 
     // take a snapshot
-    final String snapshotNameAsString = "snapshot" + originalTableNameAsString
+    final String snapshotNameAsString = "snapshot" + originalTableName
         + System.currentTimeMillis();
     final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
 
-    SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableNameAsString,
+    SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableName,
       familiesWithDataList, emptyFamiliesList, snapshotNameAsString, rootDir, fs);
 
     admin.enableTable(originalTableName);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java Thu Aug  8 04:19:49 2013
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
@@ -69,7 +70,7 @@ public class TestConstraint {
   public void testConstraintPasses() throws Exception {
     // create the table
     // it would be nice if this was also a method on the util
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     for (byte[] family : new byte[][] { dummy, test }) {
       desc.addFamily(new HColumnDescriptor(family));
     }
@@ -99,7 +100,7 @@ public class TestConstraint {
 
     // create the table
     // it would be nice if this was also a method on the util
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     for (byte[] family : new byte[][] { dummy, test }) {
       desc.addFamily(new HColumnDescriptor(family));
     }
@@ -137,7 +138,7 @@ public class TestConstraint {
   @Test
   public void testDisableConstraint() throws Throwable {
     // create the table
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     // add a family to the table
     for (byte[] family : new byte[][] { dummy, test }) {
       desc.addFamily(new HColumnDescriptor(family));
@@ -171,7 +172,7 @@ public class TestConstraint {
   @Test
   public void testDisableConstraints() throws Throwable {
     // create the table
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     // add a family to the table
     for (byte[] family : new byte[][] { dummy, test }) {
       desc.addFamily(new HColumnDescriptor(family));
@@ -202,7 +203,7 @@ public class TestConstraint {
   @Test
   public void testIsUnloaded() throws Exception {
     // create the table
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     // add a family to the table
     for (byte[] family : new byte[][] { dummy, test }) {
       desc.addFamily(new HColumnDescriptor(family));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java Thu Aug  8 04:19:49 2013
@@ -27,6 +27,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.constraint.TestConstraint.CheckWasRunConstraint;
 import org.apache.hadoop.hbase.constraint.WorksConstraint.NameConstraint;
@@ -43,7 +44,7 @@ public class TestConstraints {
   @SuppressWarnings("unchecked")
   @Test
   public void testSimpleReadWrite() throws Throwable {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     Constraints.add(desc, WorksConstraint.class);
 
     List<? extends Constraint> constraints = Constraints.getConstraints(desc,
@@ -68,7 +69,7 @@ public class TestConstraints {
   @SuppressWarnings("unchecked")
   @Test
   public void testReadWriteWithConf() throws Throwable {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     Constraints.add(
         desc,
         new Pair<Class<? extends Constraint>, Configuration>(
@@ -101,7 +102,7 @@ public class TestConstraints {
   @SuppressWarnings("unchecked")
   @Test
   public void testEnableDisableRemove() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     // check general enabling/disabling of constraints
     // first add a constraint
     Constraints.add(desc, AllPassConstraint.class);
@@ -137,7 +138,7 @@ public class TestConstraints {
   @SuppressWarnings("unchecked")
   @Test
   public void testUpdateConstraint() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     Constraints.add(desc, CheckConfigurationConstraint.class,
         CheckWasRunConstraint.class);
     Constraints.setConfiguration(desc, CheckConfigurationConstraint.class,
@@ -163,7 +164,7 @@ public class TestConstraints {
    */
   @Test
   public void testRemoveUnsetConstraint() throws Throwable {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     Constraints.remove(desc);
     Constraints.remove(desc, AlsoWorks.class);
   }
@@ -173,7 +174,7 @@ public class TestConstraints {
     Configuration conf = new Configuration();
     conf.setBoolean("_ENABLED", false);
     conf.setLong("_PRIORITY", 10);
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     Constraints.add(desc, AlsoWorks.class, conf);
     Constraints.add(desc, WorksConstraint.class);
     assertFalse(Constraints.enabled(desc, AlsoWorks.class));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java Thu Aug  8 04:19:49 2013
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Class for testing WALObserver coprocessor.
@@ -86,7 +87,7 @@ implements WALObserver {
       HRegionInfo info, HLogKey logKey, WALEdit logEdit) throws IOException {
     boolean bypass = false;
     // check table name matches or not.
-    if (!Arrays.equals(HRegionInfo.getTableName(info.getRegionName()), this.tableName)) {
+    if (!Bytes.equals(info.getTableName().getName(), this.tableName)) {
       return bypass;
     }
     preWALWriteCalled = true;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java Thu Aug  8 04:19:49 2013
@@ -27,7 +27,6 @@ import static org.junit.Assert.assertTru
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-import java.util.Arrays;
 import java.util.NavigableSet;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -289,7 +288,7 @@ public class SimpleRegionObserver extend
     assertNotNull(e.getRegion());
     assertNotNull(get);
     assertNotNull(results);
-    if (Arrays.equals(e.getRegion().getTableDesc().getName(),
+    if (e.getRegion().getTableDesc().getTableName().equals(
         TestRegionObserverInterface.TEST_TABLE)) {
       boolean foundA = false;
       boolean foundB = false;
@@ -321,7 +320,7 @@ public class SimpleRegionObserver extend
     assertNotNull(e);
     assertNotNull(e.getRegion());
     assertNotNull(familyMap);
-    if (Arrays.equals(e.getRegion().getTableDesc().getName(),
+    if (e.getRegion().getTableDesc().getTableName().equals(
         TestRegionObserverInterface.TEST_TABLE)) {
       List<? extends Cell> cells = familyMap.get(TestRegionObserverInterface.A);
       assertNotNull(cells);
@@ -355,7 +354,7 @@ public class SimpleRegionObserver extend
     assertNotNull(e.getRegion());
     assertNotNull(familyMap);
     List<? extends Cell> cells = familyMap.get(TestRegionObserverInterface.A);
-    if (Arrays.equals(e.getRegion().getTableDesc().getName(),
+    if (e.getRegion().getTableDesc().getTableName().equals(
         TestRegionObserverInterface.TEST_TABLE)) {
       assertNotNull(cells);
       assertNotNull(cells.get(0));
@@ -471,7 +470,7 @@ public class SimpleRegionObserver extend
     RegionCoprocessorEnvironment e = ctx.getEnvironment();
     assertNotNull(e);
     assertNotNull(e.getRegion());
-    if (Arrays.equals(e.getRegion().getTableDesc().getName(),
+    if (e.getRegion().getTableDesc().getTableName().equals(
         TestRegionObserverInterface.TEST_TABLE)) {
       assertNotNull(familyPaths);
       assertEquals(1,familyPaths.size());
@@ -489,7 +488,7 @@ public class SimpleRegionObserver extend
     RegionCoprocessorEnvironment e = ctx.getEnvironment();
     assertNotNull(e);
     assertNotNull(e.getRegion());
-    if (Arrays.equals(e.getRegion().getTableDesc().getName(),
+    if (e.getRegion().getTableDesc().getTableName().equals(
         TestRegionObserverInterface.TEST_TABLE)) {
       assertNotNull(familyPaths);
       assertEquals(1,familyPaths.size());

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java Thu Aug  8 04:19:49 2013
@@ -51,7 +51,8 @@ public class TestAggregateProtocol {
   /**
    * Creating the test infrastructure.
    */
-  private static final byte[] TEST_TABLE = Bytes.toBytes("TestTable");
+  private static final TableName TEST_TABLE =
+      TableName.valueOf("TestTable");
   private static final byte[] TEST_FAMILY = Bytes.toBytes("TestFamily");
   private static final byte[] TEST_QUALIFIER = Bytes.toBytes("TestQualifier");
   private static final byte[] TEST_MULTI_CQ = Bytes.toBytes("TestMultiCQ");
@@ -847,4 +848,4 @@ public class TestAggregateProtocol {
     std = aClient.std(TEST_TABLE, ci, scan);
     assertEquals(Double.NaN, std, 0);
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java Thu Aug  8 04:19:49 2013
@@ -49,7 +49,8 @@ public class TestBigDecimalColumnInterpr
   /**
    * Creating the test infrastructure.
    */
-  private static final byte[] TEST_TABLE = Bytes.toBytes("TestTable");
+  private static final TableName TEST_TABLE =
+      TableName.valueOf("TestTable");
   private static final byte[] TEST_FAMILY = Bytes.toBytes("TestFamily");
   private static final byte[] TEST_QUALIFIER = Bytes.toBytes("TestQualifier");
   private static final byte[] TEST_MULTI_CQ = Bytes.toBytes("TestMultiCQ");

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java Thu Aug  8 04:19:49 2013
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.RegionLoa
 
 import java.io.*;
 import java.util.*;
-import java.util.jar.*;
 
 import org.junit.*;
 import org.junit.experimental.categories.Category;
@@ -139,7 +138,7 @@ public class TestClassLoading {
     LOG.info("Copied jar file to HDFS: " + jarFileOnHDFS2);
 
     // create a table that references the coprocessors
-    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
     htd.addFamily(new HColumnDescriptor("test"));
       // without configuration values
     htd.setValue("COPROCESSOR$1", jarFileOnHDFS1.toString() + "|" + cpName1 +
@@ -161,7 +160,7 @@ public class TestClassLoading {
     byte[] startKey = {10, 63};
     byte[] endKey = {12, 43};
     admin.createTable(htd, startKey, endKey, 4);
-    waitForTable(htd.getName());
+    waitForTable(htd.getTableName());
 
     // verify that the coprocessors were loaded
     boolean foundTableRegion=false;
@@ -233,13 +232,13 @@ public class TestClassLoading {
     File jarFile = buildCoprocessorJar(cpName3);
 
     // create a table that references the jar
-    HTableDescriptor htd = new HTableDescriptor(cpName3);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(cpName3));
     htd.addFamily(new HColumnDescriptor("test"));
     htd.setValue("COPROCESSOR$1", getLocalPath(jarFile) + "|" + cpName3 + "|" +
       Coprocessor.PRIORITY_USER);
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     admin.createTable(htd);
-    waitForTable(htd.getName());
+    waitForTable(htd.getTableName());
 
     // verify that the coprocessor was loaded
     boolean found = false;
@@ -259,13 +258,13 @@ public class TestClassLoading {
     File jarFile = buildCoprocessorJar(cpName4);
 
     // create a table that references the jar
-    HTableDescriptor htd = new HTableDescriptor(cpName4);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(cpName4));
     htd.addFamily(new HColumnDescriptor("test"));
     htd.setValue("COPROCESSOR$1", getLocalPath(jarFile) + "|" + cpName4 + "|" +
       Coprocessor.PRIORITY_USER);
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     admin.createTable(htd);
-    waitForTable(htd.getName());
+    waitForTable(htd.getTableName());
 
     // verify that the coprocessor was loaded correctly
     boolean found = false;
@@ -307,7 +306,7 @@ public class TestClassLoading {
         " | org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver | | k=v ";
 
     // create a table that references the jar
-    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
     htd.addFamily(new HColumnDescriptor("test"));
 
     // add 3 coprocessors by setting htd attributes directly.
@@ -333,7 +332,7 @@ public class TestClassLoading {
       admin.deleteTable(tableName);
     }
     admin.createTable(htd);
-    waitForTable(htd.getName());
+    waitForTable(htd.getTableName());
 
     // verify that the coprocessor was loaded
     boolean found_2 = false, found_1 = false, found_3 = false,
@@ -409,7 +408,7 @@ public class TestClassLoading {
     LOG.info("Copied jar file to HDFS: " + jarFileOnHDFS);
 
     // create a table that references the coprocessors
-    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
     htd.addFamily(new HColumnDescriptor("test"));
       // without configuration values
     htd.setValue("COPROCESSOR$1", jarFileOnHDFS.toString() + "|" + cpName1 +
@@ -425,7 +424,7 @@ public class TestClassLoading {
       admin.deleteTable(tableName);
     }
     admin.createTable(htd);
-    waitForTable(htd.getName());
+    waitForTable(htd.getTableName());
 
     // verify that the coprocessors were loaded
     boolean found1 = false, found2 = false, found2_k1 = false,
@@ -542,9 +541,9 @@ public class TestClassLoading {
     assertEquals(loadedMasterCoprocessorsVerify, loadedMasterCoprocessors);
   }
 
-  private void waitForTable(byte[] name) throws InterruptedException, IOException {
+  private void waitForTable(TableName name) throws InterruptedException, IOException {
     // First wait until all regions are online
-    TEST_UTIL.waitTableEnabled(name);
+    TEST_UTIL.waitTableEnabled(name.getName());
     // Now wait a bit longer for the coprocessor hosts to load the CPs
     Thread.sleep(1000);
   }