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 [3/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/apa...

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Thu Aug  8 04:19:49 2013
@@ -48,6 +48,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Chore;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
@@ -72,6 +73,22 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyNamespaceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateNamespaceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteNamespaceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetNamespaceDescriptorResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetNamespaceDescriptorRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos
+    .ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos
+    .ListNamespaceDescriptorsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos
+    .GetTableDescriptorsByNamespaceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos
+    .GetTableDescriptorsByNamespaceRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
@@ -442,7 +459,7 @@ public class HConnectionManager {
    * @return Number of cached regions for the table.
    * @throws ZooKeeperConnectionException
    */
-  static int getCachedRegionCount(Configuration conf, final byte[] tableName)
+  static int getCachedRegionCount(Configuration conf, final TableName tableName)
   throws IOException {
     return execute(new HConnectable<Integer>(conf) {
       @Override
@@ -458,7 +475,9 @@ public class HConnectionManager {
    * @return true if the region where the table and row reside is cached.
    * @throws ZooKeeperConnectionException
    */
-  static boolean isRegionCached(Configuration conf, final byte[] tableName, final byte[] row)
+  static boolean isRegionCached(Configuration conf,
+                                final TableName tableName,
+                                final byte[] row)
   throws IOException {
     return execute(new HConnectable<Boolean>(conf) {
       @Override
@@ -542,11 +561,11 @@ public class HConnectionManager {
     private RpcClient rpcClient;
 
     /**
-     * Map of table to table {@link HRegionLocation}s.  The table key is made
-     * by doing a {@link Bytes#mapKey(byte[])} of the table's name.
-     */
-    private final Map<Integer, SoftValueSortedMap<byte [], HRegionLocation>> cachedRegionLocations =
-      new HashMap<Integer, SoftValueSortedMap<byte [], HRegionLocation>>();
+      * Map of table to table {@link HRegionLocation}s.
+      */
+    private final Map<TableName, SoftValueSortedMap<byte[], HRegionLocation>>
+        cachedRegionLocations =
+      new HashMap<TableName, SoftValueSortedMap<byte[], HRegionLocation>>();
 
     // The presence of a server in the map implies it's likely that there is an
     // entry in cachedRegionLocations that map to this server; but the absence
@@ -792,24 +811,41 @@ public class HConnectionManager {
     }
 
     @Override
-    public HRegionLocation getRegionLocation(final byte [] name,
+    public HRegionLocation getRegionLocation(final TableName tableName,
         final byte [] row, boolean reload)
     throws IOException {
-      return reload? relocateRegion(name, row): locateRegion(name, row);
+      return reload? relocateRegion(tableName, row): locateRegion(tableName, row);
     }
 
     @Override
-    public boolean isTableEnabled(byte[] tableName) throws IOException {
+    public HRegionLocation getRegionLocation(final byte[] tableName,
+        final byte [] row, boolean reload)
+    throws IOException {
+      return getRegionLocation(TableName.valueOf(tableName), row, reload);
+    }
+
+    @Override
+    public boolean isTableEnabled(TableName tableName) throws IOException {
       return this.registry.isTableOnlineState(tableName, true);
     }
 
     @Override
-    public boolean isTableDisabled(byte[] tableName) throws IOException {
+    public boolean isTableEnabled(byte[] tableName) throws IOException {
+      return isTableEnabled(TableName.valueOf(tableName));
+    }
+
+    @Override
+    public boolean isTableDisabled(TableName tableName) throws IOException {
       return this.registry.isTableOnlineState(tableName, false);
     }
 
     @Override
-    public boolean isTableAvailable(final byte[] tableName) throws IOException {
+    public boolean isTableDisabled(byte[] tableName) throws IOException {
+      return isTableDisabled(TableName.valueOf(tableName));
+    }
+
+    @Override
+    public boolean isTableAvailable(final TableName tableName) throws IOException {
       final AtomicBoolean available = new AtomicBoolean(true);
       final AtomicInteger regionCount = new AtomicInteger(0);
       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
@@ -817,14 +853,15 @@ public class HConnectionManager {
         public boolean processRow(Result row) throws IOException {
           HRegionInfo info = MetaScanner.getHRegionInfo(row);
           if (info != null) {
-            if (Bytes.compareTo(tableName, info.getTableName()) == 0) {
+            if (tableName.equals(info.getTableName())) {
               ServerName server = HRegionInfo.getServerName(row);
               if (server == null) {
                 available.set(false);
                 return false;
               }
               regionCount.incrementAndGet();
-            } else if (Bytes.compareTo(tableName, info.getTableName()) < 0) {
+            } else if (tableName.compareTo(
+                info.getTableName()) < 0) {
               // Return if we are done with the current table
               return false;
             }
@@ -837,7 +874,12 @@ public class HConnectionManager {
     }
 
     @Override
-    public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys)
+    public boolean isTableAvailable(final byte[] tableName) throws IOException {
+      return isTableAvailable(TableName.valueOf(tableName));
+    }
+
+    @Override
+    public boolean isTableAvailable(final TableName tableName, final byte[][] splitKeys)
         throws IOException {
       final AtomicBoolean available = new AtomicBoolean(true);
       final AtomicInteger regionCount = new AtomicInteger(0);
@@ -846,7 +888,7 @@ public class HConnectionManager {
         public boolean processRow(Result row) throws IOException {
           HRegionInfo info = MetaScanner.getHRegionInfo(row);
           if (info != null) {
-            if (Bytes.compareTo(tableName, info.getTableName()) == 0) {
+            if (tableName.equals(info.getTableName())) {
               ServerName server = HRegionInfo.getServerName(row);
               if (server == null) {
                 available.set(false);
@@ -864,7 +906,7 @@ public class HConnectionManager {
                 // Always empty start row should be counted
                 regionCount.incrementAndGet();
               }
-            } else if (Bytes.compareTo(tableName, info.getTableName()) < 0) {
+            } else if (tableName.compareTo(info.getTableName()) < 0) {
               // Return if we are done with the current table
               return false;
             }
@@ -878,6 +920,12 @@ public class HConnectionManager {
     }
 
     @Override
+    public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys)
+        throws IOException {
+      return isTableAvailable(TableName.valueOf(tableName), splitKeys);
+    }
+
+    @Override
     public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
       return locateRegion(HRegionInfo.getTableName(regionName),
           HRegionInfo.getStartKey(regionName), false, true);
@@ -893,14 +941,20 @@ public class HConnectionManager {
     }
 
     @Override
-    public List<HRegionLocation> locateRegions(final byte[] tableName)
+    public List<HRegionLocation> locateRegions(final TableName tableName)
     throws IOException {
       return locateRegions (tableName, false, true);
     }
 
     @Override
-    public List<HRegionLocation> locateRegions(final byte[] tableName, final boolean useCache,
-        final boolean offlined) throws IOException {
+    public List<HRegionLocation> locateRegions(final byte[] tableName)
+    throws IOException {
+      return locateRegions(TableName.valueOf(tableName));
+    }
+
+    @Override
+    public List<HRegionLocation> locateRegions(final TableName tableName,
+        final boolean useCache, final boolean offlined) throws IOException {
       NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, this,
           tableName, offlined);
       final List<HRegionLocation> locations = new ArrayList<HRegionLocation>();
@@ -911,41 +965,59 @@ public class HConnectionManager {
     }
 
     @Override
-    public HRegionLocation locateRegion(final byte [] tableName,
+    public List<HRegionLocation> locateRegions(final byte[] tableName,
+       final boolean useCache, final boolean offlined) throws IOException {
+      return locateRegions(TableName.valueOf(tableName), useCache, offlined);
+    }
+
+    @Override
+    public HRegionLocation locateRegion(final TableName tableName,
         final byte [] row)
     throws IOException{
       return locateRegion(tableName, row, true, true);
     }
 
     @Override
-    public HRegionLocation relocateRegion(final byte [] tableName,
+    public HRegionLocation locateRegion(final byte[] tableName,
         final byte [] row)
     throws IOException{
+      return locateRegion(TableName.valueOf(tableName), row);
+    }
 
+    @Override
+    public HRegionLocation relocateRegion(final TableName tableName,
+        final byte [] row) throws IOException{
       // Since this is an explicit request not to use any caching, finding
       // disabled tables should not be desirable.  This will ensure that an exception is thrown when
       // the first time a disabled table is interacted with.
       if (isTableDisabled(tableName)) {
-        throw new DoNotRetryIOException(Bytes.toString(tableName) + " is disabled.");
+        throw new DoNotRetryIOException(tableName.getNameAsString() + " is disabled.");
       }
 
       return locateRegion(tableName, row, false, true);
     }
 
-    private HRegionLocation locateRegion(final byte [] tableName,
+    @Override
+    public HRegionLocation relocateRegion(final byte[] tableName,
+        final byte [] row) throws IOException {
+      return relocateRegion(TableName.valueOf(tableName), row);
+    }
+
+
+    private HRegionLocation locateRegion(final TableName tableName,
       final byte [] row, boolean useCache, boolean retry)
     throws IOException {
       if (this.closed) throw new IOException(toString() + " closed");
-      if (tableName == null || tableName.length == 0) {
+      if (tableName== null || tableName.getName().length == 0) {
         throw new IllegalArgumentException(
             "table name cannot be null or zero length");
       }
 
-      if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
+      if (tableName.equals(TableName.META_TABLE_NAME)) {
         return this.registry.getMetaRegionLocation();
       } else {
         // Region not in the cache - have to go to the meta RS
-        return locateRegionInMeta(HConstants.META_TABLE_NAME, tableName, row,
+        return locateRegionInMeta(TableName.META_TABLE_NAME, tableName, row,
           useCache, userRegionLock, retry);
       }
     }
@@ -955,7 +1027,7 @@ public class HConnectionManager {
      * row we're seeking. It will prefetch certain number of regions info and
      * save them to the global region cache.
      */
-    private void prefetchRegionCache(final byte[] tableName,
+    private void prefetchRegionCache(final TableName tableName,
         final byte[] row) {
       // Implement a new visitor for MetaScanner, and use it to walk through
       // the .META.
@@ -968,7 +1040,7 @@ public class HConnectionManager {
             }
 
             // possible we got a region of a different table...
-            if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
+            if (!regionInfo.getTableName().equals(tableName)) {
               return false; // stop scanning
             }
             if (regionInfo.isOffline()) {
@@ -994,7 +1066,7 @@ public class HConnectionManager {
       try {
         // pre-fetch certain number of regions info at region cache.
         MetaScanner.metaScan(conf, this, visitor, tableName, row,
-            this.prefetchRegionLimit, HConstants.META_TABLE_NAME);
+            this.prefetchRegionLimit, TableName.META_TABLE_NAME);
       } catch (IOException e) {
         LOG.warn("Encountered problems when prefetch META table: ", e);
       }
@@ -1004,8 +1076,8 @@ public class HConnectionManager {
       * Search the .META. table for the HRegionLocation
       * info that contains the table and row we're seeking.
       */
-    private HRegionLocation locateRegionInMeta(final byte [] parentTable,
-      final byte [] tableName, final byte [] row, boolean useCache,
+    private HRegionLocation locateRegionInMeta(final TableName parentTable,
+      final TableName tableName, final byte [] row, boolean useCache,
       Object regionLockObject, boolean retry)
     throws IOException {
       HRegionLocation location;
@@ -1051,7 +1123,7 @@ public class HConnectionManager {
               }
               // If the parent table is META, we may want to pre-fetch some
               // region info into the global region cache for this table.
-              if (Bytes.equals(parentTable, HConstants.META_TABLE_NAME)
+              if (parentTable.equals(TableName.META_TABLE_NAME)
                   && (getRegionCachePrefetch(tableName))) {
                 prefetchRegionCache(tableName, row);
               }
@@ -1070,21 +1142,21 @@ public class HConnectionManager {
               HConstants.CATALOG_FAMILY);
           }
           if (regionInfoRow == null) {
-            throw new TableNotFoundException(Bytes.toString(tableName));
+            throw new TableNotFoundException(tableName);
           }
 
           // convert the row result into the HRegionLocation we need!
           HRegionInfo regionInfo = MetaScanner.getHRegionInfo(regionInfoRow);
           if (regionInfo == null) {
             throw new IOException("HRegionInfo was null or empty in " +
-              Bytes.toString(parentTable) + ", row=" + regionInfoRow);
+              parentTable + ", row=" + regionInfoRow);
           }
 
           // possible we got a region of a different table...
-          if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
+          if (!regionInfo.getTableName().equals(tableName)) {
             throw new TableNotFoundException(
-                  "Table '" + Bytes.toString(tableName) + "' was not found, got: " +
-                  Bytes.toString(regionInfo.getTableName()) + ".");
+                  "Table '" + tableName + "' was not found, got: " +
+                  regionInfo.getTableName() + ".");
           }
           if (regionInfo.isSplit()) {
             throw new RegionOfflineException("the only available region for" +
@@ -1101,7 +1173,7 @@ public class HConnectionManager {
           ServerName serverName = HRegionInfo.getServerName(regionInfoRow);
           if (serverName == null) {
             throw new NoServerForRegionException("No server address listed " +
-              "in " + Bytes.toString(parentTable) + " for region " +
+              "in " + parentTable + " for region " +
               regionInfo.getRegionNameAsString() + " containing row " +
               Bytes.toStringBinary(row));
           }
@@ -1129,7 +1201,7 @@ public class HConnectionManager {
           if (tries < numTries - 1) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("locateRegionInMeta parentTable=" +
-                Bytes.toString(parentTable) + ", metaLocation=" +
+                parentTable + ", metaLocation=" +
                 ((metaLocation == null)? "null": "{" + metaLocation + "}") +
                 ", attempt=" + tries + " of " +
                 this.numTries + " failed; retrying after sleep of " +
@@ -1165,9 +1237,9 @@ public class HConnectionManager {
      * @param row
      * @return Null or region location found in cache.
      */
-    HRegionLocation getCachedLocation(final byte [] tableName,
+    HRegionLocation getCachedLocation(final TableName tableName,
         final byte [] row) {
-      SoftValueSortedMap<byte [], HRegionLocation> tableLocations =
+      SoftValueSortedMap<byte[], HRegionLocation> tableLocations =
         getTableLocations(tableName);
 
       // start to examine the cache. we can only do cache actions
@@ -1207,7 +1279,7 @@ public class HConnectionManager {
      * @param tableName tableName
      * @param row
      */
-    void forceDeleteCachedLocation(final byte [] tableName, final byte [] row) {
+    void forceDeleteCachedLocation(final TableName tableName, final byte [] row) {
       HRegionLocation rl = null;
       synchronized (this.cachedRegionLocations) {
         Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
@@ -1223,7 +1295,7 @@ public class HConnectionManager {
       if ((rl != null) && LOG.isDebugEnabled()) {
         LOG.debug("Removed " + rl.getHostname() + ":" + rl.getPort()
           + " as a location of " + rl.getRegionInfo().getRegionNameAsString() +
-          " for tableName=" + Bytes.toString(tableName) + " from cache");
+          " for tableName=" + tableName + " from cache");
       }
     }
 
@@ -1259,18 +1331,16 @@ public class HConnectionManager {
      * @param tableName
      * @return Map of cached locations for passed <code>tableName</code>
      */
-    private SoftValueSortedMap<byte [], HRegionLocation> getTableLocations(
-        final byte [] tableName) {
+    private SoftValueSortedMap<byte[], HRegionLocation> getTableLocations(
+        final TableName tableName) {
       // find the map of cached locations for this table
-      Integer key = Bytes.mapKey(tableName);
-      SoftValueSortedMap<byte [], HRegionLocation> result;
+      SoftValueSortedMap<byte[], HRegionLocation> result;
       synchronized (this.cachedRegionLocations) {
-        result = this.cachedRegionLocations.get(key);
+        result = this.cachedRegionLocations.get(tableName);
         // if tableLocations for this table isn't built yet, make one
         if (result == null) {
-          result = new SoftValueSortedMap<byte [], HRegionLocation>(
-              Bytes.BYTES_COMPARATOR);
-          this.cachedRegionLocations.put(key, result);
+          result = new SoftValueSortedMap<byte[], HRegionLocation>(Bytes.BYTES_COMPARATOR);
+          this.cachedRegionLocations.put(tableName, result);
         }
       }
       return result;
@@ -1285,23 +1355,28 @@ public class HConnectionManager {
     }
 
     @Override
-    public void clearRegionCache(final byte [] tableName) {
+    public void clearRegionCache(final TableName tableName) {
       synchronized (this.cachedRegionLocations) {
-        this.cachedRegionLocations.remove(Bytes.mapKey(tableName));
+        this.cachedRegionLocations.remove(tableName);
       }
     }
 
+    @Override
+    public void clearRegionCache(final byte[] tableName) {
+      clearRegionCache(TableName.valueOf(tableName));
+    }
+
     /**
      * Put a newly discovered HRegionLocation into the cache.
      * @param tableName The table name.
      * @param source the source of the new location, if it's not coming from meta
      * @param location the new location
      */
-    private void cacheLocation(final byte [] tableName, final HRegionLocation source,
+    private void cacheLocation(final TableName tableName, final HRegionLocation source,
         final HRegionLocation location) {
       boolean isFromMeta = (source == null);
       byte [] startKey = location.getRegionInfo().getStartKey();
-      Map<byte [], HRegionLocation> tableLocations =
+      Map<byte[], HRegionLocation> tableLocations =
         getTableLocations(tableName);
       boolean isNewCacheEntry = false;
       boolean isStaleUpdate = false;
@@ -2004,6 +2079,36 @@ public class HConnectionManager {
         }
 
         @Override
+        public ModifyNamespaceResponse modifyNamespace(RpcController controller, ModifyNamespaceRequest request) throws ServiceException {
+          return stub.modifyNamespace(controller, request);
+        }
+
+        @Override
+        public CreateNamespaceResponse createNamespace(RpcController controller, CreateNamespaceRequest request) throws ServiceException {
+          return stub.createNamespace(controller, request);
+        }
+
+        @Override
+        public DeleteNamespaceResponse deleteNamespace(RpcController controller, DeleteNamespaceRequest request) throws ServiceException {
+          return stub.deleteNamespace(controller, request);
+        }
+
+        @Override
+        public GetNamespaceDescriptorResponse getNamespaceDescriptor(RpcController controller, GetNamespaceDescriptorRequest request) throws ServiceException {
+          return stub.getNamespaceDescriptor(controller, request);
+        }
+
+        @Override
+        public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller, ListNamespaceDescriptorsRequest request) throws ServiceException {
+          return stub.listNamespaceDescriptors(controller, request);
+        }
+
+        @Override
+        public GetTableDescriptorsByNamespaceResponse getTableDescriptorsByNamespace(RpcController controller, GetTableDescriptorsByNamespaceRequest request) throws ServiceException {
+          return stub.getTableDescriptorsByNamespace(controller, request);
+        }
+
+        @Override
         public void close() {
           release(this.mss);
         }
@@ -2147,8 +2252,9 @@ public class HConnectionManager {
         return;
       }
       synchronized (this.cachedRegionLocations) {
-        byte[] tableName = location.getRegionInfo().getTableName();
-        Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
+        TableName tableName = location.getRegionInfo().getTableName();
+        Map<byte[], HRegionLocation> tableLocations =
+            getTableLocations(tableName);
         if (!tableLocations.isEmpty()) {
           // Delete if there's something in the cache for this region.
           HRegionLocation removedLocation =
@@ -2156,7 +2262,7 @@ public class HConnectionManager {
           if (LOG.isDebugEnabled() && removedLocation != null) {
             LOG.debug("Removed " +
                 location.getRegionInfo().getRegionNameAsString() +
-                " for tableName=" + Bytes.toString(tableName) +
+                " for tableName=" + tableName +
                 " from cache");
           }
         }
@@ -2171,11 +2277,11 @@ public class HConnectionManager {
      * @param source server that is the source of the location update.
      */
     @Override
-    public void updateCachedLocations(final byte[] tableName, byte[] rowkey,
+    public void updateCachedLocations(final TableName tableName, byte[] rowkey,
       final Object exception, final HRegionLocation source) {
       if (rowkey == null || tableName == null) {
         LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) +
-            ", tableName=" + (tableName == null ? "null" : Bytes.toString(tableName)));
+            ", tableName=" + (tableName == null ? "null" : tableName));
         return;
       }
 
@@ -2206,9 +2312,15 @@ public class HConnectionManager {
     }
 
     @Override
+    public void updateCachedLocations(final byte[] tableName, byte[] rowkey,
+      final Object exception, final HRegionLocation source) {
+      updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source);
+    }
+
+    @Override
     @Deprecated
     public void processBatch(List<? extends Row> list,
-        final byte[] tableName,
+        final TableName tableName,
         ExecutorService pool,
         Object[] results) throws IOException, InterruptedException {
       // This belongs in HTable!!! Not in here.  St.Ack
@@ -2221,6 +2333,15 @@ public class HConnectionManager {
       processBatchCallback(list, tableName, pool, results, null);
     }
 
+    @Override
+    @Deprecated
+    public void processBatch(List<? extends Row> list,
+        final byte[] tableName,
+        ExecutorService pool,
+        Object[] results) throws IOException, InterruptedException {
+      processBatch(list, TableName.valueOf(tableName), pool, results);
+    }
+
     /**
      * Send the queries in parallel on the different region servers. Retries on failures.
      * If the method returns it means that there is no error, and the 'results' array will
@@ -2232,7 +2353,7 @@ public class HConnectionManager {
     @Deprecated
     public <R> void processBatchCallback(
       List<? extends Row> list,
-      byte[] tableName,
+      TableName tableName,
       ExecutorService pool,
       Object[] results,
       Batch.Callback<R> callback)
@@ -2252,8 +2373,20 @@ public class HConnectionManager {
       }
     }
 
+    @Override
+    @Deprecated
+    public <R> void processBatchCallback(
+      List<? extends Row> list,
+      byte[] tableName,
+      ExecutorService pool,
+      Object[] results,
+      Batch.Callback<R> callback)
+      throws IOException, InterruptedException {
+      processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
+    }
+
     // For tests.
-    protected <R> AsyncProcess createAsyncProcess(byte[] tableName, ExecutorService pool,
+    protected <R> AsyncProcess createAsyncProcess(TableName tableName, ExecutorService pool,
            AsyncProcess.AsyncProcessCallback<R> callback, Configuration conf) {
       return new AsyncProcess<R>(this, tableName, pool, callback, conf,
           RpcRetryingCallerFactory.instantiate(conf));
@@ -2302,10 +2435,9 @@ public class HConnectionManager {
      * Return the number of cached region for a table. It will only be called
      * from a unit test.
      */
-    int getNumberOfCachedRegionLocations(final byte[] tableName) {
-      Integer key = Bytes.mapKey(tableName);
+    int getNumberOfCachedRegionLocations(final TableName tableName) {
       synchronized (this.cachedRegionLocations) {
-        Map<byte[], HRegionLocation> tableLocs = this.cachedRegionLocations.get(key);
+        Map<byte[], HRegionLocation> tableLocs = this.cachedRegionLocations.get(tableName);
         if (tableLocs == null) {
           return 0;
         }
@@ -2320,25 +2452,36 @@ public class HConnectionManager {
      * @param row row
      * @return Region cached or not.
      */
-    boolean isRegionCached(final byte[] tableName, final byte[] row) {
+    boolean isRegionCached(TableName tableName, final byte[] row) {
       HRegionLocation location = getCachedLocation(tableName, row);
       return location != null;
     }
 
     @Override
-    public void setRegionCachePrefetch(final byte[] tableName,
+    public void setRegionCachePrefetch(final TableName tableName,
         final boolean enable) {
       if (!enable) {
-        regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName));
+        regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName.getName()));
       }
       else {
-        regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName));
+        regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName.getName()));
       }
     }
 
     @Override
-    public boolean getRegionCachePrefetch(final byte[] tableName) {
-      return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName));
+    public void setRegionCachePrefetch(final byte[] tableName,
+        final boolean enable) {
+      setRegionCachePrefetch(TableName.valueOf(tableName), enable);
+    }
+
+    @Override
+    public boolean getRegionCachePrefetch(TableName tableName) {
+      return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName.getName()));
+    }
+
+    @Override
+    public boolean getRegionCachePrefetch(byte[] tableName) {
+      return getRegionCachePrefetch(TableName.valueOf(tableName));
     }
 
     @Override
@@ -2457,7 +2600,7 @@ public class HConnectionManager {
       MasterMonitorKeepAliveConnection master = getKeepAliveMasterMonitorService();
       try {
         GetTableDescriptorsRequest req =
-          RequestConverter.buildGetTableDescriptorsRequest((List<String>)null);
+          RequestConverter.buildGetTableDescriptorsRequest((List<TableName>)null);
         return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
       } catch (ServiceException se) {
         throw ProtobufUtil.getRemoteException(se);
@@ -2467,7 +2610,8 @@ public class HConnectionManager {
     }
 
     @Override
-    public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) throws IOException {
+    public HTableDescriptor[] getHTableDescriptorsByTableName(
+        List<TableName> tableNames) throws IOException {
       if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0];
       MasterMonitorKeepAliveConnection master = getKeepAliveMasterMonitorService();
       try {
@@ -2481,6 +2625,17 @@ public class HConnectionManager {
       }
     }
 
+    @Override
+    public HTableDescriptor[] getHTableDescriptors(
+        List<String> names) throws IOException {
+      List<TableName> tableNames = new ArrayList(names.size());
+      for(String name : names) {
+        tableNames.add(TableName.valueOf(name));
+      }
+
+      return getHTableDescriptorsByTableName(tableNames);
+    }
+
     /**
      * Connects to the master to get the table descriptor.
      * @param tableName table name
@@ -2489,10 +2644,10 @@ public class HConnectionManager {
      *  is not found.
      */
     @Override
-    public HTableDescriptor getHTableDescriptor(final byte[] tableName)
+    public HTableDescriptor getHTableDescriptor(final TableName tableName)
     throws IOException {
-      if (tableName == null || tableName.length == 0) return null;
-      if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
+      if (tableName == null) return null;
+      if (tableName.equals(TableName.META_TABLE_NAME)) {
         return HTableDescriptor.META_TABLEDESC;
       }
       MasterMonitorKeepAliveConnection master = getKeepAliveMasterMonitorService();
@@ -2509,7 +2664,13 @@ public class HConnectionManager {
       if (!htds.getTableSchemaList().isEmpty()) {
         return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
       }
-      throw new TableNotFoundException(Bytes.toString(tableName));
+      throw new TableNotFoundException(tableName.getNameAsString());
+    }
+
+    @Override
+    public HTableDescriptor getHTableDescriptor(final byte[] tableName)
+    throws IOException {
+      return getHTableDescriptor(TableName.valueOf(tableName));
     }
   }
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionWrapper.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionWrapper.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionWrapper.java Thu Aug  8 04:19:49 2013
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 
@@ -28,7 +29,9 @@ import org.apache.hadoop.hbase.HRegionLo
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
@@ -106,88 +109,146 @@ public class HConnectionWrapper implemen
   }
 
   @Override
-  public boolean isTableEnabled(byte[] tableName) throws IOException {
+  public boolean isTableEnabled(TableName tableName) throws IOException {
     return hconnection.isTableEnabled(tableName);
   }
 
   @Override
-  public boolean isTableDisabled(byte[] tableName) throws IOException {
+  public boolean isTableEnabled(byte[] tableName) throws IOException {
+    return isTableEnabled(TableName.valueOf(tableName));
+  }
+
+  @Override
+  public boolean isTableDisabled(TableName tableName) throws IOException {
     return hconnection.isTableDisabled(tableName);
   }
 
   @Override
-  public boolean isTableAvailable(byte[] tableName) throws IOException {
+  public boolean isTableDisabled(byte[] tableName) throws IOException {
+    return isTableDisabled(TableName.valueOf(tableName));
+  }
+
+  @Override
+  public boolean isTableAvailable(TableName tableName) throws IOException {
     return hconnection.isTableAvailable(tableName);
   }
 
   @Override
-  public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys)
-      throws IOException {
+  public boolean isTableAvailable(byte[] tableName) throws IOException {
+    return isTableAvailable(TableName.valueOf(tableName));
+  }
+
+  @Override
+  public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
     return hconnection.isTableAvailable(tableName, splitKeys);
   }
 
   @Override
+  public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException {
+    return isTableAvailable(TableName.valueOf(tableName), splitKeys);
+  }
+
+  @Override
   public HTableDescriptor[] listTables() throws IOException {
     return hconnection.listTables();
   }
 
   @Override
-  public HTableDescriptor getHTableDescriptor(byte[] tableName)
-      throws IOException {
+  public HTableDescriptor getHTableDescriptor(TableName tableName) throws IOException {
     return hconnection.getHTableDescriptor(tableName);
   }
 
   @Override
-  public HRegionLocation locateRegion(byte[] tableName, byte[] row)
-      throws IOException {
+  public HTableDescriptor getHTableDescriptor(byte[] tableName) throws IOException {
+    return getHTableDescriptor(TableName.valueOf(tableName));
+  }
+
+  @Override
+  public HRegionLocation locateRegion(TableName tableName, byte[] row) throws IOException {
     return hconnection.locateRegion(tableName, row);
   }
 
   @Override
+  public HRegionLocation locateRegion(byte[] tableName, byte[] row) throws IOException {
+    return locateRegion(TableName.valueOf(tableName), row);
+  }
+
+  @Override
   public void clearRegionCache() {
     hconnection.clearRegionCache();
   }
 
   @Override
-  public void clearRegionCache(byte[] tableName) {
+  public void clearRegionCache(TableName tableName) {
     hconnection.clearRegionCache(tableName);
   }
 
   @Override
+  public void clearRegionCache(byte[] tableName) {
+    clearRegionCache(TableName.valueOf(tableName));
+  }
+
+  @Override
   public void deleteCachedRegionLocation(HRegionLocation location) {
     hconnection.deleteCachedRegionLocation(location);
   }
 
   @Override
-  public HRegionLocation relocateRegion(byte[] tableName, byte[] row)
-      throws IOException {
+  public HRegionLocation relocateRegion(TableName tableName, byte[] row) throws IOException {
     return hconnection.relocateRegion(tableName, row);
   }
 
   @Override
-  public void updateCachedLocations(byte[] tableName, byte[] rowkey,
-      Object exception, HRegionLocation source) {
+  public HRegionLocation relocateRegion(byte[] tableName, byte[] row) throws IOException {
+    return relocateRegion(TableName.valueOf(tableName), row);
+  }
+
+  @Override
+  public void updateCachedLocations(TableName tableName,
+                                    byte[] rowkey,
+                                    Object exception,
+                                    HRegionLocation source) {
     hconnection.updateCachedLocations(tableName, rowkey, exception, source);
   }
 
   @Override
+  public void updateCachedLocations(byte[] tableName,
+                                    byte[] rowkey,
+                                    Object exception,
+                                    HRegionLocation source) {
+    updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source);
+  }
+
+  @Override
   public HRegionLocation locateRegion(byte[] regionName) throws IOException {
     return hconnection.locateRegion(regionName);
   }
 
   @Override
-  public List<HRegionLocation> locateRegions(byte[] tableName)
-      throws IOException {
+  public List<HRegionLocation> locateRegions(TableName tableName) throws IOException {
     return hconnection.locateRegions(tableName);
   }
 
   @Override
-  public List<HRegionLocation> locateRegions(byte[] tableName,
-      boolean useCache, boolean offlined) throws IOException {
+  public List<HRegionLocation> locateRegions(byte[] tableName) throws IOException {
+    return locateRegions(TableName.valueOf(tableName));
+  }
+
+  @Override
+  public List<HRegionLocation> locateRegions(TableName tableName,
+                                             boolean useCache,
+                                             boolean offlined) throws IOException {
     return hconnection.locateRegions(tableName, useCache, offlined);
   }
 
   @Override
+  public List<HRegionLocation> locateRegions(byte[] tableName,
+                                             boolean useCache,
+                                             boolean offlined) throws IOException {
+    return locateRegions(TableName.valueOf(tableName));
+  }
+
+  @Override
   public MasterAdminService.BlockingInterface getMasterAdmin() throws IOException {
     return hconnection.getMasterAdmin();
   }
@@ -237,44 +298,86 @@ public class HConnectionWrapper implemen
   }
 
   @Override
-  public HRegionLocation getRegionLocation(byte[] tableName, byte[] row,
-      boolean reload) throws IOException {
+  public HRegionLocation getRegionLocation(TableName tableName,
+                                           byte[] row, boolean reload) throws IOException {
     return hconnection.getRegionLocation(tableName, row, reload);
   }
 
   @Override
-  public void processBatch(List<? extends Row> actions, byte[] tableName,
-      ExecutorService pool, Object[] results) throws IOException,
-      InterruptedException {
+  public HRegionLocation getRegionLocation(byte[] tableName,
+                                           byte[] row, boolean reload) throws IOException {
+    return getRegionLocation(TableName.valueOf(tableName), row, reload);
+  }
+
+  @Override
+  public void processBatch(List<? extends Row> actions, TableName tableName, ExecutorService pool,
+                           Object[] results) throws IOException, InterruptedException {
     hconnection.processBatch(actions, tableName, pool, results);
   }
 
   @Override
-  public <R> void processBatchCallback(List<? extends Row> list,
-      byte[] tableName, ExecutorService pool, Object[] results,
-      Callback<R> callback) throws IOException, InterruptedException {
+  public void processBatch(List<? extends Row> actions, byte[] tableName, ExecutorService pool,
+                           Object[] results) throws IOException, InterruptedException {
+    processBatch(actions, TableName.valueOf(tableName), pool, results);
+  }
+
+  @Override
+  public <R> void processBatchCallback(List<? extends Row> list, TableName tableName,
+                                       ExecutorService pool,
+                                       Object[] results,
+                                       Callback<R> callback)
+      throws IOException, InterruptedException {
     hconnection.processBatchCallback(list, tableName, pool, results, callback);
   }
 
   @Override
-  public void setRegionCachePrefetch(byte[] tableName, boolean enable) {
+  public <R> void processBatchCallback(List<? extends Row> list, byte[] tableName,
+                                       ExecutorService pool,
+                                       Object[] results,
+                                       Callback<R> callback)
+      throws IOException, InterruptedException {
+    processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
+  }
+
+  @Override
+  public void setRegionCachePrefetch(TableName tableName, boolean enable) {
     hconnection.setRegionCachePrefetch(tableName, enable);
   }
 
   @Override
-  public boolean getRegionCachePrefetch(byte[] tableName) {
+  public void setRegionCachePrefetch(byte[] tableName, boolean enable) {
+    setRegionCachePrefetch(TableName.valueOf(tableName), enable);
+  }
+
+  @Override
+  public boolean getRegionCachePrefetch(TableName tableName) {
     return hconnection.getRegionCachePrefetch(tableName);
   }
 
   @Override
+  public boolean getRegionCachePrefetch(byte[] tableName) {
+    return getRegionCachePrefetch(TableName.valueOf(tableName));
+  }
+
+  @Override
   public int getCurrentNrHRS() throws IOException {
     return hconnection.getCurrentNrHRS();
   }
 
   @Override
-  public HTableDescriptor[] getHTableDescriptors(List<String> tableNames)
-      throws IOException {
-    return hconnection.getHTableDescriptors(tableNames);
+  public HTableDescriptor[] getHTableDescriptorsByTableName(
+      List<TableName> tableNames) throws IOException {
+    return hconnection.getHTableDescriptorsByTableName(tableNames);
+  }
+
+  @Override
+  public HTableDescriptor[] getHTableDescriptors(
+      List<String> names) throws IOException {
+    List<TableName> tableNames = new ArrayList<TableName>(names.size());
+    for(String name : names) {
+      tableNames.add(TableName.valueOf(name));
+    }
+    return getHTableDescriptorsByTableName(tableNames);
   }
 
   @Override

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Thu Aug  8 04:19:49 2013
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -51,7 +52,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
 
 import java.io.Closeable;
@@ -119,7 +119,7 @@ import java.util.concurrent.TimeUnit;
 public class HTable implements HTableInterface {
   private static final Log LOG = LogFactory.getLog(HTable.class);
   protected HConnection connection;
-  private final byte [] tableName;
+  private final TableName tableName;
   private volatile Configuration configuration;
   protected List<Row> writeAsyncBuffer = new LinkedList<Row>();
   private long writeBufferSize;
@@ -150,10 +150,9 @@ public class HTable implements HTableInt
    */
   public HTable(Configuration conf, final String tableName)
   throws IOException {
-    this(conf, Bytes.toBytes(tableName));
+    this(conf, TableName.valueOf(tableName));
   }
 
-
   /**
    * Creates an object to access a HBase table.
    * Shares zookeeper connection and other resources with other HTable instances
@@ -164,7 +163,24 @@ public class HTable implements HTableInt
    * @param tableName Name of the table.
    * @throws IOException if a remote or network exception occurs
    */
-  public HTable(Configuration conf, final byte [] tableName)
+  public HTable(Configuration conf, final byte[] tableName)
+  throws IOException {
+    this(conf, TableName.valueOf(tableName));
+  }
+
+
+
+  /**
+   * Creates an object to access a HBase table.
+   * Shares zookeeper connection and other resources with other HTable instances
+   * created with the same <code>conf</code> instance.  Uses already-populated
+   * region cache if one is available, populated by any other HTable instances
+   * sharing this <code>conf</code> instance.  Recommended.
+   * @param conf Configuration object to use.
+   * @param tableName table name pojo
+   * @throws IOException if a remote or network exception occurs
+   */
+  public HTable(Configuration conf, final TableName tableName)
   throws IOException {
     this.tableName = tableName;
     this.cleanupPoolOnClose = this.cleanupConnectionOnClose = true;
@@ -206,6 +222,23 @@ public class HTable implements HTableInt
    */
   public HTable(Configuration conf, final byte[] tableName, final ExecutorService pool)
       throws IOException {
+    this(conf, TableName.valueOf(tableName), pool);
+  }
+
+  /**
+   * Creates an object to access a HBase table.
+   * Shares zookeeper connection and other resources with other HTable instances
+   * created with the same <code>conf</code> instance.  Uses already-populated
+   * region cache if one is available, populated by any other HTable instances
+   * sharing this <code>conf</code> instance.
+   * Use this constructor when the ExecutorService is externally managed.
+   * @param conf Configuration object to use.
+   * @param tableName Name of the table.
+   * @param pool ExecutorService to be used.
+   * @throws IOException if a remote or network exception occurs
+   */
+  public HTable(Configuration conf, final TableName tableName, final ExecutorService pool)
+      throws IOException {
     this.connection = HConnectionManager.getConnection(conf);
     this.configuration = conf;
     this.pool = pool;
@@ -229,6 +262,22 @@ public class HTable implements HTableInt
    */
   public HTable(final byte[] tableName, final HConnection connection,
       final ExecutorService pool) throws IOException {
+    this(TableName.valueOf(tableName), connection, pool);
+  }
+
+  /**
+   * Creates an object to access a HBase table.
+   * Shares zookeeper connection and other resources with other HTable instances
+   * created with the same <code>connection</code> instance.
+   * Use this constructor when the ExecutorService and HConnection instance are
+   * externally managed.
+   * @param tableName Name of the table.
+   * @param connection HConnection to be used.
+   * @param pool ExecutorService to be used.
+   * @throws IOException if a remote or network exception occurs
+   */
+  public HTable(TableName tableName, final HConnection connection,
+      final ExecutorService pool) throws IOException {
     if (connection == null || connection.isClosed()) {
       throw new IllegalArgumentException("Connection is null or closed.");
     }
@@ -245,7 +294,7 @@ public class HTable implements HTableInt
    * For internal testing.
    */
   protected HTable(){
-    tableName = new byte[]{};
+    tableName = null;
     cleanupPoolOnClose = false;
     cleanupConnectionOnClose = false;
   }
@@ -255,7 +304,7 @@ public class HTable implements HTableInt
    */
   private void finishSetup() throws IOException {
     this.connection.locateRegion(tableName, HConstants.EMPTY_START_ROW);
-    this.operationTimeout = HTableDescriptor.isMetaTable(tableName) ?
+    this.operationTimeout = HTableDescriptor.isSystemTable(tableName) ?
       this.configuration.getInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT,
         HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT):
       this.configuration.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
@@ -299,7 +348,7 @@ public class HTable implements HTableInt
    */
   @Deprecated
   public static boolean isTableEnabled(String tableName) throws IOException {
-    return isTableEnabled(Bytes.toBytes(tableName));
+    return isTableEnabled(TableName.valueOf(tableName));
   }
 
   /**
@@ -309,10 +358,24 @@ public class HTable implements HTableInt
    * @param tableName Name of table to check.
    * @return {@code true} if table is online.
    * @throws IOException if a remote or network exception occurs
-   * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
+	* @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
    */
   @Deprecated
   public static boolean isTableEnabled(byte[] tableName) throws IOException {
+    return isTableEnabled(TableName.valueOf(tableName));
+  }
+
+  /**
+   * Tells whether or not a table is enabled or not. This method creates a
+   * new HBase configuration, so it might make your unit tests fail due to
+   * incorrect ZK client port.
+   * @param tableName Name of table to check.
+   * @return {@code true} if table is online.
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
+   */
+  @Deprecated
+  public static boolean isTableEnabled(TableName tableName) throws IOException {
     return isTableEnabled(HBaseConfiguration.create(), tableName);
   }
 
@@ -327,7 +390,7 @@ public class HTable implements HTableInt
   @Deprecated
   public static boolean isTableEnabled(Configuration conf, String tableName)
   throws IOException {
-    return isTableEnabled(conf, Bytes.toBytes(tableName));
+    return isTableEnabled(conf, TableName.valueOf(tableName));
   }
 
   /**
@@ -336,11 +399,25 @@ public class HTable implements HTableInt
    * @param tableName Name of table to check.
    * @return {@code true} if table is online.
    * @throws IOException if a remote or network exception occurs
-   * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[] tableName)}
+	 * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
+   */
+  @Deprecated
+  public static boolean isTableEnabled(Configuration conf, byte[] tableName)
+  throws IOException {
+    return isTableEnabled(conf, TableName.valueOf(tableName));
+  }
+
+  /**
+   * Tells whether or not a table is enabled or not.
+   * @param conf The Configuration object to use.
+   * @param tableName Name of table to check.
+   * @return {@code true} if table is online.
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated use {@link HBaseAdmin#isTableEnabled(org.apache.hadoop.hbase.TableName tableName)}
    */
   @Deprecated
   public static boolean isTableEnabled(Configuration conf,
-      final byte[] tableName) throws IOException {
+      final TableName tableName) throws IOException {
     return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
       @Override
       public Boolean connect(HConnection connection) throws IOException {
@@ -388,7 +465,12 @@ public class HTable implements HTableInt
    */
   @Override
   public byte [] getTableName() {
-    return this.tableName;
+    return this.tableName.getName();
+  }
+
+  @Override
+  public TableName getName() {
+    return tableName;
   }
 
   /**
@@ -502,7 +584,7 @@ public class HTable implements HTableInt
    */
   public NavigableMap<HRegionInfo, ServerName> getRegionLocations() throws IOException {
     // TODO: Odd that this returns a Map of HRI to SN whereas getRegionLocation, singular, returns an HRegionLocation.
-    return MetaScanner.allTableRegions(getConfiguration(), this.connection, getTableName(), false);
+    return MetaScanner.allTableRegions(getConfiguration(), this.connection, getName(), false);
   }
 
   /**
@@ -611,7 +693,8 @@ public class HTable implements HTableInt
     if (scan.getCaching() <= 0) {
       scan.setCaching(getScannerCaching());
     }
-    return new ClientScanner(getConfiguration(), scan, getTableName(), this.connection);
+    return new ClientScanner(getConfiguration(), scan,
+        getName(), this.connection);
   }
 
   /**
@@ -641,7 +724,7 @@ public class HTable implements HTableInt
   @Override
   public Result get(final Get get) throws IOException {
     RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-        getTableName(), get.getRow()) {
+        getName(), get.getRow()) {
       public Result call() throws IOException {
         return ProtobufUtil.get(getStub(), getLocation().getRegionInfo().getRegionName(), get);
       }
@@ -813,7 +896,7 @@ public class HTable implements HTableInt
 
       if (synchronous || ap.hasError()) {
         if (ap.hasError() && LOG.isDebugEnabled()) {
-          LOG.debug(Bytes.toString(tableName) + ": One or more of the operations have failed -" +
+          LOG.debug(tableName + ": One or more of the operations have failed -" +
               " waiting for all operation in progress to finish (successfully or not)");
         }
         ap.waitUntilDone();
@@ -845,7 +928,7 @@ public class HTable implements HTableInt
   @Override
   public void mutateRow(final RowMutations rm) throws IOException {
     RegionServerCallable<Void> callable =
-        new RegionServerCallable<Void>(connection, getTableName(), rm.getRow()) {
+        new RegionServerCallable<Void>(connection, getName(), rm.getRow()) {
       public Void call() throws IOException {
         try {
           MultiRequest request = RequestConverter.buildMultiRequest(
@@ -870,7 +953,7 @@ public class HTable implements HTableInt
           "Invalid arguments to append, no columns specified");
     }
     RegionServerCallable<Result> callable =
-      new RegionServerCallable<Result>(this.connection, getTableName(), append.getRow()) {
+      new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
         public Result call() throws IOException {
           try {
             MutateRequest request = RequestConverter.buildMutateRequest(
@@ -897,7 +980,7 @@ public class HTable implements HTableInt
           "Invalid arguments to increment, no columns specified");
     }
     RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-        getTableName(), increment.getRow()) {
+        getName(), increment.getRow()) {
       public Result call() throws IOException {
         try {
           MutateRequest request = RequestConverter.buildMutateRequest(
@@ -944,7 +1027,7 @@ public class HTable implements HTableInt
     }
     
     RegionServerCallable<Long> callable =
-      new RegionServerCallable<Long>(connection, getTableName(), row) {
+      new RegionServerCallable<Long>(connection, getName(), row) {
         public Long call() throws IOException {
           try {
             MutateRequest request = RequestConverter.buildMutateRequest(
@@ -972,7 +1055,7 @@ public class HTable implements HTableInt
       final Put put)
   throws IOException {
     RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getTableName(), row) {
+      new RegionServerCallable<Boolean>(connection, getName(), row) {
         public Boolean call() throws IOException {
           try {
             MutateRequest request = RequestConverter.buildMutateRequest(
@@ -998,7 +1081,7 @@ public class HTable implements HTableInt
       final Delete delete)
   throws IOException {
     RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getTableName(), row) {
+      new RegionServerCallable<Boolean>(connection, getName(), row) {
         public Boolean call() throws IOException {
           try {
             MutateRequest request = RequestConverter.buildMutateRequest(
@@ -1020,7 +1103,7 @@ public class HTable implements HTableInt
   @Override
   public boolean exists(final Get get) throws IOException {
     RegionServerCallable<Boolean> callable =
-        new RegionServerCallable<Boolean>(connection, getTableName(), get.getRow()) {
+        new RegionServerCallable<Boolean>(connection, getName(), get.getRow()) {
       public Boolean call() throws IOException {
         try {
           GetRequest request = RequestConverter.buildGetRequest(
@@ -1124,7 +1207,7 @@ public class HTable implements HTableInt
       Callable<List<Boolean>> callable = new Callable<List<Boolean>>() {
         public List<Boolean> call() throws Exception {
           RegionServerCallable<List<Boolean>> callable =
-            new RegionServerCallable<List<Boolean>>(connection, getTableName(),
+            new RegionServerCallable<List<Boolean>>(connection, getName(),
               getsByRegionEntry.getValue().get(0).getRow()) {
             public List<Boolean> call() throws IOException {
               try {
@@ -1139,7 +1222,7 @@ public class HTable implements HTableInt
             }
           };
           return rpcCallerFactory.<List<Boolean>> newCaller().callWithRetries(callable,
-            operationTimeout);
+              operationTimeout);
         }
       };
       futures.put(getsByRegionEntry.getKey(), pool.submit(callable));
@@ -1352,6 +1435,12 @@ public class HTable implements HTableInt
    */
   public static void setRegionCachePrefetch(final byte[] tableName,
       final boolean enable) throws IOException {
+    setRegionCachePrefetch(TableName.valueOf(tableName), enable);
+  }
+
+  public static void setRegionCachePrefetch(
+      final TableName tableName,
+      final boolean enable) throws IOException {
     HConnectionManager.execute(new HConnectable<Void>(HBaseConfiguration
         .create()) {
       @Override
@@ -1374,6 +1463,12 @@ public class HTable implements HTableInt
    */
   public static void setRegionCachePrefetch(final Configuration conf,
       final byte[] tableName, final boolean enable) throws IOException {
+    setRegionCachePrefetch(conf, TableName.valueOf(tableName), enable);
+  }
+
+  public static void setRegionCachePrefetch(final Configuration conf,
+      final TableName tableName,
+      final boolean enable) throws IOException {
     HConnectionManager.execute(new HConnectable<Void>(conf) {
       @Override
       public Void connect(HConnection connection) throws IOException {
@@ -1393,6 +1488,11 @@ public class HTable implements HTableInt
    */
   public static boolean getRegionCachePrefetch(final Configuration conf,
       final byte[] tableName) throws IOException {
+    return getRegionCachePrefetch(conf, TableName.valueOf(tableName));
+  }
+
+  public static boolean getRegionCachePrefetch(final Configuration conf,
+      final TableName tableName) throws IOException {
     return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
       @Override
       public Boolean connect(HConnection connection) throws IOException {
@@ -1409,6 +1509,11 @@ public class HTable implements HTableInt
    * @throws IOException
    */
   public static boolean getRegionCachePrefetch(final byte[] tableName) throws IOException {
+    return getRegionCachePrefetch(TableName.valueOf(tableName));
+  }
+
+  public static boolean getRegionCachePrefetch(
+      final TableName tableName) throws IOException {
     return HConnectionManager.execute(new HConnectable<Boolean>(
         HBaseConfiguration.create()) {
       @Override
@@ -1416,7 +1521,7 @@ public class HTable implements HTableInt
         return connection.getRegionCachePrefetch(tableName);
       }
     });
- }
+  }
 
   /**
    * Explicitly clears the region cache to fetch the latest value from META.

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java Thu Aug  8 04:19:49 2013
@@ -23,6 +23,7 @@ import com.google.protobuf.ServiceExcept
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -50,6 +51,11 @@ public interface HTableInterface extends
   byte[] getTableName();
 
   /**
+   * Gets the fully qualified table name instance of this table.
+   */
+  TableName getName();
+
+  /**
    * Returns the {@link Configuration} object used by this instance.
    * <p>
    * The reference returned is not a copy, so any change made to it will

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java Thu Aug  8 04:19:49 2013
@@ -25,6 +25,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -67,7 +68,7 @@ public class HTableMultiplexer {
   
   static final String TABLE_MULTIPLEXER_FLUSH_FREQ_MS = "hbase.tablemultiplexer.flush.frequency.ms";
 
-  private Map<byte[], HTable> tableNameToHTableMap;
+  private Map<TableName, HTable> tableNameToHTableMap;
 
   /** The map between each region server to its corresponding buffer queue */
   private Map<HRegionLocation, LinkedBlockingQueue<PutStatus>>
@@ -92,7 +93,7 @@ public class HTableMultiplexer {
     this.serverToBufferQueueMap = new ConcurrentHashMap<HRegionLocation,
       LinkedBlockingQueue<PutStatus>>();
     this.serverToFlushWorkerMap = new ConcurrentHashMap<HRegionLocation, HTableFlushWorker>();
-    this.tableNameToHTableMap = new ConcurrentSkipListMap<byte[], HTable>(Bytes.BYTES_COMPARATOR);
+    this.tableNameToHTableMap = new ConcurrentSkipListMap<TableName, HTable>();
     this.retryNum = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
     this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize;
@@ -101,24 +102,28 @@ public class HTableMultiplexer {
   /**
    * The put request will be buffered by its corresponding buffer queue. Return false if the queue
    * is already full.
-   * @param table
+   * @param tableName
    * @param put
    * @return true if the request can be accepted by its corresponding buffer queue.
    * @throws IOException
    */
-  public boolean put(final byte[] table, final Put put) throws IOException {
-    return put(table, put, this.retryNum);
+  public boolean put(TableName tableName, final Put put) throws IOException {
+    return put(tableName, put, this.retryNum);
+  }
+
+  public boolean put(byte[] tableName, final Put put) throws IOException {
+    return put(TableName.valueOf(tableName), put);
   }
 
   /**
    * The puts request will be buffered by their corresponding buffer queue. 
    * Return the list of puts which could not be queued.
-   * @param table
+   * @param tableName
    * @param puts
    * @return the list of puts which could not be queued
    * @throws IOException
    */
-  public List<Put> put(final byte[] table, final List<Put> puts)
+  public List<Put> put(TableName tableName, final List<Put> puts)
       throws IOException {
     if (puts == null)
       return null;
@@ -126,7 +131,7 @@ public class HTableMultiplexer {
     List <Put> failedPuts = null;
     boolean result;
     for (Put put : puts) {
-      result = put(table, put, this.retryNum);
+      result = put(tableName, put, this.retryNum);
       if (result == false) {
         
         // Create the failed puts list if necessary
@@ -140,24 +145,29 @@ public class HTableMultiplexer {
     return failedPuts;
   }
 
+  public List<Put> put(byte[] tableName, final List<Put> puts) throws IOException {
+    return put(TableName.valueOf(tableName), puts);
+  }
+
+
   /**
    * The put request will be buffered by its corresponding buffer queue. And the put request will be
    * retried before dropping the request.
    * Return false if the queue is already full.
-   * @param table
+   * @param tableName
    * @param put
    * @param retry
    * @return true if the request can be accepted by its corresponding buffer queue.
    * @throws IOException
    */
-  public boolean put(final byte[] table, final Put put, int retry)
+  public boolean put(final TableName tableName, final Put put, int retry)
       throws IOException {
     if (retry <= 0) {
       return false;
     }
 
     LinkedBlockingQueue<PutStatus> queue;
-    HTable htable = getHTable(table);
+    HTable htable = getHTable(tableName);
     try {
       htable.validatePut(put);
       HRegionLocation loc = htable.getRegionLocation(put.getRow(), false);
@@ -175,6 +185,11 @@ public class HTableMultiplexer {
     return false;
   }
 
+  public boolean put(final byte[] tableName, final Put put, int retry)
+      throws IOException {
+    return put(TableName.valueOf(tableName), put, retry);
+  }
+
   /**
    * @return the current HTableMultiplexerStatus
    */
@@ -183,14 +198,14 @@ public class HTableMultiplexer {
   }
 
 
-  private HTable getHTable(final byte[] table) throws IOException {
-    HTable htable = this.tableNameToHTableMap.get(table);
+  private HTable getHTable(TableName tableName) throws IOException {
+    HTable htable = this.tableNameToHTableMap.get(tableName);
     if (htable == null) {
       synchronized (this.tableNameToHTableMap) {
-        htable = this.tableNameToHTableMap.get(table);
+        htable = this.tableNameToHTableMap.get(tableName);
         if (htable == null)  {
-          htable = new HTable(conf, table);
-          this.tableNameToHTableMap.put(table, htable);
+          htable = new HTable(conf, tableName);
+          this.tableNameToHTableMap.put(tableName, htable);
         }
       }
     }
@@ -435,7 +450,7 @@ public class HTableMultiplexer {
         HRegionLocation oldLoc) throws IOException {
       Put failedPut = failedPutStatus.getPut();
       // The currentPut is failed. So get the table name for the currentPut.
-      byte[] tableName = failedPutStatus.getRegionInfo().getTableName();
+      TableName tableName = failedPutStatus.getRegionInfo().getTableName();
       // Decrease the retry count
       int retryCount = failedPutStatus.getRetryCount() - 1;
       

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java Thu Aug  8 04:19:49 2013
@@ -23,6 +23,7 @@ import com.google.protobuf.ServiceExcept
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -341,6 +342,11 @@ public class HTablePool implements Close
     }
 
     @Override
+    public TableName getName() {
+      return table.getName();
+    }
+
+    @Override
     public Configuration getConfiguration() {
       checkState();
       return table.getConfiguration();

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.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.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
@@ -76,10 +77,10 @@ public class MetaScanner {
    * @throws IOException e
    */
   public static void metaScan(Configuration configuration, HConnection connection,
-      MetaScannerVisitor visitor, byte [] userTableName)
+      MetaScannerVisitor visitor, TableName userTableName)
   throws IOException {
     metaScan(configuration, connection, visitor, userTableName, null, Integer.MAX_VALUE,
-        HConstants.META_TABLE_NAME);
+        TableName.META_TABLE_NAME);
   }
 
   /**
@@ -98,11 +99,11 @@ public class MetaScanner {
    * @throws IOException e
    */
   public static void metaScan(Configuration configuration,
-      MetaScannerVisitor visitor, byte [] userTableName, byte[] row,
+      MetaScannerVisitor visitor, TableName userTableName, byte[] row,
       int rowLimit)
   throws IOException {
     metaScan(configuration, null, visitor, userTableName, row, rowLimit,
-      HConstants.META_TABLE_NAME);
+      TableName.META_TABLE_NAME);
   }
 
   /**
@@ -123,15 +124,15 @@ public class MetaScanner {
    * @throws IOException e
    */
   public static void metaScan(Configuration configuration, HConnection connection,
-      final MetaScannerVisitor visitor, final byte[] tableName,
-      final byte[] row, final int rowLimit, final byte[] metaTableName)
+      final MetaScannerVisitor visitor, final TableName tableName,
+      final byte[] row, final int rowLimit, final TableName metaTableName)
   throws IOException {
     int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE;
     HTable metaTable;
     if (connection == null) {
-      metaTable = new HTable(configuration, HConstants.META_TABLE_NAME, null);
+      metaTable = new HTable(configuration, TableName.META_TABLE_NAME, null);
     } else {
-      metaTable = new HTable(HConstants.META_TABLE_NAME, connection, null);      
+      metaTable = new HTable(TableName.META_TABLE_NAME, connection, null);
     }
     // Calculate startrow for scan.
     byte[] startRow;
@@ -142,17 +143,18 @@ public class MetaScanner {
         byte[] searchRow = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
         Result startRowResult = metaTable.getRowOrBefore(searchRow, HConstants.CATALOG_FAMILY);
         if (startRowResult == null) {
-          throw new TableNotFoundException("Cannot find row in .META. for table: " +
-            Bytes.toString(tableName) + ", row=" + Bytes.toStringBinary(searchRow));
+          throw new TableNotFoundException("Cannot find row in "+ TableName
+              .META_TABLE_NAME.getNameAsString()+" for table: "
+              + tableName + ", row=" + Bytes.toStringBinary(searchRow));
         }
         HRegionInfo regionInfo = getHRegionInfo(startRowResult);
         if (regionInfo == null) {
           throw new IOException("HRegionInfo was null or empty in Meta for " +
-            Bytes.toString(tableName) + ", row=" + Bytes.toStringBinary(searchRow));
+            tableName + ", row=" + Bytes.toStringBinary(searchRow));
         }
         byte[] rowBefore = regionInfo.getStartKey();
         startRow = HRegionInfo.createRegionName(tableName, rowBefore, HConstants.ZEROES, false);
-      } else if (tableName == null || tableName.length == 0) {
+      } else if (tableName == null || tableName.getName().length == 0) {
         // Full META scan
         startRow = HConstants.EMPTY_START_ROW;
       } else {
@@ -165,7 +167,7 @@ public class MetaScanner {
         HConstants.DEFAULT_HBASE_META_SCANNER_CACHING));
       scan.setCaching(rows);
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Scanning " + Bytes.toString(metaTableName) + " starting at row=" +
+        LOG.trace("Scanning " + metaTableName.getNameAsString() + " starting at row=" +
           Bytes.toStringBinary(startRow) + " for max=" + rowUpperLimit + " with caching=" + rows);
       }
       // Run the scan
@@ -267,11 +269,11 @@ public class MetaScanner {
    * @throws IOException
    */
   public static NavigableMap<HRegionInfo, ServerName> allTableRegions(Configuration conf,
-      HConnection connection,
-      final byte [] tablename, final boolean offlined) throws IOException {
+      HConnection connection, final TableName tableName,
+      final boolean offlined) throws IOException {
     final NavigableMap<HRegionInfo, ServerName> regions =
       new TreeMap<HRegionInfo, ServerName>();
-    MetaScannerVisitor visitor = new TableMetaScannerVisitor(tablename) {
+    MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
       @Override
       public boolean processRowInternal(Result rowResult) throws IOException {
         HRegionInfo info = getHRegionInfo(rowResult);
@@ -280,7 +282,7 @@ public class MetaScanner {
         return true;
       }
     };
-    metaScan(conf, connection, visitor, tablename);
+    metaScan(conf, connection, visitor, tableName);
     return regions;
   }
 
@@ -340,9 +342,9 @@ public class MetaScanner {
    * META entries for daughters are available during splits.
    */
   public static abstract class TableMetaScannerVisitor extends DefaultMetaScannerVisitor {
-    private byte[] tableName;
+    private TableName tableName;
 
-    public TableMetaScannerVisitor(byte[] tableName) {
+    public TableMetaScannerVisitor(TableName tableName) {
       super();
       this.tableName = tableName;
     }
@@ -353,7 +355,7 @@ public class MetaScanner {
       if (info == null) {
         return true;
       }
-      if (!(Bytes.equals(info.getTableName(), tableName))) {
+      if (!(info.getTableName().equals(tableName))) {
         return false;
       }
       return super.processRow(rowResult);

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java Thu Aug  8 04:19:49 2013
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -42,7 +43,7 @@ import com.google.protobuf.ServiceExcept
 class MultiServerCallable<R> extends RegionServerCallable<MultiResponse> {
   private final MultiAction<R> multi;
 
-  MultiServerCallable(final HConnection connection, final byte [] tableName,
+  MultiServerCallable(final HConnection connection, final TableName tableName,
       final HRegionLocation location, final MultiAction<R> multi) {
     super(connection, tableName, null);
     this.multi = multi;
@@ -120,4 +121,4 @@ class MultiServerCallable<R> extends Reg
     // Use the location we were given in the constructor rather than go look it up.
     setStub(getConnection().getClient(getLocation().getServerName()));
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java Thu Aug  8 04:19:49 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -44,7 +45,7 @@ public abstract class RegionServerCallab
   // Public because used outside of this package over in ipc.
   static final Log LOG = LogFactory.getLog(RegionServerCallable.class);
   private final HConnection connection;
-  private final byte [] tableName;
+  private final TableName tableName;
   private final byte [] row;
   private HRegionLocation location;
   private ClientService.BlockingInterface stub;
@@ -56,7 +57,7 @@ public abstract class RegionServerCallab
    * @param tableName Table name to which <code>row</code> belongs.
    * @param row The row we want in <code>tableName</code>.
    */
-  public RegionServerCallable(HConnection connection, byte [] tableName, byte [] row) {
+  public RegionServerCallable(HConnection connection, TableName tableName, byte [] row) {
     this.connection = connection;
     this.tableName = tableName;
     this.row = row;
@@ -71,7 +72,7 @@ public abstract class RegionServerCallab
   public void prepare(final boolean reload) throws IOException {
     this.location = connection.getRegionLocation(tableName, row, reload);
     if (this.location == null) {
-      throw new IOException("Failed to find location, tableName=" + Bytes.toString(tableName) +
+      throw new IOException("Failed to find location, tableName=" + tableName +
         ", row=" + Bytes.toString(row) + ", reload=" + reload);
     }
     setStub(getConnection().getClient(getLocation().getServerName()));
@@ -100,7 +101,7 @@ public abstract class RegionServerCallab
     this.location = location;
   }
 
-  public byte [] getTableName() {
+  public TableName getTableName() {
     return this.tableName;
   }
 
@@ -129,7 +130,7 @@ public abstract class RegionServerCallab
 
   @Override
   public String getExceptionMessageAdditionalDetail() {
-    return "row '" + Bytes.toString(row) + "' on table '" + Bytes.toString(tableName);
+    return "row '" + Bytes.toString(row) + "' on table '" + tableName;
   }
 
   @Override

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java Thu Aug  8 04:19:49 2013
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionLocation;
 
 /**
@@ -46,7 +47,7 @@ interface Registry {
    * @param enabled Return true if table is enabled
    * @throws IOException
    */
-  boolean isTableOnlineState(byte [] tableName, boolean enabled) throws IOException;
+  boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException;
 
   /**
    * @return Count of 'running' regionservers

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java Thu Aug  8 04:19:49 2013
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -89,7 +90,7 @@ public class ScannerCallable extends Reg
    * @param scanMetrics the ScanMetrics to used, if it is null, ScannerCallable
    * won't collect metrics
    */
-  public ScannerCallable (HConnection connection, byte [] tableName, Scan scan,
+  public ScannerCallable (HConnection connection, TableName tableName, Scan scan,
     ScanMetrics scanMetrics) {
     super(connection, tableName, scan.getStartRow());
     this.scan = scan;

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java Thu Aug  8 04:19:49 2013
@@ -40,7 +40,7 @@ public class UnmodifyableHTableDescripto
    * @param desc
    */
   UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
-    super(desc.getName(), getUnmodifyableFamilies(desc), desc.getValues());
+    super(desc.getTableName(), getUnmodifyableFamilies(desc), desc.getValues());
   }
 
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java Thu Aug  8 04:19:49 2013
@@ -21,10 +21,10 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
@@ -95,15 +95,14 @@ class ZooKeeperRegistry implements Regis
   }
 
   @Override
-  public boolean isTableOnlineState(byte [] tableName, boolean enabled)
+  public boolean isTableOnlineState(TableName tableName, boolean enabled)
   throws IOException {
-    String tableNameStr = Bytes.toString(tableName);
     ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
     try {
       if (enabled) {
-        return ZKTableReadOnly.isEnabledTable(zkw, tableNameStr);
+        return ZKTableReadOnly.isEnabledTable(zkw, tableName);
       }
-      return ZKTableReadOnly.isDisabledTable(zkw, tableNameStr);
+      return ZKTableReadOnly.isDisabledTable(zkw, tableName);
     } catch (KeeperException e) {
       throw new IOException("Enable/Disable failed", e);
     } finally {