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 [22/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/snapshot/SnapshotTestingUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java Thu Aug  8 06:08:23 2013
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -84,14 +85,15 @@ public class SnapshotTestingUtils {
    * name and table match the passed in parameters.
    */
   public static List<SnapshotDescription> assertExistsMatchingSnapshot(
-      HBaseAdmin admin, String snapshotName, String tableName)
+      HBaseAdmin admin, String snapshotName, TableName tableName)
       throws IOException {
     // list the snapshot
     List<SnapshotDescription> snapshots = admin.listSnapshots();
 
     List<SnapshotDescription> returnedSnapshots = new ArrayList<SnapshotDescription>();
     for (SnapshotDescription sd : snapshots) {
-      if (snapshotName.equals(sd.getName()) && tableName.equals(sd.getTable())) {
+      if (snapshotName.equals(sd.getName()) &&
+          tableName.equals(TableName.valueOf(sd.getTable()))) {
         returnedSnapshots.add(sd);
       }
     }
@@ -105,7 +107,8 @@ public class SnapshotTestingUtils {
    */
   public static void assertOneSnapshotThatMatches(HBaseAdmin admin,
       SnapshotDescription snapshot) throws IOException {
-    assertOneSnapshotThatMatches(admin, snapshot.getName(), snapshot.getTable());
+    assertOneSnapshotThatMatches(admin, snapshot.getName(),
+        TableName.valueOf(snapshot.getTable()));
   }
 
   /**
@@ -113,14 +116,14 @@ public class SnapshotTestingUtils {
    * name and table match the passed in parameters.
    */
   public static List<SnapshotDescription> assertOneSnapshotThatMatches(
-      HBaseAdmin admin, String snapshotName, String tableName)
+      HBaseAdmin admin, String snapshotName, TableName tableName)
       throws IOException {
     // list the snapshot
     List<SnapshotDescription> snapshots = admin.listSnapshots();
 
     assertEquals("Should only have 1 snapshot", 1, snapshots.size());
     assertEquals(snapshotName, snapshots.get(0).getName());
-    assertEquals(tableName, snapshots.get(0).getTable());
+    assertEquals(tableName, TableName.valueOf(snapshots.get(0).getTable()));
 
     return snapshots;
   }
@@ -130,16 +133,16 @@ public class SnapshotTestingUtils {
    * name and table match the passed in parameters.
    */
   public static List<SnapshotDescription> assertOneSnapshotThatMatches(
-      HBaseAdmin admin, byte[] snapshot, byte[] tableName) throws IOException {
+      HBaseAdmin admin, byte[] snapshot, TableName tableName) throws IOException {
     return assertOneSnapshotThatMatches(admin, Bytes.toString(snapshot),
-        Bytes.toString(tableName));
+        tableName);
   }
 
   /**
    * Multi-family version of the confirmSnapshotValid function
    */
   public static void confirmSnapshotValid(
-      SnapshotDescription snapshotDescriptor, byte[] tableName,
+      SnapshotDescription snapshotDescriptor, TableName tableName,
       List<byte[]> nonEmptyTestFamilies, List<byte[]> emptyTestFamilies,
       Path rootDir, HBaseAdmin admin, FileSystem fs, boolean requireLogs,
       Path logsDir, Set<String> snapshotServers) throws IOException {
@@ -163,7 +166,7 @@ public class SnapshotTestingUtils {
    * be in the snapshot.
    */
   public static void confirmSnapshotValid(
-      SnapshotDescription snapshotDescriptor, byte[] tableName,
+      SnapshotDescription snapshotDescriptor, TableName tableName,
       byte[] testFamily, Path rootDir, HBaseAdmin admin, FileSystem fs,
       boolean requireLogs, Path logsDir, Set<String> snapshotServers)
       throws IOException {
@@ -176,7 +179,7 @@ public class SnapshotTestingUtils {
    * be in the snapshot.
    */
   public static void confirmSnapshotValid(
-      SnapshotDescription snapshotDescriptor, byte[] tableName,
+      SnapshotDescription snapshotDescriptor, TableName tableName,
       byte[] testFamily, Path rootDir, HBaseAdmin admin, FileSystem fs,
       boolean requireLogs, Path logsDir, boolean familyEmpty,
       Set<String> snapshotServers) throws IOException {
@@ -194,7 +197,7 @@ public class SnapshotTestingUtils {
     }
     // check the table info
     HTableDescriptor desc = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir,
-        Bytes.toString(tableName));
+        tableName);
     HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptorFromFs(fs,
         snapshotDir);
     assertEquals(desc, snapshotDesc);
@@ -356,10 +359,10 @@ public class SnapshotTestingUtils {
    * in the case of an offline snapshot.
    */
   public static void createOfflineSnapshotAndValidate(HBaseAdmin admin,
-      String tableNameString, String familyName, String snapshotNameString,
+      TableName tableName, String familyName, String snapshotNameString,
       Path rootDir, FileSystem fs, boolean familyEmpty) throws Exception {
 
-    createSnapshotAndValidate(admin, tableNameString, familyName,
+    createSnapshotAndValidate(admin, tableName, familyName,
         snapshotNameString, rootDir, fs, familyEmpty, false);
   }
 
@@ -369,29 +372,28 @@ public class SnapshotTestingUtils {
    * in the case of an offline snapshot.
    */
   public static void createSnapshotAndValidate(HBaseAdmin admin,
-      String tableNameString, String familyName, String snapshotNameString,
+      TableName tableName, String familyName, String snapshotNameString,
       Path rootDir, FileSystem fs, boolean familyEmpty, boolean onlineSnapshot)
       throws Exception {
-    byte[] tableName = Bytes.toBytes(tableNameString);
 
     if (!onlineSnapshot) {
       try {
-        admin.disableTable(tableNameString);
+        admin.disableTable(tableName);
       } catch (TableNotEnabledException tne) {
-        LOG.info("In attempting to disable " + tableNameString
+        LOG.info("In attempting to disable " + tableName
             + " it turns out that this table is already disabled.");
       }
     }
 
-    admin.snapshot(snapshotNameString, tableNameString);
+    admin.snapshot(snapshotNameString, tableName);
 
     List<SnapshotDescription> snapshots = SnapshotTestingUtils
         .assertExistsMatchingSnapshot(admin, snapshotNameString,
-            tableNameString);
+            tableName);
 
     if (snapshots == null || snapshots.size() != 1) {
       Assert.fail("Incorrect number of snapshots for table "
-          + String.valueOf(tableNameString));
+          + tableName);
     }
 
     SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), tableName,
@@ -399,9 +401,9 @@ public class SnapshotTestingUtils {
             HConstants.HREGION_LOGDIR_NAME), familyEmpty, null);
   }
   public static void createSnapshotAndValidate(HBaseAdmin admin,
-      String tableNameString, String familyName, String snapshotNameString,
+      TableName tableName, String familyName, String snapshotNameString,
       Path rootDir, FileSystem fs) throws Exception {
-    createSnapshotAndValidate(admin, tableNameString, familyName,
+    createSnapshotAndValidate(admin, tableName, familyName,
         snapshotNameString, rootDir, fs, false, false);
   }
 
@@ -412,30 +414,30 @@ public class SnapshotTestingUtils {
    *
    */
   public static void createSnapshotAndValidate(HBaseAdmin admin,
-      String tableNameString, String familyName, String snapshotNameString,
+      TableName tableName, String familyName, String snapshotNameString,
       Path rootDir, FileSystem fs, boolean online) throws Exception {
-    createSnapshotAndValidate(admin, tableNameString, familyName,
+    createSnapshotAndValidate(admin, tableName, familyName,
         snapshotNameString, rootDir, fs, false, online);
   }
 
   public static void createSnapshotAndValidate(HBaseAdmin admin,
-      String tableNameString, List<byte[]> nonEmptyFamilyNames, List<byte[]> emptyFamilyNames,
+      TableName tableName, List<byte[]> nonEmptyFamilyNames, List<byte[]> emptyFamilyNames,
       String snapshotNameString, Path rootDir, FileSystem fs) throws Exception {
 
-    byte[] tableName = Bytes.toBytes(tableNameString);
     try {
-      admin.disableTable(tableNameString);
+      admin.disableTable(tableName);
     } catch (TableNotEnabledException tne) {
-      LOG.info("In attempting to disable " + tableNameString + " it turns out that the this table is already disabled.");
+      LOG.info("In attempting to disable " + tableName + " it turns out that the this table is " +
+          "already disabled.");
     }
-    admin.snapshot(snapshotNameString, tableNameString);
+    admin.snapshot(snapshotNameString, tableName);
 
     List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertExistsMatchingSnapshot(admin,
-      snapshotNameString, tableNameString);
+      snapshotNameString, tableName);
 
     // Create test-timestamp-clone
     if (snapshots == null || snapshots.size() != 1) {
-      Assert.fail("Incorrect number of snapshots for table " + String.valueOf(tableNameString));
+      Assert.fail("Incorrect number of snapshots for table " + tableName);
     }
 
     SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), tableName, nonEmptyFamilyNames, emptyFamilyNames,
@@ -445,7 +447,8 @@ public class SnapshotTestingUtils {
   // ==========================================================================
   //  Table Helpers
   // ==========================================================================
-  public static void waitForTableToBeOnline(final HBaseTestingUtility util, final byte[] tableName)
+  public static void waitForTableToBeOnline(final HBaseTestingUtility util,
+                                            final TableName tableName)
       throws IOException, InterruptedException {
     HRegionServer rs = util.getRSForFirstRegionInTable(tableName);
     List<HRegion> onlineRegions = rs.getOnlineRegions(tableName);
@@ -455,7 +458,7 @@ public class SnapshotTestingUtils {
     util.getHBaseAdmin().isTableAvailable(tableName);
   }
 
-  public static void createTable(final HBaseTestingUtility util, final byte[] tableName,
+  public static void createTable(final HBaseTestingUtility util, final TableName tableName,
       final byte[]... families) throws IOException, InterruptedException {
     HTableDescriptor htd = new HTableDescriptor(tableName);
     for (byte[] family: families) {
@@ -472,7 +475,7 @@ public class SnapshotTestingUtils {
     assertEquals(15, util.getHBaseAdmin().getTableRegions(tableName).size());
   }
 
-  public static void loadData(final HBaseTestingUtility util, final byte[] tableName, int rows,
+  public static void loadData(final HBaseTestingUtility util, final TableName tableName, int rows,
       byte[]... families) throws IOException, InterruptedException {
     loadData(util, new HTable(util.getConfiguration(), tableName), rows, families);
   }
@@ -499,7 +502,7 @@ public class SnapshotTestingUtils {
     }
     table.flushCommits();
 
-    waitForTableToBeOnline(util, table.getTableName());
+    waitForTableToBeOnline(util, table.getName());
   }
 
   private static void putData(final HTable table, final byte[][] families,
@@ -530,7 +533,7 @@ public class SnapshotTestingUtils {
     mfs.getFileSystem().delete(archiveDir, true);
   }
 
-  public static void verifyRowCount(final HBaseTestingUtility util, final byte[] tableName,
+  public static void verifyRowCount(final HBaseTestingUtility util, final TableName tableName,
       long expectedRows) throws IOException {
     HTable table = new HTable(util.getConfiguration(), tableName);
     try {

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java Thu Aug  8 06:08:23 2013
@@ -37,15 +37,17 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -71,7 +73,7 @@ public class TestExportSnapshot {
 
   private byte[] emptySnapshotName;
   private byte[] snapshotName;
-  private byte[] tableName;
+  private TableName tableName;
   private HBaseAdmin admin;
 
   @BeforeClass
@@ -97,7 +99,7 @@ public class TestExportSnapshot {
     this.admin = TEST_UTIL.getHBaseAdmin();
 
     long tid = System.currentTimeMillis();
-    tableName = Bytes.toBytes("testtb-" + tid);
+    tableName = TableName.valueOf("testtb-" + tid);
     snapshotName = Bytes.toBytes("snaptb0-" + tid);
     emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + tid);
 
@@ -183,13 +185,15 @@ public class TestExportSnapshot {
   public void testSnapshotWithRefsExportFileSystemState() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
 
-    final byte[] tableWithRefsName = Bytes.toBytes("tableWithRefs");
+    final TableName tableWithRefsName =
+        TableName.valueOf("tableWithRefs");
     final String snapshotName = "tableWithRefs";
     final String TEST_FAMILY = Bytes.toString(FAMILY);
     final String TEST_HFILE = "abc";
 
     final SnapshotDescription sd = SnapshotDescription.newBuilder()
-        .setName(snapshotName).setTable(Bytes.toString(tableWithRefsName)).build();
+        .setName(snapshotName)
+        .setTable(tableWithRefsName.getNameAsString()).build();
 
     FileSystem fs = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
     Path rootDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
@@ -198,7 +202,7 @@ public class TestExportSnapshot {
     // First region, simple with one plain hfile.
     HRegionInfo hri = new HRegionInfo(tableWithRefsName);
     HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf,
-      fs, new Path(archiveDir, hri.getTableNameAsString()), hri);
+      fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
     Path storeFile = new Path(rootDir, TEST_HFILE);
     FSDataOutputStream out = fs.create(storeFile);
     out.write(Bytes.toBytes("Test Data"));
@@ -209,14 +213,14 @@ public class TestExportSnapshot {
     // This region contains a reference to the hfile in the first region.
     hri = new HRegionInfo(tableWithRefsName);
     HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf,
-      fs, new Path(archiveDir, hri.getTableNameAsString()), hri);
+      fs, new Path(archiveDir, hri.getTableName().getNameAsString()), hri);
     storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
     out = fs.create(storeFile);
     out.write(Bytes.toBytes("Test Data"));
     out.close();
     r1fs.commitStoreFile(TEST_FAMILY, storeFile);
 
-    Path tableDir = HTableDescriptor.getTableDir(archiveDir, tableWithRefsName);
+    Path tableDir = FSUtils.getTableDir(archiveDir, tableWithRefsName);
     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
     FileUtil.copy(fs, tableDir, fs, snapshotDir, false, conf);
     SnapshotDescriptionUtils.writeSnapshotInfo(sd, snapshotDir, fs);
@@ -227,7 +231,7 @@ public class TestExportSnapshot {
   /**
    * Test ExportSnapshot
    */
-  private void testExportFileSystemState(final byte[] tableName, final byte[] snapshotName,
+  private void testExportFileSystemState(final TableName tableName, final byte[] snapshotName,
       int filesExpected) throws Exception {
     Path copyDir = TEST_UTIL.getDataTestDir("export-" + System.currentTimeMillis());
     URI hdfsUri = FileSystem.get(TEST_UTIL.getConfiguration()).getUri();
@@ -276,7 +280,7 @@ public class TestExportSnapshot {
    * Verify if the files exists
    */
   private void verifyArchive(final FileSystem fs, final Path rootDir,
-      final byte[] tableName, final String snapshotName) throws IOException {
+      final TableName tableName, final String snapshotName) throws IOException {
     final Path exportedSnapshot = new Path(rootDir,
       new Path(HConstants.SNAPSHOT_DIR_NAME, snapshotName));
     final Path exportedArchive = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
@@ -286,13 +290,14 @@ public class TestExportSnapshot {
         public void storeFile (final String region, final String family, final String hfile)
             throws IOException {
           verifyNonEmptyFile(new Path(exportedArchive,
-            new Path(Bytes.toString(tableName), new Path(region, new Path(family, hfile)))));
+            new Path(FSUtils.getTableDir(new Path("./"), tableName),
+                new Path(region, new Path(family, hfile)))));
         }
 
         public void recoveredEdits (final String region, final String logfile)
             throws IOException {
           verifyNonEmptyFile(new Path(exportedSnapshot,
-            new Path(Bytes.toString(tableName), new Path(region, logfile))));
+            new Path(tableName.getNameAsString(), new Path(region, logfile))));
         }
 
         public void logFile (final String server, final String logfile)

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java Thu Aug  8 06:08:23 2013
@@ -35,14 +35,14 @@ import org.apache.commons.logging.impl.L
 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;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -52,13 +52,11 @@ import org.apache.hadoop.hbase.master.HM
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.apache.hadoop.hbase.util.MD5Hash;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -83,7 +81,8 @@ public class TestFlushSnapshotFromClient
   private static final String STRING_TABLE_NAME = "test";
   private static final byte[] TEST_FAM = Bytes.toBytes("fam");
   private static final byte[] TEST_QUAL = Bytes.toBytes("q");
-  private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
+  private static final TableName TABLE_NAME =
+      TableName.valueOf(STRING_TABLE_NAME);
   private final int DEFAULT_NUM_ROWS = 100;
 
   /**
@@ -163,7 +162,7 @@ public class TestFlushSnapshotFromClient
 
     LOG.debug("FS state before snapshot:");
     FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+        FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     // take a snapshot of the enabled table
     String snapshotString = "offlineTableSnapshot";
@@ -223,7 +222,9 @@ public class TestFlushSnapshotFromClient
   public void testAsyncFlushSnapshot() throws Exception {
     HBaseAdmin admin = UTIL.getHBaseAdmin();
     SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("asyncSnapshot")
-        .setTable(STRING_TABLE_NAME).setType(SnapshotDescription.Type.FLUSH).build();
+        .setTable(TABLE_NAME.getNameAsString())
+        .setType(SnapshotDescription.Type.FLUSH)
+        .build();
 
     // take the snapshot async
     admin.takeSnapshotAsync(snapshot);
@@ -264,7 +265,7 @@ public class TestFlushSnapshotFromClient
     // Clone the table
     String cloneBeforeMergeName = "cloneBeforeMerge";
     admin.cloneSnapshot(snapshotBeforeMergeName, cloneBeforeMergeName);
-    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneBeforeMergeName));
+    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneBeforeMergeName));
 
     // Merge two regions
     List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
@@ -288,11 +289,11 @@ public class TestFlushSnapshotFromClient
     // Clone the table
     String cloneAfterMergeName = "cloneAfterMerge";
     admin.cloneSnapshot(snapshotBeforeMergeName, cloneAfterMergeName);
-    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneAfterMergeName));
+    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneAfterMergeName));
 
     SnapshotTestingUtils.verifyRowCount(UTIL, TABLE_NAME, numRows);
-    SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneBeforeMergeName), numRows);
-    SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneAfterMergeName), numRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneBeforeMergeName), numRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneAfterMergeName), numRows);
 
     // test that we can delete the snapshot
     UTIL.deleteTable(cloneAfterMergeName);
@@ -337,10 +338,10 @@ public class TestFlushSnapshotFromClient
     // Clone the table
     String cloneName = "cloneMerge";
     admin.cloneSnapshot(snapshotName, cloneName);
-    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneName));
+    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TableName.valueOf(cloneName));
 
     SnapshotTestingUtils.verifyRowCount(UTIL, TABLE_NAME, numRows);
-    SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneName), numRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, TableName.valueOf(cloneName), numRows);
 
     // test that we can delete the snapshot
     UTIL.deleteTable(cloneName);
@@ -369,7 +370,7 @@ public class TestFlushSnapshotFromClient
 
     // make sure we only have 1 matching snapshot
     List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin,
-      snapshotName, STRING_TABLE_NAME);
+      snapshotName, TABLE_NAME);
 
     // check the directory structure
     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
@@ -382,9 +383,9 @@ public class TestFlushSnapshotFromClient
 
     // check the table info
     HTableDescriptor desc = FSTableDescriptors.getTableDescriptorFromFs(fs,
-        rootDir,STRING_TABLE_NAME);
+        rootDir, TABLE_NAME);
     HTableDescriptor snapshotDesc = FSTableDescriptors.getTableDescriptorFromFs(fs,
-      SnapshotDescriptionUtils.getSnapshotsDir(rootDir), snapshotName);
+        new Path(SnapshotDescriptionUtils.getSnapshotsDir(rootDir), snapshotName));
     assertEquals(desc, snapshotDesc);
 
     // check the region snapshot for all the regions
@@ -422,7 +423,8 @@ public class TestFlushSnapshotFromClient
   @Test(timeout=300000)
   public void testConcurrentSnapshottingAttempts() throws IOException, InterruptedException {
     final String STRING_TABLE2_NAME = STRING_TABLE_NAME + "2";
-    final byte[] TABLE2_NAME = Bytes.toBytes(STRING_TABLE2_NAME);
+    final TableName TABLE2_NAME =
+        TableName.valueOf(STRING_TABLE2_NAME);
 
     int ssNum = 20;
     HBaseAdmin admin = UTIL.getHBaseAdmin();
@@ -462,7 +464,7 @@ public class TestFlushSnapshotFromClient
     SnapshotDescription[] descs = new SnapshotDescription[ssNum];
     for (int i = 0; i < ssNum; i++) {
       SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
-      builder.setTable((i % 2) == 0 ? STRING_TABLE_NAME : STRING_TABLE2_NAME);
+      builder.setTable(((i % 2) == 0 ? TABLE_NAME : TABLE2_NAME).getNameAsString());
       builder.setName("ss"+i);
       builder.setType(SnapshotDescription.Type.FLUSH);
       descs[i] = builder.build();
@@ -508,9 +510,9 @@ public class TestFlushSnapshotFromClient
     int t1SnapshotsCount = 0;
     int t2SnapshotsCount = 0;
     for (SnapshotDescription ss : taken) {
-      if (ss.getTable().equals(STRING_TABLE_NAME)) {
+      if (TableName.valueOf(ss.getTable()).equals(TABLE_NAME)) {
         t1SnapshotsCount++;
-      } else if (ss.getTable().equals(STRING_TABLE2_NAME)) {
+      } else if (TableName.valueOf(ss.getTable()).equals(TABLE2_NAME)) {
         t2SnapshotsCount++;
       }
     }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestReferenceRegionHFilesTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestReferenceRegionHFilesTask.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestReferenceRegionHFilesTask.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestReferenceRegionHFilesTask.java Thu Aug  8 06:08:23 2013
@@ -29,9 +29,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java Thu Aug  8 06:08:23 2013
@@ -31,6 +31,7 @@ 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.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
@@ -71,7 +72,7 @@ public class TestRestoreFlushSnapshotFro
   private byte[] snapshotName2;
   private int snapshot0Rows;
   private int snapshot1Rows;
-  private byte[] tableName;
+  private TableName tableName;
   private HBaseAdmin admin;
 
   @BeforeClass
@@ -106,7 +107,7 @@ public class TestRestoreFlushSnapshotFro
     this.admin = UTIL.getHBaseAdmin();
 
     long tid = System.currentTimeMillis();
-    tableName = Bytes.toBytes("testtb-" + tid);
+    tableName = TableName.valueOf("testtb-" + tid);
     snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
     snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
     snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
@@ -121,7 +122,7 @@ public class TestRestoreFlushSnapshotFro
       logFSTree();
 
       // take a snapshot
-      admin.snapshot(Bytes.toString(snapshotName0), Bytes.toString(tableName),
+      admin.snapshot(Bytes.toString(snapshotName0), tableName,
           SnapshotDescription.Type.FLUSH);
 
       LOG.info("=== after snapshot with 500 rows");
@@ -134,7 +135,7 @@ public class TestRestoreFlushSnapshotFro
       logFSTree();
 
       // take a snapshot of the updated table
-      admin.snapshot(Bytes.toString(snapshotName1), Bytes.toString(tableName),
+      admin.snapshot(Bytes.toString(snapshotName1), tableName,
           SnapshotDescription.Type.FLUSH);
       LOG.info("=== after snapshot with 1000 rows");
       logFSTree();
@@ -183,12 +184,12 @@ public class TestRestoreFlushSnapshotFro
 
   @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);
   }
 
-  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);
@@ -199,10 +200,10 @@ public class TestRestoreFlushSnapshotFro
 
   @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(UTIL, clonedTableName, snapshot0Rows);
-    admin.snapshot(Bytes.toString(snapshotName2), Bytes.toString(clonedTableName), SnapshotDescription.Type.FLUSH);
+    admin.snapshot(Bytes.toString(snapshotName2), clonedTableName, SnapshotDescription.Type.FLUSH);
     UTIL.deleteTable(clonedTableName);
 
     admin.cloneSnapshot(snapshotName2, clonedTableName);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java Thu Aug  8 06:08:23 2013
@@ -38,10 +38,12 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@@ -92,23 +94,23 @@ public class TestRestoreSnapshotHelper {
 
     // Test clone a snapshot
     HTableDescriptor htdClone = createTableDescriptor("testtb-clone");
-    testRestore(snapshotDir, htd.getNameAsString(), htdClone);
+    testRestore(snapshotDir, htd.getTableName().getNameAsString(), htdClone);
     verifyRestore(rootDir, htd, htdClone);
 
     // Test clone a clone ("link to link")
-    Path cloneDir = HTableDescriptor.getTableDir(rootDir, htdClone.getName());
+    Path cloneDir = FSUtils.getTableDir(rootDir, htdClone.getTableName());
     HTableDescriptor htdClone2 = createTableDescriptor("testtb-clone2");
-    testRestore(cloneDir, htdClone.getNameAsString(), htdClone2);
+    testRestore(cloneDir, htdClone.getTableName().getNameAsString(), htdClone2);
     verifyRestore(rootDir, htd, htdClone2);
   }
 
   private void verifyRestore(final Path rootDir, final HTableDescriptor sourceHtd,
       final HTableDescriptor htdClone) throws IOException {
-    String[] files = getHFiles(HTableDescriptor.getTableDir(rootDir, htdClone.getName()));
+    String[] files = getHFiles(FSUtils.getTableDir(rootDir, htdClone.getTableName()));
     assertEquals(2, files.length);
     assertTrue(files[0] + " should be a HFileLink", HFileLink.isHFileLink(files[0]));
     assertTrue(files[1] + " should be a Referene", StoreFileInfo.isReference(files[1]));
-    assertEquals(sourceHtd.getNameAsString(), HFileLink.getReferencedTableName(files[0]));
+    assertEquals(sourceHtd.getTableName(), HFileLink.getReferencedTableName(files[0]));
     assertEquals(TEST_HFILE, HFileLink.getReferencedHFileName(files[0]));
     Path refPath = getReferredToFile(files[1]);
     assertTrue(refPath.getName() + " should be a HFileLink", HFileLink.isHFileLink(refPath.getName()));
@@ -123,14 +125,14 @@ public class TestRestoreSnapshotHelper {
    */
   public void testRestore(final Path snapshotDir, final String sourceTableName,
       final HTableDescriptor htdClone) throws IOException {
-    LOG.debug("pre-restore table=" + htdClone.getNameAsString() + " snapshot=" + snapshotDir);
+    LOG.debug("pre-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir);
     FSUtils.logFileSystemState(fs, rootDir, LOG);
 
     new FSTableDescriptors(conf).createTableDescriptor(htdClone);
     RestoreSnapshotHelper helper = getRestoreHelper(rootDir, snapshotDir, sourceTableName, htdClone);
     helper.restoreHdfsRegions();
 
-    LOG.debug("post-restore table=" + htdClone.getNameAsString() + " snapshot=" + snapshotDir);
+    LOG.debug("post-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir);
     FSUtils.logFileSystemState(fs, rootDir, LOG);
   }
 
@@ -145,37 +147,39 @@ public class TestRestoreSnapshotHelper {
     MonitoredTask status = Mockito.mock(MonitoredTask.class);
 
     SnapshotDescription sd = SnapshotDescription.newBuilder()
-      .setName("snapshot").setTable(sourceTableName).build();
+      .setName("snapshot")
+      .setTable(sourceTableName)
+      .build();
 
     return new RestoreSnapshotHelper(conf, fs, sd, snapshotDir,
-      htdClone, HTableDescriptor.getTableDir(rootDir, htdClone.getName()), monitor, status);
+      htdClone, rootDir, monitor, status);
   }
 
   private void createSnapshot(final Path rootDir, final Path snapshotDir, final HTableDescriptor htd)
       throws IOException {
     // First region, simple with one plain hfile.
-    HRegionInfo hri = new HRegionInfo(htd.getName());
+    HRegionInfo hri = new HRegionInfo(htd.getTableName());
     HRegionFileSystem r0fs = HRegionFileSystem.createRegionOnFileSystem(conf,
-      fs, new Path(archiveDir, hri.getTableNameAsString()), hri);
+      fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
     Path storeFile = new Path(rootDir, TEST_HFILE);
     fs.createNewFile(storeFile);
     r0fs.commitStoreFile(TEST_FAMILY, storeFile);
 
     // Second region, used to test the split case.
     // This region contains a reference to the hfile in the first region.
-    hri = new HRegionInfo(htd.getName());
+    hri = new HRegionInfo(htd.getTableName());
     HRegionFileSystem r1fs = HRegionFileSystem.createRegionOnFileSystem(conf,
-      fs, new Path(archiveDir, hri.getTableNameAsString()), hri);
+      fs, FSUtils.getTableDir(archiveDir, hri.getTableName()), hri);
     storeFile = new Path(rootDir, TEST_HFILE + '.' + r0fs.getRegionInfo().getEncodedName());
     fs.createNewFile(storeFile);
     r1fs.commitStoreFile(TEST_FAMILY, storeFile);
 
-    Path tableDir = HTableDescriptor.getTableDir(archiveDir, htd.getName());
+    Path tableDir = FSUtils.getTableDir(archiveDir, htd.getTableName());
     FileUtil.copy(fs, tableDir, fs, snapshotDir, false, conf);
   }
 
   private HTableDescriptor createTableDescriptor(final String tableName) {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
     htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
     return htd;
   }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java Thu Aug  8 06:08:23 2013
@@ -32,6 +32,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.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
@@ -41,7 +42,6 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
-import org.apache.hadoop.hbase.snapshot.SnapshotLogSplitter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.*;
@@ -84,7 +84,7 @@ public class TestSnapshotLogSplitter {
 
   @Test
   public void testSplitLogsOnDifferentTable() throws IOException {
-    byte[] tableName = getTableName(1);
+    TableName tableName = getTableName(1);
     Map<byte[], byte[]> regionsMap = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
     for (int j = 0; j < 10; ++j) {
       byte[] regionName = getRegionName(tableName, j);
@@ -97,9 +97,9 @@ public class TestSnapshotLogSplitter {
   /*
    * Split and verify test logs for the specified table
    */
-  private void splitTestLogs(final byte[] tableName, final Map<byte[], byte[]> regionsMap)
-      throws IOException {
-    Path tableDir = new Path(TEST_UTIL.getDataTestDir(), Bytes.toString(tableName));
+  private void splitTestLogs(final TableName tableName,
+                             final Map<byte[], byte[]> regionsMap) throws IOException {
+    Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDataTestDir(), tableName);
     SnapshotLogSplitter logSplitter = new SnapshotLogSplitter(conf, fs, tableDir,
       tableName, regionsMap);
     try {
@@ -113,10 +113,10 @@ public class TestSnapshotLogSplitter {
   /*
    * Verify that every logs in the table directory has just the specified table and regions.
    */
-  private void verifyRecoverEdits(final Path tableDir, final byte[] tableName,
+  private void verifyRecoverEdits(final Path tableDir, final TableName tableName,
       final Map<byte[], byte[]> regionsMap) throws IOException {
     for (FileStatus regionStatus: FSUtils.listStatus(fs, tableDir)) {
-      assertTrue(regionStatus.getPath().getName().startsWith(Bytes.toString(tableName)));
+      assertTrue(regionStatus.getPath().getName().startsWith(tableName.getNameAsString()));
       Path regionEdits = HLogUtil.getRegionDirRecoveredEditsDir(regionStatus.getPath());
       byte[] regionName = Bytes.toBytes(regionStatus.getPath().getName());
       assertFalse(regionsMap.containsKey(regionName));
@@ -126,7 +126,7 @@ public class TestSnapshotLogSplitter {
           HLog.Entry entry;
           while ((entry = reader.next()) != null) {
             HLogKey key = entry.getKey();
-            assertArrayEquals(tableName, key.getTablename());
+            assertEquals(tableName, key.getTablename());
             assertArrayEquals(regionName, key.getEncodedRegionName());
           }
         } finally {
@@ -147,7 +147,7 @@ public class TestSnapshotLogSplitter {
     HLog.Writer writer = HLogFactory.createWriter(fs, logFile, conf);
     try {
       for (int i = 0; i < 7; ++i) {
-        byte[] tableName = getTableName(i);
+        TableName tableName = getTableName(i);
         for (int j = 0; j < 10; ++j) {
           byte[] regionName = getRegionName(tableName, j);
           for (int k = 0; k < 50; ++k) {
@@ -165,15 +165,15 @@ public class TestSnapshotLogSplitter {
     }
   }
 
-  private byte[] getTableName(int tableId) {
-    return Bytes.toBytes("testtb-" + tableId);
+  private TableName getTableName(int tableId) {
+    return TableName.valueOf("testtb-" + tableId);
   }
 
-  private byte[] getRegionName(final byte[] tableName, int regionId) {
-    return Bytes.toBytes(Bytes.toString(tableName) + "-region-" + regionId);
+  private byte[] getRegionName(final TableName tableName, int regionId) {
+    return Bytes.toBytes(tableName + "-region-" + regionId);
   }
 
-  private byte[] getNewRegionName(final byte[] tableName, int regionId) {
-    return Bytes.toBytes(Bytes.toString(tableName) + "-new-region-" + regionId);
+  private byte[] getNewRegionName(final TableName tableName, int regionId) {
+    return Bytes.toBytes(tableName + "-new-region-" + regionId);
   }
 }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java Thu Aug  8 06:08:23 2013
@@ -18,15 +18,15 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.snapshot.SnapshotTask;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java Thu Aug  8 06:08:23 2013
@@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompatibilityFactory;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -51,11 +52,6 @@ import org.apache.hadoop.hbase.thrift.ge
 import org.apache.hadoop.hbase.thrift.generated.TRowResult;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.spi.NoEmitMetricsContext;
-import org.apache.hadoop.metrics.spi.OutputRecord;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -580,7 +576,8 @@ public class TestThriftServer {
     try {
       handler.mutateRow(tableAname, rowAname, getMutations(), null);
       byte[] searchRow = HRegionInfo.createRegionName(
-          tableAname.array(), rowAname.array(), HConstants.NINES, false);
+          TableName.valueOf(tableAname.array()), rowAname.array(),
+          HConstants.NINES, false);
       TRegionInfo regionInfo = handler.getRegionInfo(ByteBuffer.wrap(searchRow));
       assertTrue(Bytes.toStringBinary(regionInfo.getName()).startsWith(
             Bytes.toStringBinary(tableAname)));

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.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.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
+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.Put;
@@ -130,7 +131,7 @@ public class TestThriftHBaseServiceHandl
   public static void beforeClass() throws Exception {
     UTIL.startMiniCluster();
     HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration());
-    HTableDescriptor tableDescriptor = new HTableDescriptor(tableAname);
+    HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(tableAname));
     for (HColumnDescriptor family : families) {
       tableDescriptor.addFamily(family);
     }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java Thu Aug  8 06:08:23 2013
@@ -32,11 +32,9 @@ 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.HRegionInfo;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.HStore;
 
 /**
  * Test helper for testing archiving of HFiles
@@ -212,9 +210,11 @@ public class HFileArchiveTestingUtil {
    * @param region region that is being archived
    * @return {@link Path} to the archive directory for the given region
    */
-  public static Path getRegionArchiveDir(Configuration conf, HRegion region) {
-    return HFileArchiveUtil.getRegionArchiveDir(region.getRegionFileSystem().getTableDir(),
-        region.getRegionFileSystem().getRegionDir());
+  public static Path getRegionArchiveDir(Configuration conf, HRegion region) throws IOException {
+    return HFileArchiveUtil.getRegionArchiveDir(
+        FSUtils.getRootDir(conf),
+        region.getTableDesc().getTableName(),
+        region.getRegionInfo().getEncodedName());
   }
 
   /**

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java Thu Aug  8 06:08:23 2013
@@ -27,6 +27,7 @@ import org.apache.commons.cli.CommandLin
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -50,7 +51,7 @@ public class LoadTestTool extends Abstra
   private static final Log LOG = LogFactory.getLog(LoadTestTool.class);
 
   /** Table name for the test */
-  protected byte[] tableName;
+  private TableName tableName;
 
   /** Table name to use of not overridden on the command line */
   protected static final String DEFAULT_TABLE_NAME = "cluster_test";
@@ -169,11 +170,11 @@ public class LoadTestTool extends Abstra
    * Apply column family options such as Bloom filters, compression, and data
    * block encoding.
    */
-  protected void applyColumnFamilyOptions(byte[] tableName,
+  protected void applyColumnFamilyOptions(TableName tableName,
       byte[][] columnFamilies) throws IOException {
     HBaseAdmin admin = new HBaseAdmin(conf);
     HTableDescriptor tableDesc = admin.getTableDescriptor(tableName);
-    LOG.info("Disabling table " + Bytes.toString(tableName));
+    LOG.info("Disabling table " + tableName);
     admin.disableTable(tableName);
     for (byte[] cf : columnFamilies) {
       HColumnDescriptor columnDesc = tableDesc.getFamily(cf);
@@ -200,7 +201,7 @@ public class LoadTestTool extends Abstra
         admin.modifyColumn(tableName, columnDesc);
       }
     }
-    LOG.info("Enabling table " + Bytes.toString(tableName));
+    LOG.info("Enabling table " + tableName);
     admin.enableTable(tableName);
   }
 
@@ -244,7 +245,7 @@ public class LoadTestTool extends Abstra
   protected void processOptions(CommandLine cmd) {
     this.cmd = cmd;
 
-    tableName = Bytes.toBytes(cmd.getOptionValue(OPT_TABLE_NAME,
+    tableName = TableName.valueOf(cmd.getOptionValue(OPT_TABLE_NAME,
         DEFAULT_TABLE_NAME));
 
     isWrite = cmd.hasOption(OPT_WRITE);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java Thu Aug  8 06:08:23 2013
@@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentSk
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
@@ -79,7 +80,7 @@ public class MockRegionServerServices im
     return this.regions.get(encodedRegionName);
   }
 
-  public List<HRegion> getOnlineRegions(byte[] tableName) throws IOException {
+  public List<HRegion> getOnlineRegions(TableName tableName) throws IOException {
     return null;
   }
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java Thu Aug  8 06:08:23 2013
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.util;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
-import java.util.NavigableMap;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -28,6 +27,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.client.Result;
 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
 import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator;
@@ -40,7 +40,7 @@ import org.apache.hadoop.util.StringUtil
 public abstract class MultiThreadedAction {
   private static final Log LOG = LogFactory.getLog(MultiThreadedAction.class);
 
-  protected final byte[] tableName;
+  protected final TableName tableName;
   protected final Configuration conf;
 
   protected int numThreads = 1;
@@ -129,8 +129,9 @@ public abstract class MultiThreadedActio
 
   public static final int REPORTING_INTERVAL_MS = 5000;
 
-  public MultiThreadedAction(LoadTestDataGenerator dataGen, Configuration conf, byte[] tableName,
-      String actionLetter) {
+  public MultiThreadedAction(LoadTestDataGenerator dataGen, Configuration conf,
+                             TableName tableName,
+                             String actionLetter) {
     this.conf = conf;
     this.dataGenerator = dataGen;
     this.tableName = tableName;

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java Thu Aug  8 06:08:23 2013
@@ -18,8 +18,6 @@ package org.apache.hadoop.hbase.util;
 
 import java.io.IOException;
 import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
@@ -27,13 +25,12 @@ 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.HRegionLocation;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
-import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator;
 
 /** Creates multiple threads that read and verify previously written data */
 public class MultiThreadedReader extends MultiThreadedAction
@@ -75,7 +72,7 @@ public class MultiThreadedReader extends
   private int keyWindow = DEFAULT_KEY_WINDOW;
 
   public MultiThreadedReader(LoadTestDataGenerator dataGen, Configuration conf,
-      byte[] tableName, double verifyPercent) {
+      TableName tableName, double verifyPercent) {
     super(dataGen, conf, tableName, "R");
     this.verifyPercent = verifyPercent;
   }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java Thu Aug  8 06:08:23 2013
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.HashSet;
-import java.util.Map;
 import java.util.PriorityQueue;
 import java.util.Queue;
 import java.util.Set;
@@ -33,13 +32,12 @@ 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.util.StringUtils;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
-import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator;
-import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
 
 /** Creates multiple threads that write key/values into the */
@@ -83,7 +81,7 @@ public class MultiThreadedWriter extends
   private boolean trackInsertedKeys;
 
   public MultiThreadedWriter(LoadTestDataGenerator dataGen, Configuration conf,
-    byte[] tableName) {
+    TableName tableName) {
     super(dataGen, conf, tableName, "W");
   }
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java Thu Aug  8 06:08:23 2013
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -171,7 +172,7 @@ public class ProcessBasedLocalHBaseClust
     int attemptsLeft = 10;
     while (attemptsLeft-- > 0) {
       try {
-        new HTable(conf, HConstants.META_TABLE_NAME);
+        new HTable(conf, TableName.META_TABLE_NAME);
       } catch (Exception e) {
         LOG.info("Waiting for HBase to startup. Retries left: " + attemptsLeft,
             e);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java Thu Aug  8 06:08:23 2013
@@ -21,6 +21,7 @@ import java.io.IOException;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTable;
@@ -44,7 +45,8 @@ public class RestartMetaTest extends Abs
   private static final int DEFAULT_NUM_RS = 2;
 
   /** Table name for the test */
-  private static byte[] TABLE_NAME = Bytes.toBytes("load_test");
+  private static TableName TABLE_NAME =
+      TableName.valueOf("load_test");
 
   /** The number of seconds to sleep after loading the data */
   private static final int SLEEP_SEC_AFTER_DATA_LOAD = 5;
@@ -118,7 +120,7 @@ public class RestartMetaTest extends Abs
 
     LOG.debug("Trying to scan meta");
 
-    HTable metaTable = new HTable(conf, HConstants.META_TABLE_NAME);
+    HTable metaTable = new HTable(conf, TableName.META_TABLE_NAME);
     ResultScanner scanner = metaTable.getScanner(new Scan());
     Result result;
     while ((result = scanner.next()) != null) {

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java Thu Aug  8 06:08:23 2013
@@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -57,7 +58,6 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -101,7 +101,8 @@ public class TestCoprocessorScanPolicy {
 
   @Test
   public void testBaseCases() throws Exception {
-    byte[] tableName = Bytes.toBytes("baseCases");
+    TableName tableName =
+        TableName.valueOf("baseCases");
     if (TEST_UTIL.getHBaseAdmin().tableExists(tableName)) {
       TEST_UTIL.deleteTable(tableName);
     }
@@ -109,7 +110,7 @@ public class TestCoprocessorScanPolicy {
     // set the version override to 2
     Put p = new Put(R);
     p.setAttribute("versions", new byte[]{});
-    p.add(F, tableName, Bytes.toBytes(2));
+    p.add(F, tableName.getName(), Bytes.toBytes(2));
     t.put(p);
 
     long now = EnvironmentEdgeManager.currentTimeMillis();
@@ -150,7 +151,8 @@ public class TestCoprocessorScanPolicy {
 
   @Test
   public void testTTL() throws Exception {
-    byte[] tableName = Bytes.toBytes("testTTL");
+    TableName tableName =
+        TableName.valueOf("testTTL");
     if (TEST_UTIL.getHBaseAdmin().tableExists(tableName)) {
       TEST_UTIL.deleteTable(tableName);
     }
@@ -170,7 +172,7 @@ public class TestCoprocessorScanPolicy {
     // Set the TTL override to 3s
     Put p = new Put(R);
     p.setAttribute("ttl", new byte[]{});
-    p.add(F, tableName, Bytes.toBytes(3000L));
+    p.add(F, tableName.getName(), Bytes.toBytes(3000L));
     t.put(p);
 
     p = new Put(R);
@@ -209,8 +211,10 @@ public class TestCoprocessorScanPolicy {
   }
 
   public static class ScanObserver extends BaseRegionObserver {
-    private Map<String, Long> ttls = new HashMap<String,Long>();
-    private Map<String, Integer> versions = new HashMap<String,Integer>();
+    private Map<TableName, Long> ttls =
+        new HashMap<TableName, Long>();
+    private Map<TableName, Integer> versions =
+        new HashMap<TableName, Integer>();
 
     // lame way to communicate with the coprocessor,
     // since it is loaded by a different class loader
@@ -220,12 +224,12 @@ public class TestCoprocessorScanPolicy {
       if (put.getAttribute("ttl") != null) {
         Cell cell = put.getFamilyMap().values().iterator().next().get(0);
         KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        ttls.put(Bytes.toString(kv.getQualifier()), Bytes.toLong(kv.getValue()));
+        ttls.put(TableName.valueOf(kv.getQualifier()), Bytes.toLong(kv.getValue()));
         c.bypass();
       } else if (put.getAttribute("versions") != null) {
         Cell cell = put.getFamilyMap().values().iterator().next().get(0);
         KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        versions.put(Bytes.toString(kv.getQualifier()), Bytes.toInt(kv.getValue()));
+        versions.put(TableName.valueOf(kv.getQualifier()), Bytes.toInt(kv.getValue()));
         c.bypass();
       }
     }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java Thu Aug  8 06:08:23 2013
@@ -34,6 +34,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -67,7 +68,7 @@ public class TestFSTableDescriptors {
   @Test
   public void testCreateAndUpdate() throws IOException {
     Path testdir = UTIL.getDataTestDir("testCreateAndUpdate");
-    HTableDescriptor htd = new HTableDescriptor("testCreate");
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCreate"));
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
     assertTrue(fstd.createTableDescriptor(htd));
@@ -79,7 +80,7 @@ public class TestFSTableDescriptors {
     }
     statuses = fs.listStatus(testdir);
     assertTrue(statuses.length == 1);
-    Path tmpTableDir = new Path(FSUtils.getTablePath(testdir, htd.getName()), ".tmp");
+    Path tmpTableDir = new Path(FSUtils.getTableDir(testdir, htd.getTableName()), ".tmp");
     statuses = fs.listStatus(tmpTableDir);
     assertTrue(statuses.length == 0);
   }
@@ -87,7 +88,8 @@ public class TestFSTableDescriptors {
   @Test
   public void testSequenceIdAdvancesOnTableInfo() throws IOException {
     Path testdir = UTIL.getDataTestDir("testSequenceidAdvancesOnTableInfo");
-    HTableDescriptor htd = new HTableDescriptor("testSequenceidAdvancesOnTableInfo");
+    HTableDescriptor htd = new HTableDescriptor(
+        TableName.valueOf("testSequenceidAdvancesOnTableInfo"));
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
     Path p0 = fstd.updateTableDescriptor(htd);
@@ -143,21 +145,21 @@ public class TestFSTableDescriptors {
     // Cleanup old tests if any detrius laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     htds.add(htd);
-    assertNotNull(htds.remove(htd.getNameAsString()));
-    assertNull(htds.remove(htd.getNameAsString()));
+    assertNotNull(htds.remove(htd.getTableName()));
+    assertNull(htds.remove(htd.getTableName()));
   }
 
   @Test public void testReadingHTDFromFS() throws IOException {
     final String name = "testReadingHTDFromFS";
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     Path rootdir = UTIL.getDataTestDir(name);
     FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
     fstd.createTableDescriptor(htd);
     HTableDescriptor htd2 =
-      FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getNameAsString());
+      FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName());
     assertTrue(htd.equals(htd2));
   }
 
@@ -169,9 +171,9 @@ public class TestFSTableDescriptors {
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     FSTableDescriptors htds = new FSTableDescriptors(fs, rootdir) {
       @Override
-      public HTableDescriptor get(byte[] tablename)
+      public HTableDescriptor get(TableName tablename)
           throws TableExistsException, FileNotFoundException, IOException {
-        LOG.info(Bytes.toString(tablename) + ", cachehits=" + this.cachehits);
+        LOG.info(tablename + ", cachehits=" + this.cachehits);
         return super.get(tablename);
       }
     };
@@ -183,29 +185,29 @@ public class TestFSTableDescriptors {
     }
 
     for (int i = 0; i < count; i++) {
-      assertTrue(htds.get(Bytes.toBytes(name + i)) !=  null);
+      assertTrue(htds.get(TableName.valueOf(name + i)) !=  null);
     }
     for (int i = 0; i < count; i++) {
-      assertTrue(htds.get(Bytes.toBytes(name + i)) !=  null);
+      assertTrue(htds.get(TableName.valueOf(name + i)) !=  null);
     }
     // Update the table infos
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       htd.addFamily(new HColumnDescriptor("" + i));
       htds.updateTableDescriptor(htd);
     }
     // Wait a while so mod time we write is for sure different.
     Thread.sleep(100);
     for (int i = 0; i < count; i++) {
-      assertTrue(htds.get(Bytes.toBytes(name + i)) !=  null);
+      assertTrue(htds.get(TableName.valueOf(name + i)) !=  null);
     }
     for (int i = 0; i < count; i++) {
-      assertTrue(htds.get(Bytes.toBytes(name + i)) !=  null);
+      assertTrue(htds.get(TableName.valueOf(name + i)) !=  null);
     }
     assertEquals(count * 4, htds.invocations);
     assertTrue("expected=" + (count * 2) + ", actual=" + htds.cachehits,
       htds.cachehits >= (count * 2));
-    assertTrue(htds.get(HConstants.ROOT_TABLE_NAME) != null);
+    assertTrue(htds.get(TableName.ROOT_TABLE_NAME) != null);
     assertEquals(htds.invocations, count * 4 + 1);
     assertTrue("expected=" + ((count * 2) + 1) + ", actual=" + htds.cachehits,
       htds.cachehits >= ((count * 2) + 1));
@@ -219,7 +221,7 @@ public class TestFSTableDescriptors {
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
     assertNull("There shouldn't be any HTD for this table",
-      htds.get("NoSuchTable"));
+      htds.get(TableName.valueOf("NoSuchTable")));
   }
 
   @Test
@@ -229,7 +231,7 @@ public class TestFSTableDescriptors {
     // Cleanup old tests if any detrius laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     htds.add(htd);
     htds.add(htd);
     htds.add(htd);
@@ -266,9 +268,9 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     try {
       new FSTableDescriptors(fs, FSUtils.getRootDir(UTIL.getConfiguration()))
-          .get(HConstants.HFILE_ARCHIVE_DIRECTORY);
+          .get(TableName.valueOf(HConstants.HFILE_ARCHIVE_DIRECTORY));
       fail("Shouldn't be able to read a table descriptor for the archive directory.");
-    } catch (IOException e) {
+    } catch (Exception e) {
       LOG.debug("Correctly got error when reading a table descriptor from the archive directory: "
           + e.getMessage());
     }
@@ -277,15 +279,15 @@ public class TestFSTableDescriptors {
   @Test
   public void testCreateTableDescriptorUpdatesIfExistsAlready() throws IOException {
     Path testdir = UTIL.getDataTestDir("testCreateTableDescriptorUpdatesIfThereExistsAlready");
-    HTableDescriptor htd = new HTableDescriptor(
-        "testCreateTableDescriptorUpdatesIfThereExistsAlready");
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(
+        "testCreateTableDescriptorUpdatesIfThereExistsAlready"));
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
     assertTrue(fstd.createTableDescriptor(htd));
     assertFalse(fstd.createTableDescriptor(htd));
     htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue"));
     assertTrue(fstd.createTableDescriptor(htd)); //this will re-create
-    Path tableDir = fstd.getTableDirectory(htd.getNameAsString());
+    Path tableDir = fstd.getTableDir(htd.getTableName());
     Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR);
     FileStatus[] statuses = fs.listStatus(tmpTableDir);
     assertTrue(statuses.length == 0);