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 08:08:31 UTC

svn commit: r1511591 [21/23] - in /hbase/branches/0.95: 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/jav...

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java Thu Aug  8 06:08:23 2013
@@ -42,6 +42,11 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.log4j.Level;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -68,12 +73,8 @@ import org.apache.hadoop.hbase.util.FSUt
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
-import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -114,14 +115,15 @@ public class TestHLogSplit {
   private static final int NUM_WRITERS = 10;
   private static final int ENTRIES = 10; // entries per writer per region
 
-  private static final byte[] TABLE_NAME = "t1".getBytes();
+  private static final TableName TABLE_NAME =
+      TableName.valueOf("t1");
   private static final byte[] FAMILY = "f1".getBytes();
   private static final byte[] QUALIFIER = "q1".getBytes();
   private static final byte[] VALUE = "v1".getBytes();
   private static final String HLOG_FILE_PREFIX = "hlog.dat.";
   private static List<String> REGIONS = new ArrayList<String>();
   private static final String HBASE_SKIP_ERRORS = "hbase.hlog.split.skip.errors";
-  private static final Path TABLEDIR = new Path(HBASEDIR, Bytes.toString(TABLE_NAME));
+  private static final Path TABLEDIR = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
   private static String ROBBER;
   private static String ZOMBIE;
   private static String [] GROUP = new String [] {"supergroup"};
@@ -336,14 +338,14 @@ public class TestHLogSplit {
   public void testRecoveredEditsPathForMeta() throws IOException {
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
     byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
-    Path tdir = new Path(HBASEDIR, Bytes.toString(HConstants.META_TABLE_NAME));
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
     Path regiondir = new Path(tdir,
         HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
     fs.mkdirs(regiondir);
     long now = System.currentTimeMillis();
     HLog.Entry entry =
         new HLog.Entry(new HLogKey(encoded,
-            HConstants.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
+            TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
       new WALEdit());
     Path p = HLogSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR, true);
     String parentOfParent = p.getParent().getParent().getName();
@@ -358,14 +360,14 @@ public class TestHLogSplit {
   public void testOldRecoveredEditsFileSidelined() throws IOException {
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
     byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
-    Path tdir = new Path(HBASEDIR, Bytes.toString(HConstants.META_TABLE_NAME));
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
     Path regiondir = new Path(tdir,
         HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
     fs.mkdirs(regiondir);
     long now = System.currentTimeMillis();
     HLog.Entry entry =
         new HLog.Entry(new HLogKey(encoded,
-            HConstants.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
+            TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
       new WALEdit());
     Path parent = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
     assertEquals(parent.getName(), HConstants.RECOVERED_EDITS_DIR);
@@ -767,9 +769,10 @@ public class TestHLogSplit {
 
     HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf);
     fs.rename(OLDLOGDIR, HLOGDIR);
-    Path firstSplitPath = new Path(HBASEDIR, Bytes.toString(TABLE_NAME) + ".first");
-    Path splitPath = new Path(HBASEDIR, Bytes.toString(TABLE_NAME));
-    fs.rename(splitPath, firstSplitPath);
+    Path firstSplitPath = new Path(HBASEDIR, TABLE_NAME+ ".first");
+    Path splitPath = new Path(HBASEDIR, TABLE_NAME.getNameAsString());
+    fs.rename(splitPath,
+            firstSplitPath);
 
     fs.initialize(fs.getUri(), conf);
     HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf);
@@ -1081,7 +1084,8 @@ public class TestHLogSplit {
 
     try {
       // put some entries in an HLog
-      byte [] tableName = Bytes.toBytes(this.getClass().getName());
+      TableName tableName =
+          TableName.valueOf(this.getClass().getName());
       HRegionInfo regioninfo = new HRegionInfo(tableName,
           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
       log = HLogFactory.createHLog(fs, HBASEDIR, logName, conf);
@@ -1089,7 +1093,7 @@ public class TestHLogSplit {
       final int total = 20;
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
-        kvs.add(new KeyValue(Bytes.toBytes(i), tableName, tableName));
+        kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
         HTableDescriptor htd = new HTableDescriptor(tableName);
         htd.addFamily(new HColumnDescriptor("column"));
         log.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd);
@@ -1151,7 +1155,7 @@ public class TestHLogSplit {
       if (stop.get()) {
         return;
       }
-      Path tableDir = new Path(HBASEDIR, new String(TABLE_NAME));
+      Path tableDir = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
       Path regionDir = new Path(tableDir, REGIONS.get(0));
       Path recoveredEdits = new Path(regionDir, HConstants.RECOVERED_EDITS_DIR);
       String region = "juliet";
@@ -1166,7 +1170,7 @@ public class TestHLogSplit {
         fs.mkdirs(new Path(tableDir, region));
         HLog.Writer writer = HLogFactory.createWriter(fs,
             julietLog, conf);
-        appendEntry(writer, "juliet".getBytes(), ("juliet").getBytes(),
+        appendEntry(writer, TableName.valueOf("juliet"), ("juliet").getBytes(),
             ("r").getBytes(), FAMILY, QUALIFIER, VALUE, 0);
         writer.close();
         LOG.info("Juliet file creator: created file " + julietLog);
@@ -1224,7 +1228,7 @@ public class TestHLogSplit {
     fs.initialize(fs.getUri(), conf);
 
     HLogSplitter.split(HBASEDIR, HLOGDIR, OLDLOGDIR, fs, conf);
-    Path tdir = HTableDescriptor.getTableDir(HBASEDIR, TABLE_NAME);
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
     assertFalse(fs.exists(tdir));
 
     assertEquals(0, countHLog(fs.listStatus(OLDLOGDIR)[0].getPath(), fs, conf));
@@ -1362,9 +1366,9 @@ public class TestHLogSplit {
     return ws;
   }
 
-  private Path[] getLogForRegion(Path rootdir, byte[] table, String region)
+  private Path[] getLogForRegion(Path rootdir, TableName table, String region)
   throws IOException {
-    Path tdir = HTableDescriptor.getTableDir(rootdir, table);
+    Path tdir = FSUtils.getTableDir(rootdir, table);
     @SuppressWarnings("deprecation")
     Path editsdir = HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
       Bytes.toString(region.getBytes())));
@@ -1475,7 +1479,7 @@ public class TestHLogSplit {
   }
 
 
-  public static long appendEntry(HLog.Writer writer, byte[] table, byte[] region,
+  public static long appendEntry(HLog.Writer writer, TableName table, byte[] region,
                           byte[] row, byte[] family, byte[] qualifier,
                           byte[] value, long seq)
           throws IOException {
@@ -1487,7 +1491,7 @@ public class TestHLogSplit {
   }
 
   private static HLog.Entry createTestEntry(
-      byte[] table, byte[] region,
+      TableName table, byte[] region,
       byte[] row, byte[] family, byte[] qualifier,
       byte[] value, long seq) {
     long time = System.nanoTime();

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java Thu Aug  8 06:08:23 2013
@@ -119,7 +119,7 @@ public class TestLogRollAbort {
 
     // When the META table can be opened, the region servers are running
     new HTable(TEST_UTIL.getConfiguration(),
-      HConstants.META_TABLE_NAME).close();
+      TableName.META_TABLE_NAME).close();
 
     // Create the test table and open it
     String tableName = this.getClass().getSimpleName();

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java Thu Aug  8 06:08:23 2013
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -190,7 +191,7 @@ public class TestLogRolling  {
 
   private void startAndWriteData() throws IOException, InterruptedException {
     // When the META table can be opened, the region servers are running
-    new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
+    new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
     this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
     this.log = server.getWAL();
 
@@ -330,7 +331,7 @@ public class TestLogRolling  {
 
     // Create the test table and open it
     String tableName = getName();
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
 
     admin.createTable(desc);
@@ -426,14 +427,14 @@ public class TestLogRolling  {
       fs.getDefaultReplication() > 1);
     LOG.info("Replication=" + fs.getDefaultReplication());
     // When the META table can be opened, the region servers are running
-    new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
+    new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
 
     this.server = cluster.getRegionServer(0);
     this.log = server.getWAL();
 
     // Create the test table and open it
     String tableName = getName();
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
 
     admin.createTable(desc);
@@ -583,7 +584,7 @@ public class TestLogRolling  {
   @Test
   public void testCompactionRecordDoesntBlockRolling() throws Exception {
     // When the META table can be opened, the region servers are running
-    new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
+    new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
 
     String tableName = getName();
     HTable table = createTestTable(tableName);
@@ -593,9 +594,11 @@ public class TestLogRolling  {
     server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
     this.log = server.getWAL();
     FSHLog fshLog = (FSHLog)log;
-    HRegion region = server.getOnlineRegions(table2.getTableName()).get(0);
+    HRegion region = server.getOnlineRegions(table2.getName()).get(0);
     Store s = region.getStore(HConstants.CATALOG_FAMILY);
 
+    //have to flush namespace to ensure it doesn't affect wall tests
+    admin.flush(TableName.NAMESPACE_TABLE_NAME.getName());
 
     // Put some stuff into table2, to make sure we have some files to compact.
     for (int i = 1; i <= 2; ++i) {
@@ -641,7 +644,7 @@ public class TestLogRolling  {
 
   private HTable createTestTable(String tableName) throws IOException {
     // 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);
     return new HTable(TEST_UTIL.getConfiguration(), tableName);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java Thu Aug  8 06:08:23 2013
@@ -122,7 +122,7 @@ public class TestLogRollingNoCluster {
           edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY));
 
           this.wal.append(HRegionInfo.FIRST_META_REGIONINFO,
-              HTableDescriptor.META_TABLEDESC.getName(),
+              HTableDescriptor.META_TABLEDESC.getTableName(),
               edit, now, HTableDescriptor.META_TABLEDESC);
         }
         String msg = getName() + " finished";

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java Thu Aug  8 06:08:23 2013
@@ -89,7 +89,7 @@ public class TestWALActionsListener {
     DummyWALActionsListener laterobserver = new DummyWALActionsListener();
     HLog hlog = HLogFactory.createHLog(fs, TEST_UTIL.getDataTestDir(), logName,
                                        conf, list, null);
-    HRegionInfo hri = new HRegionInfo(SOME_BYTES,
+    HRegionInfo hri = new HRegionInfo(TableName.valueOf(SOME_BYTES),
              SOME_BYTES, SOME_BYTES, false);
 
     for (int i = 0; i < 20; i++) {
@@ -100,7 +100,7 @@ public class TestWALActionsListener {
       HTableDescriptor htd = new HTableDescriptor();
       htd.addFamily(new HColumnDescriptor(b));
 
-      hlog.append(hri, b, edit, 0, htd);
+      hlog.append(hri, TableName.valueOf(b), edit, 0, htd);
       if (i == 10) {
         hlog.registerWALActionsListener(laterobserver);
       }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Thu Aug  8 06:08:23 2013
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -150,8 +151,8 @@ public class TestWALReplay {
    */
   @Test
   public void testReplayEditsAfterRegionMovedWithMultiCF() throws Exception {
-    final byte[] tableName = Bytes
-        .toBytes("testReplayEditsAfterRegionMovedWithMultiCF");
+    final TableName tableName =
+        TableName.valueOf("testReplayEditsAfterRegionMovedWithMultiCF");
     byte[] family1 = Bytes.toBytes("cf1");
     byte[] family2 = Bytes.toBytes("cf2");
     byte[] qualifier = Bytes.toBytes("q");
@@ -251,17 +252,17 @@ public class TestWALReplay {
   public void test2727() throws Exception {
     // Test being able to have > 1 set of edits in the recovered.edits directory.
     // Ensure edits are replayed properly.
-    final String tableNameStr = "test2727";
-    HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
-    Path basedir = new Path(hbaseRootDir, tableNameStr);
+    final TableName tableName =
+        TableName.valueOf("test2727");
+    HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
+    Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
     deleteDir(basedir);
 
-    HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
+    HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region2 = HRegion.createHRegion(hri,
         hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
-    final byte [] tableName = Bytes.toBytes(tableNameStr);
-    final byte [] rowName = tableName;
+    final byte [] rowName = tableName.getName();
 
     HLog wal1 = createWAL(this.conf);
     // Add 1k to each family.
@@ -312,11 +313,12 @@ public class TestWALReplay {
   public void testRegionMadeOfBulkLoadedFilesOnly()
   throws IOException, SecurityException, IllegalArgumentException,
       NoSuchFieldException, IllegalAccessException, InterruptedException {
-    final String tableNameStr = "testReplayEditsWrittenViaHRegion";
-    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
-    final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
+    final TableName tableName =
+        TableName.valueOf("testReplayEditsWrittenViaHRegion");
+    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
+    final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString());
     deleteDir(basedir);
-    final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
+    final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region2 = HRegion.createHRegion(hri,
         hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
@@ -326,7 +328,7 @@ public class TestWALReplay {
     HFile.Writer writer =
       HFile.getWriterFactoryNoCache(conf).withPath(fs, f).create();
     byte [] family = htd.getFamilies().iterator().next().getName();
-    byte [] row = Bytes.toBytes(tableNameStr);
+    byte [] row = tableName.getName();
     writer.append(new KeyValue(row, family, family, row));
     writer.close();
     List <Pair<byte[],String>>  hfs= new ArrayList<Pair<byte[],String>>(1);
@@ -339,7 +341,7 @@ public class TestWALReplay {
     // Now 'crash' the region by stealing its wal
     final Configuration newConf = HBaseConfiguration.create(this.conf);
     User user = HBaseTestingUtility.getDifferentUser(newConf,
-        tableNameStr);
+        tableName.getNameAsString());
     user.runAs(new PrivilegedExceptionAction() {
       public Object run() throws Exception {
         runWALSplit(newConf);
@@ -371,13 +373,14 @@ public class TestWALReplay {
   public void testReplayEditsWrittenViaHRegion()
   throws IOException, SecurityException, IllegalArgumentException,
       NoSuchFieldException, IllegalAccessException, InterruptedException {
-    final String tableNameStr = "testReplayEditsWrittenViaHRegion";
-    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
-    final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
+    final TableName tableName =
+        TableName.valueOf("testReplayEditsWrittenViaHRegion");
+    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
-    final byte[] rowName = Bytes.toBytes(tableNameStr);
+    final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
-    final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
+    final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HRegion.createHRegion(hri,
             hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region3);
@@ -433,7 +436,7 @@ public class TestWALReplay {
     HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal2).getOutputStream(), 1);
     final Configuration newConf = HBaseConfiguration.create(this.conf);
     User user = HBaseTestingUtility.getDifferentUser(newConf,
-      tableNameStr);
+      tableName.getNameAsString());
     user.runAs(new PrivilegedExceptionAction() {
       public Object run() throws Exception {
         runWALSplit(newConf);
@@ -488,13 +491,14 @@ public class TestWALReplay {
   public void testReplayEditsAfterPartialFlush()
   throws IOException, SecurityException, IllegalArgumentException,
       NoSuchFieldException, IllegalAccessException, InterruptedException {
-    final String tableNameStr = "testReplayEditsWrittenViaHRegion";
-    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
-    final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
+    final TableName tableName =
+        TableName.valueOf("testReplayEditsWrittenViaHRegion");
+    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
-    final byte[] rowName = Bytes.toBytes(tableNameStr);
+    final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
-    final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
+    final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HRegion.createHRegion(hri,
             hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region3);
@@ -578,11 +582,12 @@ public class TestWALReplay {
    */
   @Test
   public void testReplayEditsAfterAbortingFlush() throws IOException {
-    final String tableNameStr = "testReplayEditsAfterAbortingFlush";
-    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
-    final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
+    final TableName tableName =
+        TableName.valueOf("testReplayEditsAfterAbortingFlush");
+    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
+    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
-    final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
+    final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
     region3.close();
     region3.getLog().closeAndDelete();
@@ -605,7 +610,7 @@ public class TestWALReplay {
     List<HColumnDescriptor> families = new ArrayList<HColumnDescriptor>(
         htd.getFamilies());
     for (int i = 0; i < writtenRowCount; i++) {
-      Put put = new Put(Bytes.toBytes(tableNameStr + Integer.toString(i)));
+      Put put = new Put(Bytes.toBytes(tableName + Integer.toString(i)));
       put.add(families.get(i % families.size()).getName(), Bytes.toBytes("q"),
           Bytes.toBytes("val"));
       region.put(put);
@@ -629,7 +634,7 @@ public class TestWALReplay {
     // writing more data
     int moreRow = 10;
     for (int i = writtenRowCount; i < writtenRowCount + moreRow; i++) {
-      Put put = new Put(Bytes.toBytes(tableNameStr + Integer.toString(i)));
+      Put put = new Put(Bytes.toBytes(tableName + Integer.toString(i)));
       put.add(families.get(i % families.size()).getName(), Bytes.toBytes("q"),
           Bytes.toBytes("val"));
       region.put(put);
@@ -683,18 +688,18 @@ public class TestWALReplay {
    */
   @Test
   public void testReplayEditsWrittenIntoWAL() throws Exception {
-    final String tableNameStr = "testReplayEditsWrittenIntoWAL";
-    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
-    final Path basedir = new Path(hbaseRootDir, tableNameStr);
+    final TableName tableName =
+        TableName.valueOf("testReplayEditsWrittenIntoWAL");
+    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
+    final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
     deleteDir(basedir);
 
-    final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
+    final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region2 = HRegion.createHRegion(hri,
             hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
     final HLog wal = createWAL(this.conf);
-    final byte[] tableName = Bytes.toBytes(tableNameStr);
-    final byte[] rowName = tableName;
+    final byte[] rowName = tableName.getName();
     final byte[] regionName = hri.getEncodedNameAsBytes();
 
     // Add 1k to each family.
@@ -776,13 +781,15 @@ public class TestWALReplay {
   @Test
   // the following test is for HBASE-6065
   public void testSequentialEditLogSeqNum() throws IOException {
-    final String tableNameStr = "testSequentialEditLogSeqNum";
-    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
-    final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
+    final TableName tableName =
+        TableName.valueOf("testSequentialEditLogSeqNum");
+    final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
+    final Path basedir =
+        FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
-    final byte[] rowName = Bytes.toBytes(tableNameStr);
+    final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
-    final HTableDescriptor htd = createBasic1FamilyHTD(tableNameStr);
+    final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
 
     // Mock the HLog
     MockHLog wal = createMockWAL(this.conf);
@@ -814,8 +821,9 @@ public class TestWALReplay {
     FileStatus[] listStatus = this.fs.listStatus(wal.getDir());
     HLogSplitter.splitLogFile(hbaseRootDir, listStatus[0],
       this.fs, this.conf, null, null, null);
-    FileStatus[] listStatus1 = this.fs.listStatus(new Path(hbaseRootDir + "/"
-        + tableNameStr + "/" + hri.getEncodedName() + "/recovered.edits"));
+    FileStatus[] listStatus1 = this.fs.listStatus(
+        new Path(FSUtils.getTableDir(hbaseRootDir, tableName),
+            new Path(hri.getEncodedName(), "recovered.edits")));
     int editCount = 0;
     for (FileStatus fileStatus : listStatus1) {
       editCount = Integer.parseInt(fileStatus.getPath().getName());
@@ -842,7 +850,7 @@ public class TestWALReplay {
     }
   }
 
-  private HTableDescriptor createBasic1FamilyHTD(final String tableName) {
+  private HTableDescriptor createBasic1FamilyHTD(final TableName tableName) {
     HTableDescriptor htd = new HTableDescriptor(tableName);
     HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
     htd.addFamily(a);
@@ -878,7 +886,7 @@ public class TestWALReplay {
     }
   }
 
-  private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
+  private void addWALEdits (final TableName tableName, final HRegionInfo hri,
       final byte [] rowName, final byte [] family,
       final int count, EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
   throws IOException {
@@ -910,8 +918,8 @@ public class TestWALReplay {
    * column families named 'a','b', and 'c'.
    * @param tableName Name of table to use when we create HTableDescriptor.
    */
-   private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
-    return new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
+   private HRegionInfo createBasic3FamilyHRegionInfo(final TableName tableName) {
+    return new HRegionInfo(tableName, null, null, false);
    }
 
   /*
@@ -945,7 +953,7 @@ public class TestWALReplay {
     return wal;
   }
 
-  private HTableDescriptor createBasic3FamilyHTD(final String tableName) {
+  private HTableDescriptor createBasic3FamilyHTD(final TableName tableName) {
     HTableDescriptor htd = new HTableDescriptor(tableName);
     HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
     htd.addFamily(a);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java Thu Aug  8 06:08:23 2013
@@ -1,4 +1,4 @@
-/*
+/**
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -66,7 +66,7 @@ public class TestMasterReplication {
   private MiniZooKeeperCluster miniZK; 
 
   private static final long SLEEP_TIME = 500;
-  private static final int NB_RETRIES = 10;
+  private static final int NB_RETRIES = 100;
 
   private static final byte[] tableName = Bytes.toBytes("test");
   private static final byte[] famName = Bytes.toBytes("f");
@@ -121,7 +121,7 @@ public class TestMasterReplication {
     utility3.setZkCluster(miniZK);
     new ZooKeeperWatcher(conf3, "cluster3", null, true);
 
-    table = new HTableDescriptor(tableName);
+    table = new HTableDescriptor(TableName.valueOf(tableName));
     HColumnDescriptor fam = new HColumnDescriptor(famName);
     fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
     table.addFamily(fam);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java Thu Aug  8 06:08:23 2013
@@ -57,7 +57,7 @@ public class TestMultiSlaveReplication {
   private static HBaseTestingUtility utility2;
   private static HBaseTestingUtility utility3;
   private static final long SLEEP_TIME = 500;
-  private static final int NB_RETRIES = 10;
+  private static final int NB_RETRIES = 100;
 
   private static final byte[] tableName = Bytes.toBytes("test");
   private static final byte[] famName = Bytes.toBytes("f");
@@ -105,7 +105,7 @@ public class TestMultiSlaveReplication {
     utility3.setZkCluster(miniZK);
     new ZooKeeperWatcher(conf3, "cluster3", null, true);
 
-    table = new HTableDescriptor(tableName);
+    table = new HTableDescriptor(TableName.valueOf(tableName));
     HColumnDescriptor fam = new HColumnDescriptor(famName);
     fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
     table.addFamily(fam);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java Thu Aug  8 06:08:23 2013
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
@@ -119,7 +120,7 @@ public class TestReplicationBase {
     utility1.startMiniCluster(2);
     utility2.startMiniCluster(2);
 
-    HTableDescriptor table = new HTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(TableName.valueOf(tableName));
     HColumnDescriptor fam = new HColumnDescriptor(famName);
     fam.setMaxVersions(3);
     fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java Thu Aug  8 06:08:23 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.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -33,7 +34,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
-import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -86,7 +86,8 @@ public class TestReplicationSource {
       KeyValue kv = new KeyValue(b,b,b);
       WALEdit edit = new WALEdit();
       edit.add(kv);
-      HLogKey key = new HLogKey(b, b, 0, 0, HConstants.DEFAULT_CLUSTER_ID);
+      HLogKey key = new HLogKey(b, TableName.valueOf(b), 0, 0,
+          HConstants.DEFAULT_CLUSTER_ID);
       writer.append(new HLog.Entry(key, edit));
       writer.sync();
     }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationHLogReaderManager.java Thu Aug  8 06:08:23 2013
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
@@ -59,7 +60,7 @@ public class TestReplicationHLogReaderMa
   private static Path hbaseDir;
   private static FileSystem fs;
   private static MiniDFSCluster cluster;
-  private static final byte [] tableName = Bytes.toBytes("tablename");
+  private static final TableName tableName = TableName.valueOf("tablename");
   private static final byte [] family = Bytes.toBytes("column");
   private static final byte [] qualifier = Bytes.toBytes("qualifier");
   private static final HRegionInfo info = new HRegionInfo(tableName,

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java Thu Aug  8 06:08:23 2013
@@ -34,6 +34,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.ClusterId;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -93,7 +94,8 @@ public class TestReplicationSourceManage
 
   private static final byte[] f1 = Bytes.toBytes("f1");
 
-  private static final byte[] test = Bytes.toBytes("test");
+  private static final TableName test =
+      TableName.valueOf("test");
 
   private static final String slaveId = "1";
 
@@ -152,7 +154,7 @@ public class TestReplicationSourceManage
     col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
     htd.addFamily(col);
 
-    hri = new HRegionInfo(htd.getName(), r1, r2);
+    hri = new HRegionInfo(htd.getTableName(), r1, r2);
   }
 
   @AfterClass

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java Thu Aug  8 06:08:23 2013
@@ -44,6 +44,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -107,7 +108,7 @@ public class PerformanceEvaluation  {
 
   protected static final HTableDescriptor TABLE_DESCRIPTOR;
   static {
-    TABLE_DESCRIPTOR = new HTableDescriptor(TABLE_NAME);
+    TABLE_DESCRIPTOR = new HTableDescriptor(TableName.valueOf(TABLE_NAME));
     TABLE_DESCRIPTOR.addFamily(new HColumnDescriptor(FAMILY_NAME));
   }
 
@@ -448,7 +449,7 @@ public class PerformanceEvaluation  {
   private boolean checkTable() throws IOException {
     HTableDescriptor tableDescriptor = getTableDescriptor();
     RemoteAdmin admin = new RemoteAdmin(new Client(cluster), conf);
-    if (!admin.isTableAvailable(tableDescriptor.getName())) {
+    if (!admin.isTableAvailable(tableDescriptor.getTableName().getName())) {
       admin.createTable(tableDescriptor);
       return true;
     }
@@ -1069,7 +1070,7 @@ public class PerformanceEvaluation  {
 
     Test t = null;
     TestOptions options = new TestOptions(startRow, perClientRunRows,
-      totalRows, getTableDescriptor().getName(), rowsPerPut);
+      totalRows, getTableDescriptor().getTableName().getName(), rowsPerPut);
     try {
       Constructor<? extends Test> constructor = cmd.getDeclaredConstructor(
           Configuration.class, TestOptions.class, Status.class);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java Thu Aug  8 06:08:23 2013
@@ -66,7 +66,7 @@ public class TestGzipFilter {
     if (admin.tableExists(TABLE)) {
       return;
     }
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
     htd.addFamily(new HColumnDescriptor(CFA));
     admin.createTable(htd);
   }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java Thu Aug  8 06:08:23 2013
@@ -83,7 +83,7 @@ public class TestMultiRowResource {
     if (admin.tableExists(TABLE)) {
       return;
     }
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
     htd.addFamily(new HColumnDescriptor(CFA));
     htd.addFamily(new HColumnDescriptor(CFB));
     admin.createTable(htd);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java Thu Aug  8 06:08:23 2013
@@ -93,7 +93,7 @@ public class TestRowResource {
     if (admin.tableExists(TABLE)) {
       return;
     }
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
     htd.addFamily(new HColumnDescriptor(CFA));
     htd.addFamily(new HColumnDescriptor(CFB));
     admin.createTable(htd);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java Thu Aug  8 06:08:23 2013
@@ -166,7 +166,7 @@ public class TestScannerResource {
     if (admin.tableExists(TABLE)) {
       return;
     }
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
     htd.addFamily(new HColumnDescriptor(CFA));
     htd.addFamily(new HColumnDescriptor(CFB));
     admin.createTable(htd);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java Thu Aug  8 06:08:23 2013
@@ -130,7 +130,7 @@ public class TestScannersWithFilters {
       REST_TEST_UTIL.getServletPort()));
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     if (!admin.tableExists(TABLE)) {
-      HTableDescriptor htd = new HTableDescriptor(TABLE);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
       htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
       htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
       admin.createTable(htd);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java Thu Aug  8 06:08:23 2013
@@ -27,6 +27,7 @@ import javax.xml.bind.JAXBException;
 
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.rest.client.Client;
 import org.apache.hadoop.hbase.rest.client.Cluster;
 import org.apache.hadoop.hbase.rest.client.Response;
@@ -42,7 +43,7 @@ import org.junit.experimental.categories
 
 @Category(MediumTests.class)
 public class TestStatusResource {
-  private static final byte[] META_REGION_NAME = Bytes.toBytes(".META.,,1");
+  private static final byte[] META_REGION_NAME = Bytes.toBytes(TableName.META_TABLE_NAME+",,1");
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final HBaseRESTTestingUtility REST_TEST_UTIL = 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java Thu Aug  8 06:08:23 2013
@@ -82,7 +82,7 @@ public class TestTableResource {
     if (admin.tableExists(TABLE)) {
       return;
     }
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
     htd.addFamily(new HColumnDescriptor(COLUMN_FAMILY));
     admin.createTable(htd);
     HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java Thu Aug  8 06:08:23 2013
@@ -77,7 +77,7 @@ public class TestRemoteTable {
     REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     if (!admin.tableExists(TABLE)) {
-      HTableDescriptor htd = new HTableDescriptor(TABLE);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
       htd.addFamily(new HColumnDescriptor(COLUMN_1).setMaxVersions(3));
       htd.addFamily(new HColumnDescriptor(COLUMN_2).setMaxVersions(3));
       htd.addFamily(new HColumnDescriptor(COLUMN_3).setMaxVersions(3));

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java Thu Aug  8 06:08:23 2013
@@ -28,6 +28,7 @@ import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -42,8 +43,8 @@ public class TestStorageClusterStatusMod
     "<DeadNodes/>" + 
     "<LiveNodes><Node startCode=\"1245219839331\" requests=\"0\"" + 
       " name=\"test1\" maxHeapSizeMB=\"1024\" heapSizeMB=\"128\">" + 
-        "<Region stores=\"1\" storefiles=\"1\" storefileSizeMB=\"0\"" + 
-        " storefileIndexSizeMB=\"0\" name=\"LVJPT1QtLCww\"" + 
+        "<Region stores=\"1\" storefiles=\"1\" storefileSizeMB=\"0\"" +
+        " storefileIndexSizeMB=\"0\" name=\"aGJhc2U6cm9vdCwsMA==\"" +
         " memstoreSizeMB=\"0\" readRequestsCount=\"1\"" +
         " writeRequestsCount=\"2\" rootIndexSizeKB=\"1\"" +
         " totalStaticIndexSizeKB=\"1\" totalStaticBloomSizeKB=\"1\"" +
@@ -51,7 +52,7 @@ public class TestStorageClusterStatusMod
       "<Node startCode=\"1245239331198\" requests=\"0\" name=\"test2\"" + 
         " maxHeapSizeMB=\"1024\" heapSizeMB=\"512\">" + 
         "<Region stores=\"1\" storefiles=\"1\" storefileSizeMB=\"0\"" +
-        " storefileIndexSizeMB=\"0\" name=\"Lk1FVEEuLCwxMjQ2MDAwMDQzNzI0\"" +
+        " storefileIndexSizeMB=\"0\" name=\"aGJhc2U6bWV0YSwsMTI0NjAwMDA0MzcyNA==\"" +
         " memstoreSizeMB=\"0\" readRequestsCount=\"1\"" +
         " writeRequestsCount=\"2\" rootIndexSizeKB=\"1\"" +
         " totalStaticIndexSizeKB=\"1\" totalStaticBloomSizeKB=\"1\"" +
@@ -59,9 +60,10 @@ public class TestStorageClusterStatusMod
     "</LiveNodes></ClusterStatus>";
 
   private static final String AS_PB =
-  "CjsKBXRlc3QxEOO6i+eeJBgAIIABKIAIMiMKCS1ST09ULSwsMBABGAEgACgAMAA4AUACSAFQAVgB" +
-  "YAFoAQpHCgV0ZXN0MhD+krHwniQYACCABCiACDIvChUuTUVUQS4sLDEyNDYwMDAwNDM3MjQQARgB" +
-  "IAAoADAAOAFAAkgBUAFYAWABaAEYAiAAKQAAAAAAAPA/";
+      "Cj8KBXRlc3QxEOO6i+eeJBgAIIABKIAIMicKDWhiYXNlOnJvb3QsLDAQARgBIAAoADAAOAFAAkgB"+
+      "UAFYAWABaAEKSwoFdGVzdDIQ/pKx8J4kGAAggAQogAgyMwoZaGJhc2U6bWV0YSwsMTI0NjAwMDA0"+
+      "MzcyNBABGAEgACgAMAA4AUACSAFQAVgBYAFoARgCIAApAAAAAAAA8D8=";
+
   
   private JAXBContext context;
 
@@ -76,9 +78,9 @@ public class TestStorageClusterStatusMod
     model.setRequests(0);
     model.setAverageLoad(1.0);
     model.addLiveNode("test1", 1245219839331L, 128, 1024)
-      .addRegion(Bytes.toBytes("-ROOT-,,0"), 1, 1, 0, 0, 0, 1, 2, 1, 1, 1, 1, 1);
+      .addRegion(Bytes.toBytes(TableName.ROOT_TABLE_NAME+",,0"), 1, 1, 0, 0, 0, 1, 2, 1, 1, 1, 1, 1);
     model.addLiveNode("test2", 1245239331198L, 512, 1024)
-      .addRegion(Bytes.toBytes(".META.,,1246000043724"),1, 1, 0, 0, 0,
+      .addRegion(Bytes.toBytes(TableName.META_TABLE_NAME+",,1246000043724"),1, 1, 0, 0, 0,
           1, 2, 1, 1, 1, 1, 1);
     return model;
   }
@@ -119,7 +121,8 @@ public class TestStorageClusterStatusMod
     Iterator<StorageClusterStatusModel.Node.Region> regions = 
       node.getRegions().iterator();
     StorageClusterStatusModel.Node.Region region = regions.next();
-    assertTrue(Bytes.toString(region.getName()).equals("-ROOT-,,0"));
+    assertTrue(Bytes.toString(region.getName()).equals(
+        TableName.ROOT_TABLE_NAME+",,0"));
     assertEquals(region.getStores(), 1);
     assertEquals(region.getStorefiles(), 1);
     assertEquals(region.getStorefileSizeMB(), 0);
@@ -140,7 +143,8 @@ public class TestStorageClusterStatusMod
     assertEquals(node.getMaxHeapSizeMB(), 1024);
     regions = node.getRegions().iterator();
     region = regions.next();
-    assertEquals(Bytes.toString(region.getName()), ".META.,,1246000043724");
+    assertEquals(Bytes.toString(region.getName()),
+        TableName.META_TABLE_NAME+",,1246000043724");
     assertEquals(region.getStores(), 1);
     assertEquals(region.getStorefiles(), 1);
     assertEquals(region.getStorefileSizeMB(), 0);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/model/TestTableRegionModel.java Thu Aug  8 06:08:23 2013
@@ -88,7 +88,7 @@ public class TestTableRegionModel extend
   public void testGetName() {
     TableRegionModel model = buildTestModel();
     String modelName = model.getName();
-    HRegionInfo hri = new HRegionInfo(Bytes.toBytes(TABLE),
+    HRegionInfo hri = new HRegionInfo(TableName.valueOf(TABLE),
       START_KEY, END_KEY, false, ID);
     assertEquals(modelName, hri.getRegionNameAsString());
   }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java Thu Aug  8 06:08:23 2013
@@ -32,6 +32,7 @@ import java.io.IOException;
 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.LargeTests;
 import org.apache.hadoop.hbase.client.HTable;
@@ -61,7 +62,8 @@ public class TestAccessControlFilter {
   private static User LIMITED;
   private static User DENIED;
 
-  private static byte[] TABLE = Bytes.toBytes("testtable");
+  private static TableName TABLE =
+      TableName.valueOf("testtable");
   private static byte[] FAMILY = Bytes.toBytes("f1");
   private static byte[] PRIVATE_COL = Bytes.toBytes("private");
   private static byte[] PUBLIC_COL = Bytes.toBytes("public");

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java Thu Aug  8 06:08:23 2013
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -80,13 +81,10 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
-import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.util.TableName;
+import org.apache.hadoop.hbase.util.TestTableName;
 
 import org.junit.After;
 import org.junit.AfterClass;
@@ -97,7 +95,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
 
 /**
@@ -108,7 +105,7 @@ import com.google.protobuf.ServiceExcept
 @SuppressWarnings("rawtypes")
 public class TestAccessController {
   private static final Log LOG = LogFactory.getLog(TestAccessController.class);
-  @Rule public TableName TEST_TABLE = new TableName();
+  @Rule public TestTableName TEST_TABLE = new TestTableName();
   private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Configuration conf;
 
@@ -127,7 +124,8 @@ public class TestAccessController {
   // user with no permissions
   private static User USER_NONE;
 
-  private static byte[] TEST_TABLE2 = Bytes.toBytes("testtable2");
+  private static TableName TEST_TABLE2 =
+      TableName.valueOf("testtable2");
   private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
 
   private static MasterCoprocessorEnvironment CP_ENV;
@@ -183,7 +181,7 @@ public class TestAccessController {
     htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
     htd.setOwner(USER_OWNER);
     admin.createTable(htd);
-    TEST_UTIL.waitTableEnabled(TEST_TABLE.getTableName());
+    TEST_UTIL.waitTableEnabled(TEST_TABLE.getTableName().getName());
 
     HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE.getTableName()).get(0);
     RegionCoprocessorHost rcpHost = region.getCoprocessorHost();
@@ -193,7 +191,7 @@ public class TestAccessController {
     // initilize access control
     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName());
+      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
 
@@ -232,7 +230,7 @@ public class TestAccessController {
       TEST_UTIL.deleteTable(TEST_TABLE.getTableName());
     } catch (TableNotFoundException ex) {
       // Test deleted the table, no problem
-      LOG.info("Test deleted table " + Bytes.toString(TEST_TABLE.getTableName()));
+      LOG.info("Test deleted table " + TEST_TABLE.getTableName());
     }
     assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE.getTableName()).size());
   }
@@ -313,7 +311,7 @@ public class TestAccessController {
   public void testTableCreate() throws Exception {
     PrivilegedExceptionAction createTable = new PrivilegedExceptionAction() {
       public Object run() throws Exception {
-        HTableDescriptor htd = new HTableDescriptor("testnewtable");
+        HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testnewtable"));
         htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
         ACCESS_CONTROLLER.preCreateTable(ObserverContext.createAndPrepare(CP_ENV, null), htd, null);
         return null;
@@ -416,7 +414,7 @@ public class TestAccessController {
     PrivilegedExceptionAction disableAclTable = new PrivilegedExceptionAction() {
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null),
-            AccessControlLists.ACL_TABLE_NAME);
+            AccessControlLists.ACL_TABLE);
         return null;
       }
     };
@@ -872,7 +870,7 @@ public class TestAccessController {
     }
 
     private void bulkLoadHFile(
-        byte[] tableName,
+        TableName tableName,
         byte[] family,
         byte[] qualifier,
         byte[][][] hfileRanges,
@@ -893,7 +891,7 @@ public class TestAccessController {
       HTable table = new HTable(conf, tableName);
       try {
         HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
-        TEST_UTIL.waitTableEnabled(admin, tableName);
+        TEST_UTIL.waitTableEnabled(admin, tableName.getName());
         LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
         loader.doBulkLoad(loadPath, table);
       } finally {
@@ -947,7 +945,7 @@ public class TestAccessController {
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
         try {
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName());
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
           ProtobufUtil.grant(protocol, USER_RO.getShortName(), TEST_TABLE.getTableName(),
@@ -963,7 +961,7 @@ public class TestAccessController {
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
         try {
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName());
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
           ProtobufUtil.revoke(protocol, USER_RO.getShortName(), TEST_TABLE.getTableName(),
@@ -979,7 +977,7 @@ public class TestAccessController {
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
         try {
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName());
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
           ProtobufUtil.getUserPermissions(protocol, TEST_TABLE.getTableName());
@@ -1002,7 +1000,8 @@ public class TestAccessController {
 
   @Test
   public void testPostGrantRevoke() throws Exception {
-    final byte[] tableName = Bytes.toBytes("TempTable");
+    final TableName tableName =
+        TableName.valueOf("TempTable");
     final byte[] family1 = Bytes.toBytes("f1");
     final byte[] family2 = Bytes.toBytes("f2");
     final byte[] qualifier = Bytes.toBytes("q");
@@ -1158,7 +1157,7 @@ public class TestAccessController {
     // grant table read permission
     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, tblUser.getShortName(),
@@ -1182,7 +1181,7 @@ public class TestAccessController {
     // grant table write permission
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, tblUser.getShortName(),
@@ -1206,7 +1205,7 @@ public class TestAccessController {
     // revoke table permission
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, tblUser.getShortName(), tableName, null, null,
@@ -1230,7 +1229,7 @@ public class TestAccessController {
     // grant column family read permission
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, tblUser.getShortName(),
@@ -1256,7 +1255,7 @@ public class TestAccessController {
     // grant column family write permission
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, tblUser.getShortName(),
@@ -1283,7 +1282,7 @@ public class TestAccessController {
     // revoke column family permission
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.revoke(protocol, tblUser.getShortName(), tableName, family2, null);
@@ -1316,7 +1315,8 @@ public class TestAccessController {
 
   @Test
   public void testPostGrantRevokeAtQualifierLevel() throws Exception {
-    final byte[] tableName = Bytes.toBytes("testGrantRevokeAtQualifierLevel");
+    final TableName tableName =
+        TableName.valueOf("testGrantRevokeAtQualifierLevel");
     final byte[] family1 = Bytes.toBytes("f1");
     final byte[] family2 = Bytes.toBytes("f2");
     final byte[] qualifier = Bytes.toBytes("q");
@@ -1378,7 +1378,7 @@ public class TestAccessController {
 
     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.revoke(protocol, user.getShortName(), tableName, family1, null);
@@ -1394,7 +1394,7 @@ public class TestAccessController {
 
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, user.getShortName(),
@@ -1413,7 +1413,7 @@ public class TestAccessController {
     // TODO: comment this portion after HBASE-3583
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, user.getShortName(),
@@ -1431,7 +1431,7 @@ public class TestAccessController {
     // grant both read and write permission.
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, user.getShortName(),
@@ -1450,7 +1450,7 @@ public class TestAccessController {
     // revoke family level permission won't impact column level.
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.revoke(protocol, user.getShortName(),
@@ -1472,7 +1472,8 @@ public class TestAccessController {
 
   @Test
   public void testPermissionList() throws Exception {
-    final byte[] tableName = Bytes.toBytes("testPermissionList2");
+    final TableName tableName =
+        TableName.valueOf("testPermissionList");
     final byte[] family1 = Bytes.toBytes("f1");
     final byte[] family2 = Bytes.toBytes("f2");
     final byte[] qualifier = Bytes.toBytes("q");
@@ -1493,7 +1494,7 @@ public class TestAccessController {
 
     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       perms = ProtobufUtil.getUserPermissions(protocol, tableName);
@@ -1517,7 +1518,7 @@ public class TestAccessController {
     // grant read permission
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, user.getShortName(),
@@ -1540,7 +1541,7 @@ public class TestAccessController {
     // grant read+write
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, user.getShortName(),
@@ -1558,7 +1559,7 @@ public class TestAccessController {
 
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.revoke(protocol, user.getShortName(), tableName, family1, qualifier,
@@ -1580,7 +1581,7 @@ public class TestAccessController {
 
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       perms = ProtobufUtil.getUserPermissions(protocol, tableName);
@@ -1610,7 +1611,7 @@ public class TestAccessController {
       acl.close();
     }
     UserPermission adminPerm = new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()),
-      AccessControlLists.ACL_TABLE_NAME, null, null, Bytes.toBytes("ACRW"));
+      AccessControlLists.ACL_TABLE, null, null, Bytes.toBytes("ACRW"));
     assertTrue("Only user admin has permission on table _acl_ per setup",
       perms.size() == 1 && hasFoundUserPermission(adminPerm, perms));
   }
@@ -1647,7 +1648,7 @@ public class TestAccessController {
     }
   }
 
-  public void checkTablePerms(byte[] table, byte[] family, byte[] column,
+  public void checkTablePerms(TableName table, byte[] family, byte[] column,
       Permission.Action... actions) throws IOException {
     Permission[] perms = new Permission[actions.length];
     for (int i = 0; i < actions.length; i++) {
@@ -1657,7 +1658,7 @@ public class TestAccessController {
     checkTablePerms(table, perms);
   }
 
-  public void checkTablePerms(byte[] table, Permission... perms) throws IOException {
+  public void checkTablePerms(TableName table, Permission... perms) throws IOException {
     CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
     for (Permission p : perms) {
       request.addPermission(ProtobufUtil.toPermission(p));
@@ -1811,7 +1812,7 @@ public class TestAccessController {
     // check for wrong table region
     CheckPermissionsRequest checkRequest = CheckPermissionsRequest.newBuilder()
       .addPermission(AccessControlProtos.Permission.newBuilder()
-        .setTable(ByteString.copyFrom(TEST_TABLE.getTableName()))
+        .setTableName(ProtobufUtil.toProtoTableName(TEST_TABLE.getTableName()))
         .addAction(AccessControlProtos.Permission.Action.CREATE)
       ).build();
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
@@ -1926,7 +1927,7 @@ public class TestAccessController {
     // permissions for the new user.
     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName());
+      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       String currentUser = User.getCurrent().getShortName();
@@ -2005,7 +2006,7 @@ public class TestAccessController {
     // Grant TABLE ADMIN privs
     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName());
+      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, TABLE_ADMIN.getShortName(), TEST_TABLE.getTableName(),
@@ -2052,7 +2053,7 @@ public class TestAccessController {
     // Grant TABLE ADMIN privs
     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName());
+      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
       ProtobufUtil.grant(protocol, TABLE_ADMIN.getShortName(), TEST_TABLE.getTableName(),

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java Thu Aug  8 06:08:23 2013
@@ -38,6 +38,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.LargeTests;
@@ -80,8 +81,10 @@ public class TestTablePermissions {
     }
   };
 
-  private static byte[] TEST_TABLE = Bytes.toBytes("perms_test");
-  private static byte[] TEST_TABLE2 = Bytes.toBytes("perms_test2");
+  private static TableName TEST_TABLE =
+      TableName.valueOf("perms_test");
+  private static TableName TEST_TABLE2 =
+      TableName.valueOf("perms_test2");
   private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
   private static byte[] TEST_QUALIFIER = Bytes.toBytes("col1");
 
@@ -113,7 +116,7 @@ public class TestTablePermissions {
     Configuration conf = UTIL.getConfiguration();
     AccessControlLists.removeTablePermissions(conf, TEST_TABLE);
     AccessControlLists.removeTablePermissions(conf, TEST_TABLE2);
-    AccessControlLists.removeTablePermissions(conf, AccessControlLists.ACL_TABLE_NAME);
+    AccessControlLists.removeTablePermissions(conf, AccessControlLists.ACL_TABLE);
   }
 
   /**
@@ -187,8 +190,8 @@ public class TestTablePermissions {
     assertNotNull("Should have permissions for george", userPerms);
     assertEquals("Should have 1 permission for george", 1, userPerms.size());
     TablePermission permission = userPerms.get(0);
-    assertTrue("Permission should be for " + TEST_TABLE,
-        Bytes.equals(TEST_TABLE, permission.getTable()));
+    assertEquals("Permission should be for " + TEST_TABLE,
+        TEST_TABLE, permission.getTable());
     assertNull("Column family should be empty", permission.getFamily());
 
     // check actions
@@ -202,8 +205,8 @@ public class TestTablePermissions {
     assertNotNull("Should have permissions for hubert", userPerms);
     assertEquals("Should have 1 permission for hubert", 1, userPerms.size());
     permission = userPerms.get(0);
-    assertTrue("Permission should be for " + TEST_TABLE,
-        Bytes.equals(TEST_TABLE, permission.getTable()));
+    assertEquals("Permission should be for " + TEST_TABLE,
+        TEST_TABLE, permission.getTable());
     assertNull("Column family should be empty", permission.getFamily());
 
     // check actions
@@ -217,8 +220,8 @@ public class TestTablePermissions {
     assertNotNull("Should have permissions for humphrey", userPerms);
     assertEquals("Should have 1 permission for humphrey", 1, userPerms.size());
     permission = userPerms.get(0);
-    assertTrue("Permission should be for " + TEST_TABLE,
-        Bytes.equals(TEST_TABLE, permission.getTable()));
+    assertEquals("Permission should be for " + TEST_TABLE,
+        TEST_TABLE, permission.getTable());
     assertTrue("Permission should be for family " + TEST_FAMILY,
         Bytes.equals(TEST_FAMILY, permission.getFamily()));
     assertTrue("Permission should be for qualifier " + TEST_QUALIFIER,
@@ -237,7 +240,7 @@ public class TestTablePermissions {
             TablePermission.Action.READ, TablePermission.Action.WRITE));
 
     // check full load
-    Map<byte[],ListMultimap<String,TablePermission>> allPerms =
+    Map<TableName,ListMultimap<String,TablePermission>> allPerms =
         AccessControlLists.loadAll(conf);
     assertEquals("Full permission map should have entries for both test tables",
         2, allPerms.size());
@@ -246,7 +249,7 @@ public class TestTablePermissions {
     assertNotNull(userPerms);
     assertEquals(1, userPerms.size());
     permission = userPerms.get(0);
-    assertTrue(Bytes.equals(TEST_TABLE, permission.getTable()));
+    assertEquals(TEST_TABLE, permission.getTable());
     assertEquals(1, permission.getActions().length);
     assertEquals(TablePermission.Action.READ, permission.getActions()[0]);
 
@@ -254,7 +257,7 @@ public class TestTablePermissions {
     assertNotNull(userPerms);
     assertEquals(1, userPerms.size());
     permission = userPerms.get(0);
-    assertTrue(Bytes.equals(TEST_TABLE2, permission.getTable()));
+    assertEquals(TEST_TABLE2, permission.getTable());
     assertEquals(2, permission.getActions().length);
     actions = Arrays.asList(permission.getActions());
     assertTrue(actions.contains(TablePermission.Action.READ));
@@ -290,7 +293,7 @@ public class TestTablePermissions {
     table.put(new Put(Bytes.toBytes("row2"))
         .add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("v2")));
     HBaseAdmin admin = UTIL.getHBaseAdmin();
-    admin.split(TEST_TABLE);
+    admin.split(TEST_TABLE.getName());
 
     // wait for split
     Thread.sleep(10000);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java Thu Aug  8 06:08:23 2013
@@ -29,9 +29,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+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;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -62,7 +62,8 @@ public class TestZKPermissionsWatcher {
     }
   };
 
-  private static byte[] TEST_TABLE = Bytes.toBytes("perms_test");
+  private static TableName TEST_TABLE =
+      TableName.valueOf("perms_test");
 
   @BeforeClass
   public static void beforeClass() throws Exception {

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java Thu Aug  8 06:08:23 2013
@@ -38,6 +38,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MediumTests;
@@ -213,7 +214,8 @@ public class TestTokenAuthentication {
         public Configuration getConfiguration() { return conf; }
 
         @Override
-        public HTableInterface getTable(byte[] tableName) throws IOException { return null; }
+        public HTableInterface getTable(TableName tableName) throws IOException
+          { return null; }
       });
 
       started = true;