You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/08/08 06:19:56 UTC

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

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Thu Aug  8 04:19:49 2013
@@ -856,7 +856,7 @@ public class HBaseTestingUtility extends
     this.hbaseCluster =
         new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass);
     // Don't leave here till we've done a successful scan of the .META.
-    HTable t = new HTable(c, HConstants.META_TABLE_NAME);
+    HTable t = new HTable(c, TableName.META_TABLE_NAME);
     ResultScanner s = t.getScanner(new Scan());
     while (s.next() != null) {
       continue;
@@ -878,7 +878,7 @@ public class HBaseTestingUtility extends
   public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
     this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
     // Don't leave here till we've done a successful scan of the .META.
-    HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
+    HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
     ResultScanner s = t.getScanner(new Scan());
     while (s.next() != null) {
       // do nothing
@@ -985,7 +985,7 @@ public class HBaseTestingUtility extends
    * Flushes all caches in the mini hbase cluster
    * @throws IOException
    */
-  public void flush(byte [] tableName) throws IOException {
+  public void flush(TableName tableName) throws IOException {
     getMiniHBaseCluster().flushcache(tableName);
   }
 
@@ -1001,7 +1001,7 @@ public class HBaseTestingUtility extends
    * Compact all of a table's reagion in the mini hbase cluster
    * @throws IOException
    */
-  public void compact(byte [] tableName, boolean major) throws IOException {
+  public void compact(TableName tableName, boolean major) throws IOException {
     getMiniHBaseCluster().compact(tableName, major);
   }
 
@@ -1014,7 +1014,19 @@ public class HBaseTestingUtility extends
    */
   public HTable createTable(String tableName, String family)
   throws IOException{
-    return createTable(tableName, new String[] { family });
+    return createTable(TableName.valueOf(tableName), new String[]{family});
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
+   * @param family
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(byte[] tableName, byte[] family)
+  throws IOException{
+    return createTable(TableName.valueOf(tableName), new byte[][]{family});
   }
 
   /**
@@ -1026,11 +1038,23 @@ public class HBaseTestingUtility extends
    */
   public HTable createTable(String tableName, String[] families)
   throws IOException {
+    return createTable(tableName, families);
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
+   * @param families
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(TableName tableName, String[] families)
+  throws IOException {
     List<byte[]> fams = new ArrayList<byte[]>(families.length);
     for (String family : families) {
       fams.add(Bytes.toBytes(family));
     }
-    return createTable(Bytes.toBytes(tableName), fams.toArray(new byte[0][]));
+    return createTable(tableName, fams.toArray(new byte[0][]));
   }
 
   /**
@@ -1040,11 +1064,12 @@ public class HBaseTestingUtility extends
    * @return An HTable instance for the created table.
    * @throws IOException
    */
-  public HTable createTable(byte[] tableName, byte[] family)
+  public HTable createTable(TableName tableName, byte[] family)
   throws IOException{
     return createTable(tableName, new byte[][]{family});
   }
 
+
   /**
    * Create a table.
    * @param tableName
@@ -1058,7 +1083,32 @@ public class HBaseTestingUtility extends
         new Configuration(getConfiguration()));
   }
 
+  /**
+   * Create a table.
+   * @param tableName
+   * @param families
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(TableName tableName, byte[][] families)
+  throws IOException {
+    return createTable(tableName, families,
+        new Configuration(getConfiguration()));
+  }
+
   public HTable createTable(byte[] tableName, byte[][] families,
+      int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
+    return createTable(TableName.valueOf(tableName), families, numVersions,
+        startKey, endKey, numRegions);
+  }
+
+  public HTable createTable(String tableName, byte[][] families,
+      int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
+    return createTable(TableName.valueOf(tableName), families, numVersions,
+        startKey, endKey, numRegions);
+  }
+
+  public HTable createTable(TableName tableName, byte[][] families,
       int numVersions, byte[] startKey, byte[] endKey, int numRegions)
   throws IOException{
     HTableDescriptor desc = new HTableDescriptor(tableName);
@@ -1081,7 +1131,7 @@ public class HBaseTestingUtility extends
    * @return An HTable instance for the created table.
    * @throws IOException
    */
-  public HTable createTable(byte[] tableName, byte[][] families,
+  public HTable createTable(TableName tableName, byte[][] families,
       final Configuration c)
   throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
@@ -1104,11 +1154,35 @@ public class HBaseTestingUtility extends
    * @param tableName
    * @param families
    * @param c Configuration to use
-   * @param numVersions
    * @return An HTable instance for the created table.
    * @throws IOException
    */
   public HTable createTable(byte[] tableName, byte[][] families,
+      final Configuration c)
+  throws IOException {
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+    for(byte[] family : families) {
+      HColumnDescriptor hcd = new HColumnDescriptor(family);
+      // Disable blooms (they are on by default as of 0.95) but we disable them here because
+      // tests have hard coded counts of what to expect in block cache, etc., and blooms being
+      // on is interfering.
+      hcd.setBloomFilterType(BloomType.NONE);
+      desc.addFamily(hcd);
+    }
+    getHBaseAdmin().createTable(desc);
+    return new HTable(c, tableName);
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
+   * @param families
+   * @param c Configuration to use
+   * @param numVersions
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(TableName tableName, byte[][] families,
       final Configuration c, int numVersions)
   throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
@@ -1126,6 +1200,28 @@ public class HBaseTestingUtility extends
   /**
    * Create a table.
    * @param tableName
+   * @param families
+   * @param c Configuration to use
+   * @param numVersions
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(byte[] tableName, byte[][] families,
+      final Configuration c, int numVersions)
+  throws IOException {
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+    for(byte[] family : families) {
+      HColumnDescriptor hcd = new HColumnDescriptor(family)
+          .setMaxVersions(numVersions);
+      desc.addFamily(hcd);
+    }
+    getHBaseAdmin().createTable(desc);
+    return new HTable(c, tableName);
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
    * @param family
    * @param numVersions
    * @return An HTable instance for the created table.
@@ -1139,6 +1235,19 @@ public class HBaseTestingUtility extends
   /**
    * Create a table.
    * @param tableName
+   * @param family
+   * @param numVersions
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(TableName tableName, byte[] family, int numVersions)
+  throws IOException {
+    return createTable(tableName, new byte[][]{family}, numVersions);
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
    * @param families
    * @param numVersions
    * @return An HTable instance for the created table.
@@ -1147,6 +1256,20 @@ public class HBaseTestingUtility extends
   public HTable createTable(byte[] tableName, byte[][] families,
       int numVersions)
   throws IOException {
+    return createTable(TableName.valueOf(tableName), families, numVersions);
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
+   * @param families
+   * @param numVersions
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(TableName tableName, byte[][] families,
+      int numVersions)
+  throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     for (byte[] family : families) {
       HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions);
@@ -1168,6 +1291,20 @@ public class HBaseTestingUtility extends
    */
   public HTable createTable(byte[] tableName, byte[][] families,
     int numVersions, int blockSize) throws IOException {
+    return createTable(TableName.valueOf(tableName),
+        families, numVersions, blockSize);
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
+   * @param families
+   * @param numVersions
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(TableName tableName, byte[][] families,
+    int numVersions, int blockSize) throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     for (byte[] family : families) {
       HColumnDescriptor hcd = new HColumnDescriptor(family)
@@ -1192,6 +1329,20 @@ public class HBaseTestingUtility extends
   public HTable createTable(byte[] tableName, byte[][] families,
       int[] numVersions)
   throws IOException {
+    return createTable(TableName.valueOf(tableName), families, numVersions);
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
+   * @param families
+   * @param numVersions
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(TableName tableName, byte[][] families,
+      int[] numVersions)
+  throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     int i = 0;
     for (byte[] family : families) {
@@ -1215,6 +1366,19 @@ public class HBaseTestingUtility extends
    * @throws IOException
    */
   public HTable createTable(byte[] tableName, byte[] family, byte[][] splitRows)
+    throws IOException{
+    return createTable(TableName.valueOf(tableName), family, splitRows);
+  }
+
+  /**
+   * Create a table.
+   * @param tableName
+   * @param family
+   * @param splitRows
+   * @return An HTable instance for the created table.
+   * @throws IOException
+   */
+  public HTable createTable(TableName tableName, byte[] family, byte[][] splitRows)
       throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor(family);
@@ -1235,14 +1399,14 @@ public class HBaseTestingUtility extends
    */
   public HTable createTable(byte[] tableName, byte[][] families, byte[][] splitRows)
       throws IOException {
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     for(byte[] family:families) {
       HColumnDescriptor hcd = new HColumnDescriptor(family);
       desc.addFamily(hcd);
     }
     getHBaseAdmin().createTable(desc, splitRows);
     // HBaseAdmin only waits for regions to appear in META we should wait until they are assigned
-    waitUntilAllRegionsAssigned(tableName);
+    waitUntilAllRegionsAssigned(TableName.valueOf(tableName));
     return new HTable(getConfiguration(), tableName);
   }
 
@@ -1251,7 +1415,7 @@ public class HBaseTestingUtility extends
    * @param tableName existing table
    */
   public void deleteTable(String tableName) throws IOException {
-    deleteTable(Bytes.toBytes(tableName));
+    deleteTable(TableName.valueOf(tableName));
   }
 
   /**
@@ -1259,21 +1423,40 @@ public class HBaseTestingUtility extends
    * @param tableName existing table
    */
   public void deleteTable(byte[] tableName) throws IOException {
+    deleteTable(TableName.valueOf(tableName));
+  }
+
+  /**
+   * Drop an existing table
+   * @param tableName existing table
+   */
+  public void deleteTable(TableName tableName) throws IOException {
     try {
       getHBaseAdmin().disableTable(tableName);
     } catch (TableNotEnabledException e) {
-      LOG.debug("Table: " + Bytes.toString(tableName) + " already disabled, so just deleting it.");
+      LOG.debug("Table: " + tableName + " already disabled, so just deleting it.");
     }
     getHBaseAdmin().deleteTable(tableName);
   }
 
+
   /**
    * Provide an existing table name to truncate
    * @param tableName existing table
    * @return HTable to that new table
    * @throws IOException
    */
-  public HTable truncateTable(byte [] tableName) throws IOException {
+  public HTable truncateTable(byte[] tableName) throws IOException {
+    return truncateTable(TableName.valueOf(tableName));
+  }
+
+  /**
+   * Provide an existing table name to truncate
+   * @param tableName existing table
+   * @return HTable to that new table
+   * @throws IOException
+   */
+  public HTable truncateTable(TableName tableName) throws IOException {
     HTable table = new HTable(getConfiguration(), tableName);
     Scan scan = new Scan();
     ResultScanner resScan = table.getScanner(scan);
@@ -1510,7 +1693,7 @@ public class HBaseTestingUtility extends
       final byte[] columnFamily, byte [][] startKeys)
   throws IOException {
     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
-    HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
+    HTable meta = new HTable(c, TableName.META_TABLE_NAME);
     HTableDescriptor htd = table.getTableDescriptor();
     if(!htd.hasFamily(columnFamily)) {
       HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
@@ -1520,7 +1703,7 @@ public class HBaseTestingUtility extends
     // setup already has the "<tablename>,,123456789" row with an empty start
     // and end key. Adding the custom regions below adds those blindly,
     // including the new start region from empty to "bbb". lg
-    List<byte[]> rows = getMetaTableRows(htd.getName());
+    List<byte[]> rows = getMetaTableRows(htd.getTableName());
     String regionToDeleteInFS = table
         .getRegionsInRange(Bytes.toBytes(""), Bytes.toBytes("")).get(0)
         .getRegionInfo().getEncodedName();
@@ -1529,7 +1712,7 @@ public class HBaseTestingUtility extends
     int count = 0;
     for (int i = 0; i < startKeys.length; i++) {
       int j = (i + 1) % startKeys.length;
-      HRegionInfo hri = new HRegionInfo(table.getTableName(),
+      HRegionInfo hri = new HRegionInfo(table.getName(),
         startKeys[i], startKeys[j]);
       MetaEditor.addRegionToMeta(meta, hri);
       newRegions.add(hri);
@@ -1543,7 +1726,7 @@ public class HBaseTestingUtility extends
     }
     // remove the "old" region from FS
     Path tableDir = new Path(getDefaultRootDirPath().toString()
-        + System.getProperty("file.separator") + htd.getNameAsString()
+        + System.getProperty("file.separator") + htd.getTableName()
         + System.getProperty("file.separator") + regionToDeleteInFS);
     FileSystem.get(c).delete(tableDir);
     // flush cache of regions
@@ -1575,13 +1758,13 @@ public class HBaseTestingUtility extends
   public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
       final HTableDescriptor htd, byte [][] startKeys)
   throws IOException {
-    HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
+    HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
     // add custom ones
     for (int i = 0; i < startKeys.length; i++) {
       int j = (i + 1) % startKeys.length;
-      HRegionInfo hri = new HRegionInfo(htd.getName(), startKeys[i],
+      HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKeys[i],
           startKeys[j]);
       MetaEditor.addRegionToMeta(meta, hri);
       newRegions.add(hri);
@@ -1598,7 +1781,7 @@ public class HBaseTestingUtility extends
    */
   public List<byte[]> getMetaTableRows() throws IOException {
     // TODO: Redo using MetaReader class
-    HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
+    HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
     List<byte[]> rows = new ArrayList<byte[]>();
     ResultScanner s = t.getScanner(new Scan());
     for (Result result : s) {
@@ -1616,9 +1799,9 @@ public class HBaseTestingUtility extends
    *
    * @throws IOException When reading the rows fails.
    */
-  public List<byte[]> getMetaTableRows(byte[] tableName) throws IOException {
+  public List<byte[]> getMetaTableRows(TableName tableName) throws IOException {
     // TODO: Redo using MetaReader.
-    HTable t = new HTable(new Configuration(this.conf), HConstants.META_TABLE_NAME);
+    HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
     List<byte[]> rows = new ArrayList<byte[]>();
     ResultScanner s = t.getScanner(new Scan());
     for (Result result : s) {
@@ -1629,7 +1812,7 @@ public class HBaseTestingUtility extends
         continue;
       }
 
-      if (Bytes.compareTo(info.getTableName(), tableName) == 0) {
+      if (info.getTableName().equals(tableName)) {
         LOG.info("getMetaTableRows: row -> " +
             Bytes.toStringBinary(result.getRow()) + info);
         rows.add(result.getRow());
@@ -1652,13 +1835,27 @@ public class HBaseTestingUtility extends
    * @throws InterruptedException
    */
   public HRegionServer getRSForFirstRegionInTable(byte[] tableName)
-  throws IOException, InterruptedException {
+      throws IOException, InterruptedException {
+    return getRSForFirstRegionInTable(TableName.valueOf(tableName));
+  }
+  /**
+   * Tool to get the reference to the region server object that holds the
+   * region of the specified user table.
+   * It first searches for the meta rows that contain the region of the
+   * specified table, then gets the index of that RS, and finally retrieves
+   * the RS's reference.
+   * @param tableName user table to lookup in .META.
+   * @return region server that holds it, null if the row doesn't exist
+   * @throws IOException
+   */
+  public HRegionServer getRSForFirstRegionInTable(TableName tableName)
+      throws IOException, InterruptedException {
     List<byte[]> metaRows = getMetaTableRows(tableName);
     if (metaRows == null || metaRows.isEmpty()) {
       return null;
     }
     LOG.debug("Found " + metaRows.size() + " rows for table " +
-      Bytes.toString(tableName));
+      tableName);
     byte [] firstrow = metaRows.get(0);
     LOG.debug("FirstRow=" + Bytes.toString(firstrow));
     long pause = getConfiguration().getLong(HConstants.HBASE_CLIENT_PAUSE,
@@ -1903,7 +2100,7 @@ public class HBaseTestingUtility extends
     monitor.close();
 
     if (checkStatus) {
-      new HTable(new Configuration(conf), HConstants.META_TABLE_NAME).close();
+      new HTable(new Configuration(conf), TableName.META_TABLE_NAME).close();
     }
   }
 
@@ -2025,7 +2222,7 @@ public class HBaseTestingUtility extends
    * @param maxAttempts maximum number of attempts, unlimited for value of -1
    * @return the HRegion chosen, null if none was found within limit of maxAttempts
    */
-  public HRegion getSplittableRegion(byte[] tableName, int maxAttempts) {
+  public HRegion getSplittableRegion(TableName tableName, int maxAttempts) {
     List<HRegion> regions = getHBaseCluster().getRegions(tableName);
     int regCount = regions.size();
     Set<Integer> attempted = new HashSet<Integer>();
@@ -2173,7 +2370,7 @@ public class HBaseTestingUtility extends
     // Below we do a get.  The get will retry if a NotServeringRegionException or a
     // RegionOpeningException.  It is crass but when done all will be online.
     try {
-      Canary.sniff(admin, Bytes.toString(table));
+      Canary.sniff(admin, TableName.valueOf(table));
     } catch (Exception e) {
       throw new IOException(e);
     }
@@ -2297,7 +2494,7 @@ public class HBaseTestingUtility extends
    * @param tableName the table name
    * @throws IOException
    */
-  public void waitUntilAllRegionsAssigned(final byte[] tableName) throws IOException {
+  public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOException {
     waitUntilAllRegionsAssigned(tableName, 60000);
   }
 
@@ -2310,9 +2507,9 @@ public class HBaseTestingUtility extends
    * @param timeout timeout, in milliseconds
    * @throws IOException
    */
-  public void waitUntilAllRegionsAssigned(final byte[] tableName, final long timeout)
+  public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout)
       throws IOException {
-    final HTable meta = new HTable(getConfiguration(), HConstants.META_TABLE_NAME);
+    final HTable meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME);
     try {
       waitFor(timeout, 200, true, new Predicate<IOException>() {
         @Override
@@ -2326,7 +2523,7 @@ public class HBaseTestingUtility extends
             while ((r = s.next()) != null) {
               byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
               HRegionInfo info = HRegionInfo.parseFromOrNull(b);
-              if (info != null && Bytes.equals(info.getTableName(), tableName)) {
+              if (info != null && info.getTableName().equals(tableName)) {
                 b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
                 allRegionsAssigned &= (b != null);
               }
@@ -2509,8 +2706,6 @@ public class HBaseTestingUtility extends
     final Random rand = new Random(tableName.hashCode() * 17L + 12938197137L);
     final int numCF = families.size();
     final byte[][] cfBytes = new byte[numCF][];
-    final byte[] tableNameBytes = Bytes.toBytes(tableName);
-
     {
       int cfIndex = 0;
       for (String cf : families) {
@@ -2524,14 +2719,14 @@ public class HBaseTestingUtility extends
     final int splitStartKey = actualStartKey + keysPerRegion;
     final int splitEndKey = actualEndKey - keysPerRegion;
     final String keyFormat = "%08x";
-    final HTable table = createTable(tableNameBytes, cfBytes,
+    final HTable table = createTable(tableName, cfBytes,
         maxVersions,
         Bytes.toBytes(String.format(keyFormat, splitStartKey)),
         Bytes.toBytes(String.format(keyFormat, splitEndKey)),
         numRegions);
 
     if (hbaseCluster != null) {
-      getMiniHBaseCluster().flushcache(HConstants.META_TABLE_NAME);
+      getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME);
     }
 
     for (int iFlush = 0; iFlush < numFlushes; ++iFlush) {
@@ -2568,7 +2763,7 @@ public class HBaseTestingUtility extends
       LOG.info("Initiating flush #" + iFlush + " for table " + tableName);
       table.flushCommits();
       if (hbaseCluster != null) {
-        getMiniHBaseCluster().flushcache(tableNameBytes);
+        getMiniHBaseCluster().flushcache(table.getName());
       }
     }
 
@@ -2650,7 +2845,7 @@ public class HBaseTestingUtility extends
    * @return the number of regions the table was split into
    */
   public static int createPreSplitLoadTestTable(Configuration conf,
-      byte[] tableName, byte[] columnFamily, Algorithm compression,
+      TableName tableName, byte[] columnFamily, Algorithm compression,
       DataBlockEncoding dataBlockEncoding) throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
@@ -2695,7 +2890,7 @@ public class HBaseTestingUtility extends
       LOG.error("Master not running", e);
       throw new IOException(e);
     } catch (TableExistsException e) {
-      LOG.warn("Table " + Bytes.toStringBinary(desc.getName()) +
+      LOG.warn("Table " + desc.getTableName() +
           " already exists, continuing");
     } finally {
       admin.close();
@@ -2704,7 +2899,7 @@ public class HBaseTestingUtility extends
   }
 
   public static int getMetaRSPort(Configuration conf) throws IOException {
-    HTable table = new HTable(conf, HConstants.META_TABLE_NAME);
+    HTable table = new HTable(conf, TableName.META_TABLE_NAME);
     HRegionLocation hloc = table.getRegionLocation(Bytes.toBytes(""));
     table.close();
     return hloc.getPort();
@@ -2733,10 +2928,10 @@ public class HBaseTestingUtility extends
 
   public HRegion createTestRegion(String tableName, HColumnDescriptor hcd)
       throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
     htd.addFamily(hcd);
     HRegionInfo info =
-        new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
+        new HRegionInfo(TableName.valueOf(tableName), null, null, false);
     HRegion region =
         HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), htd);
     return region;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HTestConst.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HTestConst.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HTestConst.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HTestConst.java Thu Aug  8 04:19:49 2013
@@ -34,6 +34,8 @@ public class HTestConst {
 
   public static final String DEFAULT_TABLE_STR = "MyTestTable";
   public static final byte[] DEFAULT_TABLE_BYTES = Bytes.toBytes(DEFAULT_TABLE_STR);
+  public static final TableName DEFAULT_TABLE =
+      TableName.valueOf(DEFAULT_TABLE_BYTES);
 
   public static final String DEFAULT_CF_STR = "MyDefaultCF";
   public static final byte[] DEFAULT_CF_BYTES = Bytes.toBytes(DEFAULT_CF_STR);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java Thu Aug  8 04:19:49 2013
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@@ -537,11 +536,11 @@ public class MiniHBaseCluster extends HB
    * Call flushCache on all regions of the specified table.
    * @throws IOException
    */
-  public void flushcache(byte [] tableName) throws IOException {
+  public void flushcache(TableName tableName) throws IOException {
     for (JVMClusterUtil.RegionServerThread t:
         this.hbaseCluster.getRegionServers()) {
       for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
-        if(Bytes.equals(r.getTableDesc().getName(), tableName)) {
+        if(r.getTableDesc().getTableName().equals(tableName)) {
           r.flushcache();
         }
       }
@@ -565,11 +564,11 @@ public class MiniHBaseCluster extends HB
    * Call flushCache on all regions of the specified table.
    * @throws IOException
    */
-  public void compact(byte [] tableName, boolean major) throws IOException {
+  public void compact(TableName tableName, boolean major) throws IOException {
     for (JVMClusterUtil.RegionServerThread t:
         this.hbaseCluster.getRegionServers()) {
       for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
-        if(Bytes.equals(r.getTableDesc().getName(), tableName)) {
+        if(r.getTableDesc().getTableName().equals(tableName)) {
           r.compactStores(major);
         }
       }
@@ -600,11 +599,15 @@ public class MiniHBaseCluster extends HB
   }
 
   public List<HRegion> getRegions(byte[] tableName) {
+    return getRegions(TableName.valueOf(tableName));
+  }
+
+  public List<HRegion> getRegions(TableName tableName) {
     List<HRegion> ret = new ArrayList<HRegion>();
     for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
       HRegionServer hrs = rst.getRegionServer();
       for (HRegion region : hrs.getOnlineRegionsLocalContext()) {
-        if (Bytes.equals(region.getTableDesc().getName(), tableName)) {
+        if (region.getTableDesc().getTableName().equals(tableName)) {
           ret.add(region);
         }
       }
@@ -683,12 +686,12 @@ public class MiniHBaseCluster extends HB
     this.hbaseCluster.join();
   }
 
-  public List<HRegion> findRegionsForTable(byte[] tableName) {
+  public List<HRegion> findRegionsForTable(TableName tableName) {
     ArrayList<HRegion> ret = new ArrayList<HRegion>();
     for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
       HRegionServer hrs = rst.getRegionServer();
       for (HRegion region : hrs.getOnlineRegions(tableName)) {
-        if (Bytes.equals(region.getTableDesc().getName(), tableName)) {
+        if (region.getTableDesc().getTableName().equals(tableName)) {
           ret.add(region);
         }
       }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java Thu Aug  8 04:19:49 2013
@@ -31,8 +31,6 @@ import java.util.Map;
 import java.util.Random;
 import java.util.TreeMap;
 import java.util.Arrays;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Semaphore;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.lang.reflect.Constructor;
@@ -60,7 +58,6 @@ import org.apache.hadoop.hbase.filter.Co
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -113,7 +110,8 @@ public class PerformanceEvaluation exten
   private static final int ROWS_PER_GB = ONE_GB / VALUE_LENGTH;
 
   public static final byte[] COMPRESSION = Bytes.toBytes("NONE");
-  public static final byte[] TABLE_NAME = Bytes.toBytes("TestTable");
+  public static final TableName TABLE_NAME =
+      TableName.valueOf("TestTable");
   public static final byte[] FAMILY_NAME = Bytes.toBytes("info");
   public static final byte[] QUALIFIER_NAME = Bytes.toBytes("data");
 
@@ -126,7 +124,7 @@ public class PerformanceEvaluation exten
   private int rowPrefixLength = DEFAULT_ROW_PREFIX_LENGTH;
   private int N = 1;
   private int R = ROWS_PER_GB;
-  private byte[] tableName = TABLE_NAME;
+  private TableName tableName = TABLE_NAME;
   private Compression.Algorithm compression = Compression.Algorithm.NONE;
   private DataBlockEncoding blockEncoding = DataBlockEncoding.NONE;
   private boolean flushCommits = true;
@@ -482,9 +480,9 @@ public class PerformanceEvaluation exten
     HTableDescriptor tableDescriptor = getTableDescriptor();
     if (this.presplitRegions > 0) {
       // presplit requested
-      if (admin.tableExists(tableDescriptor.getName())) {
-        admin.disableTable(tableDescriptor.getName());
-        admin.deleteTable(tableDescriptor.getName());
+      if (admin.tableExists(tableDescriptor.getTableName())) {
+        admin.disableTable(tableDescriptor.getTableName());
+        admin.deleteTable(tableDescriptor.getTableName());
       }
 
       byte[][] splits = getSplits();
@@ -495,13 +493,13 @@ public class PerformanceEvaluation exten
       LOG.info ("Table created with " + this.presplitRegions + " splits");
     }
     else {
-      boolean tableExists = admin.tableExists(tableDescriptor.getName());
+      boolean tableExists = admin.tableExists(tableDescriptor.getTableName());
       if (!tableExists) {
         admin.createTable(tableDescriptor);
         LOG.info("Table " + tableDescriptor + " created");
       }
     }
-    boolean tableExists = admin.tableExists(tableDescriptor.getName());
+    boolean tableExists = admin.tableExists(tableDescriptor.getTableName());
     return tableExists;
   }
 
@@ -563,7 +561,7 @@ public class PerformanceEvaluation exten
     final List<Thread> threads = new ArrayList<Thread>(this.N);
     final long[] timings = new long[this.N];
     final int perClientRows = R/N;
-    final byte[] tableName = this.tableName;
+    final TableName tableName = this.tableName;
     final DataBlockEncoding encoding = this.blockEncoding;
     final boolean flushCommits = this.flushCommits;
     final Compression.Algorithm compression = this.compression;
@@ -746,7 +744,7 @@ public class PerformanceEvaluation exten
     private int perClientRunRows;
     private int totalRows;
     private int numClientThreads;
-    private byte[] tableName;
+    private TableName tableName;
     private boolean flushCommits;
     private boolean writeToWAL = true;
 
@@ -754,7 +752,7 @@ public class PerformanceEvaluation exten
     }
 
     TestOptions(int startRow, int perClientRunRows, int totalRows,
-                int numClientThreads, byte[] tableName,
+                int numClientThreads, TableName tableName,
                 boolean flushCommits, boolean writeToWAL) {
       this.startRow = startRow;
       this.perClientRunRows = perClientRunRows;
@@ -781,7 +779,7 @@ public class PerformanceEvaluation exten
       return numClientThreads;
     }
 
-    public byte[] getTableName() {
+    public TableName getTableName() {
       return tableName;
     }
 
@@ -812,7 +810,7 @@ public class PerformanceEvaluation exten
     protected final int perClientRunRows;
     protected final int totalRows;
     private final Status status;
-    protected byte[] tableName;
+    protected TableName tableName;
     protected HTable table;
     protected volatile Configuration conf;
     protected boolean flushCommits;
@@ -1370,7 +1368,7 @@ public class PerformanceEvaluation exten
 
         final String table = "--table=";
         if (cmd.startsWith(table)) {
-          this.tableName = Bytes.toBytes(cmd.substring(table.length()));
+          this.tableName = TableName.valueOf(cmd.substring(table.length()));
           continue;
         }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCompare.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCompare.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCompare.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCompare.java Thu Aug  8 04:19:49 2013
@@ -33,25 +33,25 @@ public class TestCompare extends TestCas
    * Sort of HRegionInfo.
    */
   public void testHRegionInfo() {
-    HRegionInfo a = new HRegionInfo(Bytes.toBytes("a"), null, null);
-    HRegionInfo b = new HRegionInfo(Bytes.toBytes("b"), null, null);
+    HRegionInfo a = new HRegionInfo(TableName.valueOf("a"), null, null);
+    HRegionInfo b = new HRegionInfo(TableName.valueOf("b"), null, null);
     assertTrue(a.compareTo(b) != 0);
-    HTableDescriptor t = new HTableDescriptor("t");
+    HTableDescriptor t = new HTableDescriptor(TableName.valueOf("t"));
     byte [] midway = Bytes.toBytes("midway");
-    a = new HRegionInfo(t.getName(), null, midway);
-    b = new HRegionInfo(t.getName(), midway, null);
+    a = new HRegionInfo(t.getTableName(), null, midway);
+    b = new HRegionInfo(t.getTableName(), midway, null);
     assertTrue(a.compareTo(b) < 0);
     assertTrue(b.compareTo(a) > 0);
     assertEquals(a, a);
     assertTrue(a.compareTo(a) == 0);
-    a = new HRegionInfo(t.getName(), Bytes.toBytes("a"), Bytes.toBytes("d"));
-    b = new HRegionInfo(t.getName(), Bytes.toBytes("e"), Bytes.toBytes("g"));
+    a = new HRegionInfo(t.getTableName(), Bytes.toBytes("a"), Bytes.toBytes("d"));
+    b = new HRegionInfo(t.getTableName(), Bytes.toBytes("e"), Bytes.toBytes("g"));
     assertTrue(a.compareTo(b) < 0);
-    a = new HRegionInfo(t.getName(), Bytes.toBytes("aaaa"), Bytes.toBytes("dddd"));
-    b = new HRegionInfo(t.getName(), Bytes.toBytes("e"), Bytes.toBytes("g"));
+    a = new HRegionInfo(t.getTableName(), Bytes.toBytes("aaaa"), Bytes.toBytes("dddd"));
+    b = new HRegionInfo(t.getTableName(), Bytes.toBytes("e"), Bytes.toBytes("g"));
     assertTrue(a.compareTo(b) < 0);
-    a = new HRegionInfo(t.getName(), Bytes.toBytes("aaaa"), Bytes.toBytes("dddd"));
-    b = new HRegionInfo(t.getName(), Bytes.toBytes("aaaa"), Bytes.toBytes("eeee"));
+    a = new HRegionInfo(t.getTableName(), Bytes.toBytes("aaaa"), Bytes.toBytes("dddd"));
+    b = new HRegionInfo(t.getTableName(), Bytes.toBytes("aaaa"), Bytes.toBytes("eeee"));
     assertTrue(a.compareTo(b) < 0);
   }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java Thu Aug  8 04:19:49 2013
@@ -74,7 +74,6 @@ public class TestDrainingServer {
     final List<String> families = new ArrayList<String>(1);
     families.add("family");
     TEST_UTIL.createRandomTable("table", families, 1, 0, 0, COUNT_OF_REGIONS, 0);
-
     // Ensure a stable env
     TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, false);
 
@@ -306,7 +305,7 @@ public class TestDrainingServer {
   }
 
   private static boolean isAllRegionsOnline() {
-    return TEST_UTIL.getMiniHBaseCluster().countServedRegions() ==
-        (COUNT_OF_REGIONS + 1 /*catalog regions*/);
+    return TEST_UTIL.getMiniHBaseCluster().countServedRegions() >=
+        (COUNT_OF_REGIONS + 2 /*catalog and namespace regions*/);
   }
 }
\ No newline at end of file

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java Thu Aug  8 04:19:49 2013
@@ -40,7 +40,7 @@ public class TestFSTableDescriptorForceC
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
 
     assertTrue("Should create new table descriptor", fstd.createTableDescriptor(htd, false));
   }
@@ -65,7 +65,7 @@ public class TestFSTableDescriptorForceC
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     fstd.createTableDescriptor(htd, false);
     assertTrue("Should create new table descriptor",
         fstd.createTableDescriptor(htd, true));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java Thu Aug  8 04:19:49 2013
@@ -63,7 +63,7 @@ public class TestHTableDescriptor {
    */
   @Test
   public void testGetSetRemoveCP() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     // simple CP
     String className = BaseRegionObserver.class.getName();
     // add and check that it is present
@@ -80,7 +80,7 @@ public class TestHTableDescriptor {
    */
   @Test
   public void testSetListRemoveCP() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor("testGetSetRemoveCP");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("testGetSetRemoveCP"));
     // simple CP
     String className1 = BaseRegionObserver.class.getName();
     String className2 = SampleRegionWALObserver.class.getName();
@@ -117,7 +117,7 @@ public class TestHTableDescriptor {
    */
   @Test
   public void testRemoveString() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     String key = "Some";
     String value = "value";
     desc.setValue(key, value);
@@ -126,13 +126,18 @@ public class TestHTableDescriptor {
     assertEquals(null, desc.getValue(key));
   }
 
-  String legalTableNames[] = { "foo", "with-dash_under.dot", "_under_start_ok",  };
-  String illegalTableNames[] = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok" };
+  String legalTableNames[] = { "foo", "with-dash_under.dot", "_under_start_ok",
+      "with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02"
+      , "dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
+      "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02"};
+  String illegalTableNames[] = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
+      "-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
+      "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2"};
 
   @Test
   public void testLegalHTableNames() {
     for (String tn : legalTableNames) {
-      HTableDescriptor.isLegalTableName(Bytes.toBytes(tn));
+      TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
     }
   }
 
@@ -140,7 +145,7 @@ public class TestHTableDescriptor {
   public void testIllegalHTableNames() {
     for (String tn : illegalTableNames) {
       try {
-        HTableDescriptor.isLegalTableName(Bytes.toBytes(tn));
+        TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
         fail("invalid tablename " + tn + " should have failed");
       } catch (Exception e) {
         // expected
@@ -151,8 +156,9 @@ public class TestHTableDescriptor {
   @Test
   public void testLegalHTableNamesRegex() {
     for (String tn : legalTableNames) {
-      LOG.info("Testing: '" + tn + "'");
-      assertTrue(Pattern.matches(HTableDescriptor.VALID_USER_TABLE_REGEX, tn));
+      TableName tName = TableName.valueOf(tn);
+      assertTrue("Testing: '" + tn + "'", Pattern.matches(TableName.VALID_USER_TABLE_REGEX,
+          tName.getNameAsString()));
     }
   }
 
@@ -160,7 +166,7 @@ public class TestHTableDescriptor {
   public void testIllegalHTableNamesRegex() {
     for (String tn : illegalTableNames) {
       LOG.info("Testing: '" + tn + "'");
-      assertFalse(Pattern.matches(HTableDescriptor.VALID_USER_TABLE_REGEX, tn));
+      assertFalse(Pattern.matches(TableName.VALID_USER_TABLE_REGEX, tn));
     }
   }
 
@@ -169,7 +175,7 @@ public class TestHTableDescriptor {
    */
   @Test
   public void testGetMaxFileSize() {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     assertEquals(-1, desc.getMaxFileSize());
     desc.setMaxFileSize(1111L);
     assertEquals(1111L, desc.getMaxFileSize());
@@ -180,7 +186,7 @@ public class TestHTableDescriptor {
    */
   @Test
   public void testGetMemStoreFlushSize() {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     assertEquals(-1, desc.getMemStoreFlushSize());
     desc.setMemStoreFlushSize(1111L);
     assertEquals(1111L, desc.getMemStoreFlushSize());
@@ -191,7 +197,7 @@ public class TestHTableDescriptor {
    */
   @Test
   public void testAddGetRemoveConfiguration() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor("table");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("table"));
     String key = "Some";
     String value = "value";
     desc.setConfiguration(key, value);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java Thu Aug  8 04:19:49 2013
@@ -190,7 +190,8 @@ public class TestIOFencing {
   }
 
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private final static byte[] TABLE_NAME = Bytes.toBytes("tabletest");
+  private final static TableName TABLE_NAME =
+      TableName.valueOf("tabletest");
   private final static byte[] FAMILY = Bytes.toBytes("family");
   private static final int FIRST_BATCH_COUNT = 4000;
   private static final int SECOND_BATCH_COUNT = FIRST_BATCH_COUNT;
@@ -263,7 +264,7 @@ public class TestIOFencing {
       assertTrue(compactingRegion.countStoreFiles() > 1);
       final byte REGION_NAME[] = compactingRegion.getRegionName();
       LOG.info("Asking for compaction");
-      admin.majorCompact(TABLE_NAME);
+      admin.majorCompact(TABLE_NAME.getName());
       LOG.info("Waiting for compaction to be about to start");
       compactingRegion.waitForCompactionToBlock();
       LOG.info("Starting a new server");
@@ -297,7 +298,7 @@ public class TestIOFencing {
       // If we survive the split keep going...
       // Now we make sure that the region isn't totally confused.  Load up more rows.
       TEST_UTIL.loadNumericRows(table, FAMILY, FIRST_BATCH_COUNT, FIRST_BATCH_COUNT + SECOND_BATCH_COUNT);
-      admin.majorCompact(TABLE_NAME);
+      admin.majorCompact(TABLE_NAME.getName());
       startWaitTime = System.currentTimeMillis();
       while (newRegion.compactCount == 0) {
         Thread.sleep(1000);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java Thu Aug  8 04:19:49 2013
@@ -65,7 +65,7 @@ public class TestInfoServers {
   @Test
   public void testInfoServersRedirect() throws Exception {
     // give the cluster time to start up
-    new HTable(UTIL.getConfiguration(), ".META.").close();
+    new HTable(UTIL.getConfiguration(), TableName.META_TABLE_NAME).close();
     int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
     assertContainsContent(new URL("http://localhost:" + port +
         "/index.html"), "master-status");
@@ -85,7 +85,7 @@ public class TestInfoServers {
   @Test
   public void testInfoServersStatusPages() throws Exception {
     // give the cluster time to start up
-    new HTable(UTIL.getConfiguration(), ".META.").close();
+    new HTable(UTIL.getConfiguration(), TableName.META_TABLE_NAME).close();
     int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
     assertContainsContent(new URL("http://localhost:" + port +
         "/master-status"), "META");

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java Thu Aug  8 04:19:49 2013
@@ -92,12 +92,12 @@ public class TestMultiVersions {
    */
   @Test
   public void testTimestamps() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor("testTimestamps");
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("testTimestamps"));
     HColumnDescriptor hcd = new HColumnDescriptor(TimestampTestBase.FAMILY_NAME);
     hcd.setMaxVersions(3);
     desc.addFamily(hcd);
     this.admin.createTable(desc);
-    HTable table = new HTable(UTIL.getConfiguration(), desc.getName());
+    HTable table = new HTable(UTIL.getConfiguration(), desc.getTableName());
     // TODO: Remove these deprecated classes or pull them in here if this is
     // only test using them.
     Incommon incommon = new HTableIncommon(table);
@@ -133,7 +133,7 @@ public class TestMultiVersions {
     final byte [] value2 = Bytes.toBytes("value2");
     final long timestamp1 = 100L;
     final long timestamp2 = 200L;
-    final HTableDescriptor desc = new HTableDescriptor(tableName);
+    final HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     HColumnDescriptor hcd = new HColumnDescriptor(contents);
     hcd.setMaxVersions(3);
     desc.addFamily(hcd);
@@ -194,7 +194,7 @@ public class TestMultiVersions {
   @Test
   public void testScanMultipleVersions() throws Exception {
     final byte [] tableName = Bytes.toBytes("testScanMultipleVersions");
-    final HTableDescriptor desc = new HTableDescriptor(tableName);
+    final HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     final byte [][] rows = new byte[][] {
       Bytes.toBytes("row_0200"),

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java?rev=1511577&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java Thu Aug  8 04:19:49 2013
@@ -0,0 +1,309 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
+@Category(MediumTests.class)
+public class TestNamespace {
+  protected static final Log LOG = LogFactory.getLog(TestNamespace.class);
+  private static HMaster master;
+  protected final static int NUM_SLAVES_BASE = 4;
+  private static HBaseTestingUtility TEST_UTIL;
+  protected static HBaseAdmin admin;
+  protected static HBaseCluster cluster;
+  private static ZKNamespaceManager zkNamespaceManager;
+  private String prefix = "TestNamespace";
+
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL.getConfiguration().setInt("hbase.namespacejanitor.interval", 5000);
+    TEST_UTIL.startMiniCluster(NUM_SLAVES_BASE);
+    admin = TEST_UTIL.getHBaseAdmin();
+    cluster = TEST_UTIL.getHBaseCluster();
+    master = ((MiniHBaseCluster)cluster).getMaster();
+    zkNamespaceManager =
+        new ZKNamespaceManager(master.getZooKeeperWatcher());
+    zkNamespaceManager.start();
+    LOG.info("Done initializing cluster");
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void beforeMethod() throws IOException {
+    for (HTableDescriptor desc : admin.listTables(prefix+".*")) {
+      admin.disableTable(desc.getTableName());
+      admin.deleteTable(desc.getTableName());
+    }
+    for (NamespaceDescriptor ns : admin.listNamespaceDescriptors()) {
+      if (ns.getName().startsWith(prefix)) {
+        admin.deleteNamespace(ns.getName());
+      }
+    }
+  }
+
+  @Test
+  public void verifyReservedNS() throws IOException {
+    //verify existence of reserved namespaces
+    NamespaceDescriptor ns =
+        admin.getNamespaceDescriptor(NamespaceDescriptor.DEFAULT_NAMESPACE.getName());
+    assertNotNull(ns);
+    assertEquals(ns.getName(), NamespaceDescriptor.DEFAULT_NAMESPACE.getName());
+    assertNotNull(zkNamespaceManager.get(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR));
+
+    ns = admin.getNamespaceDescriptor(NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
+    assertNotNull(ns);
+    assertEquals(ns.getName(), NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
+    assertNotNull(zkNamespaceManager.get(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR));
+
+    assertEquals(2, admin.listNamespaceDescriptors().length);
+
+    //verify existence of system tables
+    Set<TableName> systemTables = Sets.newHashSet(
+        TableName.META_TABLE_NAME,
+        TableName.NAMESPACE_TABLE_NAME);
+    HTableDescriptor[] descs =
+        admin.getTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
+    assertEquals(systemTables.size(), descs.length);
+    for (HTableDescriptor desc : descs) {
+      assertTrue(systemTables.contains(desc.getTableName()));
+    }
+    //verify system tables aren't listed
+    assertEquals(0, admin.listTables().length);
+    
+    //Try creating default and system namespaces. 
+    boolean exceptionCaught = false;
+    try {
+      admin.createNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE);
+    } catch (IOException exp) {
+      LOG.warn(exp);
+      exceptionCaught = true;
+    } finally {
+      assertTrue(exceptionCaught);
+    }
+
+    exceptionCaught = false;
+    try {
+      admin.createNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
+    } catch (IOException exp) {
+      LOG.warn(exp);
+      exceptionCaught = true;
+    } finally {
+      assertTrue(exceptionCaught);
+    }
+  }
+  
+  @Test
+  public void testDeleteReservedNS() throws Exception {
+    boolean exceptionCaught = false;
+    try {
+      admin.deleteNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR);
+    } catch (IOException exp) {
+      LOG.warn(exp);
+      exceptionCaught = true;
+    } finally {
+      assertTrue(exceptionCaught);
+    }
+
+    try {
+      admin.deleteNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR);
+    } catch (IOException exp) {
+      LOG.warn(exp);
+      exceptionCaught = true;
+    } finally {
+      assertTrue(exceptionCaught);
+    }
+  }
+
+  @Test
+  public void createRemoveTest() throws Exception {
+    String testName = "createRemoveTest";
+    String nsName = prefix+"_"+testName;
+    LOG.info(testName);
+
+    //create namespace and verify
+    admin.createNamespace(NamespaceDescriptor.create(nsName).build());
+    assertEquals(3, admin.listNamespaceDescriptors().length);
+    TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return zkNamespaceManager.list().size() == 3;
+      }
+    });
+    assertNotNull(zkNamespaceManager.get(nsName));
+    //remove namespace and verify
+    admin.deleteNamespace(nsName);
+    assertEquals(2, admin.listNamespaceDescriptors().length);
+    assertEquals(2, zkNamespaceManager.list().size());
+    assertNull(zkNamespaceManager.get(nsName));
+  }
+
+  @Test
+  public void createDoubleTest() throws IOException, InterruptedException {
+    String testName = "createDoubleTest";
+    String nsName = prefix+"_"+testName;
+    LOG.info(testName);
+
+    byte[] tableName = Bytes.toBytes("my_table");
+    byte[] tableNameFoo = Bytes.toBytes(nsName+".my_table");
+    //create namespace and verify
+    admin.createNamespace(NamespaceDescriptor.create(nsName).build());
+    TEST_UTIL.createTable(tableName, Bytes.toBytes(nsName));
+    TEST_UTIL.createTable(tableNameFoo,Bytes.toBytes(nsName));
+    assertEquals(2, admin.listTables().length);
+    assertNotNull(admin
+        .getTableDescriptor(tableName));
+    assertNotNull(admin
+        .getTableDescriptor(tableNameFoo));
+    //remove namespace and verify
+    admin.disableTable(tableName);
+    admin.deleteTable(tableName);
+    assertEquals(1, admin.listTables().length);
+  }
+
+  @Test
+  public void createTableTest() throws IOException, InterruptedException {
+    String testName = "createTableTest";
+    String nsName = prefix+"_"+testName;
+    LOG.info(testName);
+
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(nsName+":my_table"));
+    HColumnDescriptor colDesc = new HColumnDescriptor("my_cf");
+    desc.addFamily(colDesc);
+    try {
+      admin.createTable(desc);
+      fail("Expected no namespace constraint exception");
+    } catch (ConstraintException ex) {
+    }
+    //create table and in new namespace
+    admin.createNamespace(NamespaceDescriptor.create(nsName).build());
+    admin.createTable(desc);
+    TEST_UTIL.waitTableAvailable(desc.getTableName().getName(), 10000);
+    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
+    assertTrue(fs.exists(
+        new Path(master.getMasterFileSystem().getRootDir(),
+            new Path(HConstants.BASE_NAMESPACE_DIR,
+                new Path(nsName, desc.getTableName().getQualifierAsString())))));
+    assertEquals(1, admin.listTables().length);
+
+    //verify non-empty namespace can't be removed
+    try {
+      admin.deleteNamespace(nsName);
+      fail("Expected non-empty namespace constraint exception");
+    } catch (Exception ex) {
+      LOG.info("Caught expected exception: " + ex);
+    }
+
+    //sanity check try to write and read from table
+    HTable table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
+    Put p = new Put(Bytes.toBytes("row1"));
+    p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1"));
+    table.put(p);
+    //flush and read from disk to make sure directory changes are working
+    admin.flush(desc.getTableName().getName());
+    Get g = new Get(Bytes.toBytes("row1"));
+    assertTrue(table.exists(g));
+
+    //normal case of removing namespace
+    TEST_UTIL.deleteTable(desc.getTableName());
+    admin.deleteNamespace(nsName);
+  }
+
+  @Test
+  public void createTableInDefaultNamespace() throws Exception {
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf("default_table"));
+    HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
+    desc.addFamily(colDesc);
+    admin.createTable(desc);
+    assertTrue(admin.listTables().length == 1);  
+    admin.disableTable(desc.getTableName());
+    admin.deleteTable(desc.getTableName());
+  }
+
+  @Test
+  public void createTableInSystemNamespace() throws Exception {
+    String tableName = "hbase:createTableInSystemNamespace";
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+    HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
+    desc.addFamily(colDesc);
+    admin.createTable(desc);
+    assertEquals(0, admin.listTables().length);
+    assertTrue(admin.tableExists(Bytes.toBytes(tableName)));
+    admin.disableTable(desc.getTableName());
+    admin.deleteTable(desc.getTableName());
+  }
+
+  @Test
+  public void testNamespaceJanitor() throws Exception {
+    FileSystem fs = TEST_UTIL.getTestFileSystem();
+
+    int fsCount = fs.listStatus(new Path(FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
+        HConstants.BASE_NAMESPACE_DIR)).length;
+    Path fakeNSPath =
+        FSUtils.getNamespaceDir(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), "foo");
+    assertTrue(fs.mkdirs(fakeNSPath));
+
+    String fakeZnode = ZKUtil.joinZNode(ZooKeeperWatcher.namespaceZNode, "foo");
+    int zkCount = ZKUtil.listChildrenNoWatch(TEST_UTIL.getZooKeeperWatcher(),
+        ZooKeeperWatcher.namespaceZNode).size();
+    ZKUtil.createWithParents(TEST_UTIL.getZooKeeperWatcher(), fakeZnode);
+    Thread.sleep(10000);
+
+    //verify namespace count is the same and orphan is removed
+    assertFalse(fs.exists(fakeNSPath));
+    assertEquals(fsCount, fs.listStatus(new Path(FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
+            HConstants.BASE_NAMESPACE_DIR)).length);
+
+    assertEquals(-1, ZKUtil.checkExists(TEST_UTIL.getZooKeeperWatcher(), fakeZnode));
+    assertEquals(zkCount,
+        ZKUtil.listChildrenNoWatch(TEST_UTIL.getZooKeeperWatcher(),
+            ZooKeeperWatcher.namespaceZNode).size());
+  }
+
+}

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java Thu Aug  8 04:19:49 2013
@@ -81,7 +81,7 @@ public class TestRegionRebalancing {
   public void before() throws Exception {
     UTIL.getConfiguration().set("hbase.master.loadbalancer.class", this.balancerName);
     UTIL.startMiniCluster(1);
-    this.desc = new HTableDescriptor("test");
+    this.desc = new HTableDescriptor(TableName.valueOf("test"));
     this.desc.addFamily(new HColumnDescriptor(FAMILY_NAME));
   }
 
@@ -97,7 +97,7 @@ public class TestRegionRebalancing {
     HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration());
     admin.createTable(this.desc, Arrays.copyOfRange(HBaseTestingUtility.KEYS,
         1, HBaseTestingUtility.KEYS.length));
-    this.table = new HTable(UTIL.getConfiguration(), this.desc.getName());
+    this.table = new HTable(UTIL.getConfiguration(), this.desc.getTableName());
     CatalogTracker ct = new CatalogTracker(UTIL.getConfiguration());
     ct.start();
     try {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java Thu Aug  8 04:19:49 2013
@@ -97,7 +97,7 @@ public class TestSerialization {
     byte [] mb = Writables.getBytes(htd);
     HTableDescriptor deserializedHtd =
       (HTableDescriptor)Writables.getWritable(mb, new HTableDescriptor());
-    assertEquals(htd.getNameAsString(), deserializedHtd.getNameAsString());
+    assertEquals(htd.getTableName(), deserializedHtd.getTableName());
   }
 
   /**
@@ -136,12 +136,12 @@ public class TestSerialization {
   }
 
   private HRegionInfo createRandomRegion(final String name) {
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     String [] families = new String [] {"info", "anchor"};
     for (int i = 0; i < families.length; i++) {
       htd.addFamily(new HColumnDescriptor(families[i]));
     }
-    return new HRegionInfo(htd.getName(), HConstants.EMPTY_START_ROW,
+    return new HRegionInfo(htd.getTableName(), HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW);
   }
 
@@ -543,7 +543,7 @@ public class TestSerialization {
    */
   protected HTableDescriptor createTableDescriptor(final String name,
       final int versions) {
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     htd.addFamily(new HColumnDescriptor(fam1)
         .setMaxVersions(versions)
         .setBlockCacheEnabled(false)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Thu Aug  8 04:19:49 2013
@@ -232,8 +232,8 @@ public class TestZooKeeper {
    * @throws Exception
    */
   private void testSanity(final String testName) throws Exception{
-    String tableName = testName + "." + System.currentTimeMillis();
-    HTableDescriptor desc = new HTableDescriptor(tableName);
+    String tableName = testName + "_" + System.currentTimeMillis();
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     HColumnDescriptor family = new HColumnDescriptor("fam");
     desc.addFamily(family);
     LOG.info("Creating table " + tableName);
@@ -258,10 +258,10 @@ public class TestZooKeeper {
   public void testMultipleZK()
   throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException {
     HTable localMeta =
-      new HTable(new Configuration(TEST_UTIL.getConfiguration()), HConstants.META_TABLE_NAME);
+      new HTable(new Configuration(TEST_UTIL.getConfiguration()), TableName.META_TABLE_NAME);
     Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
     otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
-    HTable ipMeta = new HTable(otherConf, HConstants.META_TABLE_NAME);
+    HTable ipMeta = new HTable(otherConf, TableName.META_TABLE_NAME);
 
     // dummy, just to open the connection
     final byte [] row = new byte [] {'r'};
@@ -483,7 +483,7 @@ public class TestZooKeeper {
         Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
         Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i"), Bytes.toBytes("j") };
       String tableName = "testRegionAssignmentAfterMasterRecoveryDueToZKExpiry";
-      admin.createTable(new HTableDescriptor(tableName), SPLIT_KEYS);
+      admin.createTable(new HTableDescriptor(TableName.valueOf(tableName)), SPLIT_KEYS);
       ZooKeeperWatcher zooKeeperWatcher = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
       ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
       m.getZooKeeperWatcher().close();
@@ -519,7 +519,7 @@ public class TestZooKeeper {
         Bytes.toBytes("3"), Bytes.toBytes("4"), Bytes.toBytes("5") };
 
       String tableName = "testLogSplittingAfterMasterRecoveryDueToZKExpiry";
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
       HColumnDescriptor hcd = new HColumnDescriptor("col");
       htd.addFamily(hcd);
       admin.createTable(htd, SPLIT_KEYS);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java Thu Aug  8 04:19:49 2013
@@ -33,6 +33,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.HConstants;
 import org.apache.hadoop.hbase.MediumTests;
@@ -112,7 +113,8 @@ public class TestHFileArchiving {
 
   @Test
   public void testRemovesRegionDirOnArchive() throws Exception {
-    byte[] TABLE_NAME = Bytes.toBytes("testRemovesRegionDirOnArchive");
+    TableName TABLE_NAME =
+        TableName.valueOf("testRemovesRegionDirOnArchive");
     UTIL.createTable(TABLE_NAME, TEST_FAM);
 
     final HBaseAdmin admin = UTIL.getHBaseAdmin();
@@ -162,7 +164,8 @@ public class TestHFileArchiving {
    */
   @Test
   public void testDeleteRegionWithNoStoreFiles() throws Exception {
-    byte[] TABLE_NAME = Bytes.toBytes("testDeleteRegionWithNoStoreFiles");
+    TableName TABLE_NAME =
+        TableName.valueOf("testDeleteRegionWithNoStoreFiles");
     UTIL.createTable(TABLE_NAME, TEST_FAM);
 
     // get the current store files for the region
@@ -210,7 +213,8 @@ public class TestHFileArchiving {
 
   @Test
   public void testArchiveOnTableDelete() throws Exception {
-    byte[] TABLE_NAME = Bytes.toBytes("testArchiveOnTableDelete");
+    TableName TABLE_NAME =
+        TableName.valueOf("testArchiveOnTableDelete");
     UTIL.createTable(TABLE_NAME, TEST_FAM);
 
     List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(TABLE_NAME);
@@ -288,7 +292,8 @@ public class TestHFileArchiving {
    */
   @Test
   public void testArchiveOnTableFamilyDelete() throws Exception {
-    byte[] TABLE_NAME = Bytes.toBytes("testArchiveOnTableFamilyDelete");
+    TableName TABLE_NAME =
+        TableName.valueOf("testArchiveOnTableFamilyDelete");
     UTIL.createTable(TABLE_NAME, TEST_FAM);
 
     List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(TABLE_NAME);
@@ -342,7 +347,8 @@ public class TestHFileArchiving {
     FileSystem fs = UTIL.getTestFileSystem();
 
     Path archiveDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
-    Path regionDir = new Path("table", "abcdef");
+    Path regionDir = new Path(FSUtils.getTableDir(new Path("./"),
+        TableName.valueOf("table")), "abcdef");
     Path familyDir = new Path(regionDir, "cf");
 
     Path sourceRegionDir = new Path(rootDir, regionDir);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java Thu Aug  8 04:19:49 2013
@@ -30,6 +30,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.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -326,10 +327,10 @@ public class TestCatalogTracker {
     // Make it so we return any old location when asked.
     final HRegionLocation anyLocation =
       new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, SN);
-    Mockito.when(connection.getRegionLocation((byte[]) Mockito.any(),
+    Mockito.when(connection.getRegionLocation((TableName) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(anyLocation);
-    Mockito.when(connection.locateRegion((byte[]) Mockito.any(),
+    Mockito.when(connection.locateRegion((TableName) Mockito.any(),
         (byte[]) Mockito.any())).
       thenReturn(anyLocation);
     if (admin != null) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java Thu Aug  8 04:19:49 2013
@@ -36,6 +36,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.migration.NamespaceUpgrade;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -51,6 +53,7 @@ import org.apache.hadoop.hbase.client.Du
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.util.ToolRunner;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -118,6 +121,12 @@ public class TestMetaMigrationConverting
       new Path(hbaseRootDir, ".META.").toString()});
     // See whats in minihdfs.
     doFsCommand(shell, new String [] {"-lsr", "/"});
+
+    //upgrade to namespace as well
+    Configuration toolConf = TEST_UTIL.getConfiguration();
+    conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDefaultRootDirPath().toString());
+    ToolRunner.run(toolConf, new NamespaceUpgrade(), new String[]{"--upgrade"});
+
     TEST_UTIL.startMiniHBaseCluster(1, 1);
     // Assert we are running against the copied-up filesystem.  The copied-up
     // rootdir should have had a table named 'TestTable' in it.  Assert it
@@ -182,7 +191,7 @@ public class TestMetaMigrationConverting
   public void testMetaMigration() throws Exception {
     LOG.info("Starting testMetaMigration");
     final byte [] FAMILY = Bytes.toBytes("family");
-    HTableDescriptor htd = new HTableDescriptor("testMetaMigration");
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testMetaMigration"));
     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
       htd.addFamily(hcd);
     Configuration conf = TEST_UTIL.getConfiguration();
@@ -190,7 +199,9 @@ public class TestMetaMigrationConverting
         HConstants.EMPTY_START_ROW,
         Bytes.toBytes("region_a"),
         Bytes.toBytes("region_b")};
-    createMultiRegionsWithWritableSerialization(conf, htd.getName(), regionNames);
+    createMultiRegionsWithWritableSerialization(conf,
+        htd.getTableName().getName(),
+        regionNames);
     CatalogTracker ct =
       TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker();
     // Erase the current version of root meta for this test.
@@ -228,14 +239,16 @@ public class TestMetaMigrationConverting
   @Test
   public void testMasterCrashDuringMetaMigration() throws Exception {
     final byte[] FAMILY = Bytes.toBytes("family");
-    HTableDescriptor htd = new HTableDescriptor("testMasterCrashDuringMetaMigration");
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf
+        ("testMasterCrashDuringMetaMigration"));
     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
       htd.addFamily(hcd);
     Configuration conf = TEST_UTIL.getConfiguration();
     // Create 10 New regions.
-    createMultiRegionsWithPBSerialization(conf, htd.getName(), 10);
+    createMultiRegionsWithPBSerialization(conf, htd.getTableName().getName(), 10);
     // Create 10 Legacy regions.
-    createMultiRegionsWithWritableSerialization(conf, htd.getName(), 10);
+    createMultiRegionsWithWritableSerialization(conf,
+        htd.getTableName().getName(), 10);
     CatalogTracker ct =
       TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker();
     // Erase the current version of root meta for this test.
@@ -315,14 +328,21 @@ public class TestMetaMigrationConverting
     return createMultiRegionsWithWritableSerialization(c, tableName, regionStartKeys);
   }
 
+  public int createMultiRegionsWithWritableSerialization(final Configuration c,
+      final byte[] tableName, byte [][] startKeys)
+  throws IOException {
+    return createMultiRegionsWithWritableSerialization(c,
+        TableName.valueOf(tableName), startKeys);
+  }
+
   /**
    * Inserts multiple regions into META using Writable serialization instead of PB
    */
   public int createMultiRegionsWithWritableSerialization(final Configuration c,
-      final byte[] tableName, byte [][] startKeys)
+      final TableName tableName, byte [][] startKeys)
   throws IOException {
     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
-    HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
+    HTable meta = new HTable(c, TableName.META_TABLE_NAME);
 
     List<HRegionInfo> newRegions
         = new ArrayList<HRegionInfo>(startKeys.length);
@@ -388,8 +408,15 @@ public class TestMetaMigrationConverting
    */
   int createMultiRegionsWithPBSerialization(final Configuration c, final byte[] tableName,
       byte [][] startKeys) throws IOException {
+    return createMultiRegionsWithPBSerialization(c,
+        TableName.valueOf(tableName), startKeys);
+  }
+
+  int createMultiRegionsWithPBSerialization(final Configuration c,
+      final TableName tableName,
+      byte [][] startKeys) throws IOException {
     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
-    HTable meta = new HTable(c, HConstants.META_TABLE_NAME);
+    HTable meta = new HTable(c, TableName.META_TABLE_NAME);
 
     List<HRegionInfo> newRegions
         = new ArrayList<HRegionInfo>(startKeys.length);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java Thu Aug  8 04:19:49 2013
@@ -31,6 +31,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.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -98,14 +99,14 @@ public class TestMetaReaderEditor {
    */
   @Test public void testRetrying()
   throws IOException, InterruptedException {
-    final String name = "testRetrying";
+    final TableName name =
+        TableName.valueOf("testRetrying");
     LOG.info("Started " + name);
-    final byte [] nameBytes = Bytes.toBytes(name);
-    HTable t = UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
+    HTable t = UTIL.createTable(name, HConstants.CATALOG_FAMILY);
     int regionCount = UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
     // Test it works getting a region from just made user table.
     final List<HRegionInfo> regions =
-      testGettingTableRegions(CT, nameBytes, regionCount);
+      testGettingTableRegions(CT, name, regionCount);
     MetaTask reader = new MetaTask(CT, "reader") {
       @Override
       void metaTask() throws Throwable {
@@ -218,24 +219,24 @@ public class TestMetaReaderEditor {
   @Test public void testGetRegionsCatalogTables()
   throws IOException, InterruptedException {
     List<HRegionInfo> regions =
-      MetaReader.getTableRegions(CT, HConstants.META_TABLE_NAME);
+      MetaReader.getTableRegions(CT, TableName.META_TABLE_NAME);
     assertTrue(regions.size() >= 1);
     assertTrue(MetaReader.getTableRegionsAndLocations(CT,
-      Bytes.toString(HConstants.META_TABLE_NAME)).size() >= 1);
+      TableName.META_TABLE_NAME).size() >= 1);
   }
 
   @Test public void testTableExists() throws IOException {
-    final String name = "testTableExists";
-    final byte [] nameBytes = Bytes.toBytes(name);
+    final TableName name =
+        TableName.valueOf("testTableExists");
     assertFalse(MetaReader.tableExists(CT, name));
-    UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
+    UTIL.createTable(name, HConstants.CATALOG_FAMILY);
     assertTrue(MetaReader.tableExists(CT, name));
     HBaseAdmin admin = UTIL.getHBaseAdmin();
     admin.disableTable(name);
     admin.deleteTable(name);
     assertFalse(MetaReader.tableExists(CT, name));
     assertTrue(MetaReader.tableExists(CT,
-      Bytes.toString(HConstants.META_TABLE_NAME)));
+      TableName.META_TABLE_NAME));
   }
 
   @Test public void testGetRegion() throws IOException, InterruptedException {
@@ -251,7 +252,8 @@ public class TestMetaReaderEditor {
   // Test for the optimization made in HBASE-3650
   @Test public void testScanMetaForTable()
   throws IOException, InterruptedException {
-    final String name = "testScanMetaForTable";
+    final TableName name =
+        TableName.valueOf("testScanMetaForTable");
     LOG.info("Started " + name);
 
     /** Create 2 tables
@@ -259,21 +261,22 @@ public class TestMetaReaderEditor {
      - testScanMetaForTablf
     **/
 
-    UTIL.createTable(Bytes.toBytes(name), HConstants.CATALOG_FAMILY);
+    UTIL.createTable(name, HConstants.CATALOG_FAMILY);
     // name that is +1 greater than the first one (e+1=f)
-    byte[] greaterName = Bytes.toBytes("testScanMetaForTablf");
+    TableName greaterName =
+        TableName.valueOf("testScanMetaForTablf");
     UTIL.createTable(greaterName, HConstants.CATALOG_FAMILY);
 
     // Now make sure we only get the regions from 1 of the tables at a time
 
-    assertEquals(1, MetaReader.getTableRegions(CT, Bytes.toBytes(name)).size());
+    assertEquals(1, MetaReader.getTableRegions(CT, name).size());
     assertEquals(1, MetaReader.getTableRegions(CT, greaterName).size());
   }
 
   private static List<HRegionInfo> testGettingTableRegions(final CatalogTracker ct,
-      final byte [] nameBytes, final int regionCount)
+      final TableName name, final int regionCount)
   throws IOException, InterruptedException {
-    List<HRegionInfo> regions = MetaReader.getTableRegions(ct, nameBytes);
+    List<HRegionInfo> regions = MetaReader.getTableRegions(ct, name);
     assertEquals(regionCount, regions.size());
     Pair<HRegionInfo, ServerName> pair =
       MetaReader.getRegion(ct, regions.get(0).getRegionName());

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java Thu Aug  8 04:19:49 2013
@@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.client.HC
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultCellMeta;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
@@ -197,9 +197,9 @@ public class TestMetaReaderEditorNoClust
       // The ugly format below comes of 'Important gotcha on spying real objects!' from
       // http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html
       Mockito.doReturn(anyLocation).
-        when(connection).locateRegion((byte[]) Mockito.any(), (byte[]) Mockito.any());
+        when(connection).locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any());
       Mockito.doReturn(anyLocation).
-        when(connection).getRegionLocation((byte[]) Mockito.any(),
+        when(connection).getRegionLocation((TableName) Mockito.any(),
           (byte[]) Mockito.any(), Mockito.anyBoolean());
 
       // Now shove our HRI implementation into the spied-upon connection.