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 [2/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/ClientScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java Thu Aug  8 04:19:49 2013
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFac
 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.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
@@ -63,7 +64,7 @@ public class ClientScanner extends Abstr
     private ScanMetrics scanMetrics = null;
     private final long maxScannerResultSize;
     private final HConnection connection;
-    private final byte[] tableName;
+    private final TableName tableName;
     private final int scannerTimeout;
     private boolean scanMetricsPublished = false;
     private RpcRetryingCaller<Result []> caller;
@@ -79,7 +80,7 @@ public class ClientScanner extends Abstr
      * @throws IOException
      */
     public ClientScanner(final Configuration conf, final Scan scan,
-        final byte[] tableName) throws IOException {
+        final TableName tableName) throws IOException {
       this(conf, scan, tableName, HConnectionManager.getConnection(conf));
     }
 
@@ -94,7 +95,7 @@ public class ClientScanner extends Abstr
      * @param connection Connection identifying the cluster
      * @throws IOException
      */
-  public ClientScanner(final Configuration conf, final Scan scan, final byte[] tableName,
+  public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
       HConnection connection) throws IOException {
     this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf));
   }
@@ -108,10 +109,10 @@ public class ClientScanner extends Abstr
    * @param connection Connection identifying the cluster
    * @throws IOException
    */
-  public ClientScanner(final Configuration conf, final Scan scan, final byte[] tableName,
+  public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
       HConnection connection, RpcRetryingCallerFactory rpcFactory) throws IOException {
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Scan table=" + Bytes.toString(tableName)
+        LOG.trace("Scan table=" + tableName
             + ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
       }
       this.scan = scan;
@@ -156,7 +157,7 @@ public class ClientScanner extends Abstr
       return this.connection;
     }
 
-    protected byte[] getTableName() {
+    protected TableName getTableName() {
       return this.tableName;
     }
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Thu Aug  8 04:19:49 2013
@@ -22,6 +22,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.SocketTimeoutException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
@@ -36,6 +37,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -43,6 +45,7 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionException;
@@ -83,7 +86,10 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest;
@@ -248,7 +254,7 @@ public class HBaseAdmin implements Abort
    * @return True if table exists already.
    * @throws IOException
    */
-  public boolean tableExists(final String tableName)
+  public boolean tableExists(final TableName tableName)
   throws IOException {
     boolean b = false;
     CatalogTracker ct = getCatalogTracker();
@@ -260,14 +266,14 @@ public class HBaseAdmin implements Abort
     return b;
   }
 
-  /**
-   * @param tableName Table to check.
-   * @return True if table exists already.
-   * @throws IOException
-   */
-  public boolean tableExists(final byte [] tableName)
+  public boolean tableExists(final byte[] tableName)
+  throws IOException {
+    return tableExists(TableName.valueOf(tableName));
+  }
+
+  public boolean tableExists(final String tableName)
   throws IOException {
-    return tableExists(Bytes.toString(tableName));
+    return tableExists(TableName.valueOf(tableName));
   }
 
   /**
@@ -296,7 +302,7 @@ public class HBaseAdmin implements Abort
     List<HTableDescriptor> matched = new LinkedList<HTableDescriptor>();
     HTableDescriptor[] tables = listTables();
     for (HTableDescriptor table : tables) {
-      if (pattern.matcher(table.getNameAsString()).matches()) {
+      if (pattern.matcher(table.getTableName().getNameAsString()).matches()) {
         matched.add(table);
       }
     }
@@ -323,11 +329,16 @@ public class HBaseAdmin implements Abort
    * @throws TableNotFoundException
    * @throws IOException if a remote or network exception occurs
    */
-  public HTableDescriptor getTableDescriptor(final byte [] tableName)
+  public HTableDescriptor getTableDescriptor(final TableName tableName)
   throws TableNotFoundException, IOException {
     return this.connection.getHTableDescriptor(tableName);
   }
 
+  public HTableDescriptor getTableDescriptor(final byte[] tableName)
+  throws TableNotFoundException, IOException {
+    return getTableDescriptor(TableName.valueOf(tableName));
+  }
+
   private long getPauseTime(int tries) {
     int triesCount = tries;
     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
@@ -381,7 +392,6 @@ public class HBaseAdmin implements Abort
   public void createTable(HTableDescriptor desc, byte [] startKey,
       byte [] endKey, int numRegions)
   throws IOException {
-    HTableDescriptor.isLegalTableName(desc.getName());
     if(numRegions < 3) {
       throw new IllegalArgumentException("Must create at least three regions");
     } else if(Bytes.compareTo(startKey, endKey) >= 0) {
@@ -417,11 +427,10 @@ public class HBaseAdmin implements Abort
    */
   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
   throws IOException {
-    HTableDescriptor.isLegalTableName(desc.getName());
     try {
       createTableAsync(desc, splitKeys);
     } catch (SocketTimeoutException ste) {
-      LOG.warn("Creating " + desc.getNameAsString() + " took too long", ste);
+      LOG.warn("Creating " + desc.getTableName() + " took too long", ste);
     }
     int numRegs = splitKeys == null ? 1 : splitKeys.length + 1;
     int prevRegCount = 0;
@@ -439,7 +448,7 @@ public class HBaseAdmin implements Abort
               LOG.warn("No serialized HRegionInfo in " + rowResult);
               return true;
             }
-            if (!(Bytes.equals(info.getTableName(), desc.getName()))) {
+            if (!info.getTableName().equals(desc.getTableName())) {
               return false;
             }
             ServerName serverName = HRegionInfo.getServerName(rowResult);
@@ -451,7 +460,7 @@ public class HBaseAdmin implements Abort
             return true;
           }
         };
-        MetaScanner.metaScan(conf, connection, visitor, desc.getName());
+        MetaScanner.metaScan(conf, connection, visitor, desc.getTableName());
         if (actualRegCount.get() != numRegs) {
           if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
             throw new RegionOfflineException("Only " + actualRegCount.get() +
@@ -472,7 +481,7 @@ public class HBaseAdmin implements Abort
           doneWithMetaScan = true;
           tries = -1;
         }
-      } else if (isTableEnabled(desc.getName())) {
+      } else if (isTableEnabled(desc.getTableName())) {
         return;
       } else {
         try { // Sleep
@@ -485,7 +494,7 @@ public class HBaseAdmin implements Abort
     }
     throw new TableNotEnabledException(
       "Retries exhausted while still waiting for table: "
-      + desc.getNameAsString() + " to be enabled");
+      + desc.getTableName() + " to be enabled");
   }
 
   /**
@@ -507,7 +516,9 @@ public class HBaseAdmin implements Abort
   public void createTableAsync(
     final HTableDescriptor desc, final byte [][] splitKeys)
   throws IOException {
-    HTableDescriptor.isLegalTableName(desc.getName());
+    if(desc.getTableName() == null) {
+      throw new IllegalArgumentException("TableName cannot be null");
+    }
     if(splitKeys != null && splitKeys.length > 0) {
       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
       // Verify there are no duplicate split keys
@@ -536,15 +547,12 @@ public class HBaseAdmin implements Abort
     });
   }
 
-  /**
-   * Deletes a table.
-   * Synchronous operation.
-   *
-   * @param tableName name of table to delete
-   * @throws IOException if a remote or network exception occurs
-   */
   public void deleteTable(final String tableName) throws IOException {
-    deleteTable(Bytes.toBytes(tableName));
+    deleteTable(TableName.valueOf(tableName));
+  }
+
+  public void deleteTable(final byte[] tableName) throws IOException {
+    deleteTable(TableName.valueOf(tableName));
   }
 
   /**
@@ -554,8 +562,7 @@ public class HBaseAdmin implements Abort
    * @param tableName name of table to delete
    * @throws IOException if a remote or network exception occurs
    */
-  public void deleteTable(final byte [] tableName) throws IOException {
-    HTableDescriptor.isLegalTableName(tableName);
+  public void deleteTable(final TableName tableName) throws IOException {
     HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
     boolean tableExists = true;
 
@@ -626,11 +633,11 @@ public class HBaseAdmin implements Abort
 
     if (tableExists) {
       throw new IOException("Retries exhausted, it took too long to wait"+
-        " for the table " + Bytes.toString(tableName) + " to be deleted.");
+        " for the table " + tableName + " to be deleted.");
     }
     // Delete cached information to prevent clients from using old locations
     this.connection.clearRegionCache(tableName);
-    LOG.info("Deleted " + Bytes.toString(tableName));
+    LOG.info("Deleted " + tableName);
   }
 
   /**
@@ -665,9 +672,9 @@ public class HBaseAdmin implements Abort
     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
     for (HTableDescriptor table : listTables(pattern)) {
       try {
-        deleteTable(table.getName());
+        deleteTable(table.getTableName());
       } catch (IOException ex) {
-        LOG.info("Failed to delete table " + table.getNameAsString(), ex);
+        LOG.info("Failed to delete table " + table.getTableName(), ex);
         failed.add(table);
       }
     }
@@ -675,11 +682,6 @@ public class HBaseAdmin implements Abort
   }
 
 
-  public void enableTable(final String tableName)
-  throws IOException {
-    enableTable(Bytes.toBytes(tableName));
-  }
-
   /**
    * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
    * and {@link #isTableEnabled(byte[])} instead.
@@ -693,14 +695,24 @@ public class HBaseAdmin implements Abort
    * @see #disableTable(byte[])
    * @see #enableTableAsync(byte[])
    */
-  public void enableTable(final byte [] tableName)
+  public void enableTable(final TableName tableName)
   throws IOException {
     enableTableAsync(tableName);
 
     // Wait until all regions are enabled
     waitUntilTableIsEnabled(tableName);
 
-    LOG.info("Enabled table " + Bytes.toString(tableName));
+    LOG.info("Enabled table " + tableName);
+  }
+
+  public void enableTable(final byte[] tableName)
+  throws IOException {
+    enableTable(TableName.valueOf(tableName));
+  }
+
+  public void enableTable(final String tableName)
+  throws IOException {
+    enableTable(TableName.valueOf(tableName));
   }
 
   /**
@@ -710,7 +722,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs or
    *    table is not enabled after the retries period.
    */
-  private void waitUntilTableIsEnabled(final byte[] tableName) throws IOException {
+  private void waitUntilTableIsEnabled(final TableName tableName) throws IOException {
     boolean enabled = false;
     long start = EnvironmentEdgeManager.currentTimeMillis();
     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
@@ -721,7 +733,7 @@ public class HBaseAdmin implements Abort
       long sleep = getPauseTime(tries);
       if (LOG.isDebugEnabled()) {
         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
-          "enabled in " + Bytes.toString(tableName));
+          "enabled in " + tableName);
       }
       try {
         Thread.sleep(sleep);
@@ -734,16 +746,11 @@ public class HBaseAdmin implements Abort
     }
     if (!enabled) {
       long msec = EnvironmentEdgeManager.currentTimeMillis() - start;
-      throw new IOException("Table '" + Bytes.toString(tableName) +
+      throw new IOException("Table '" + tableName +
         "' not yet enabled, after " + msec + "ms.");
     }
   }
 
-  public void enableTableAsync(final String tableName)
-  throws IOException {
-    enableTableAsync(Bytes.toBytes(tableName));
-  }
-
   /**
    * Brings a table on-line (enables it).  Method returns immediately though
    * enable of table may take some time to complete, especially if the table
@@ -754,13 +761,13 @@ public class HBaseAdmin implements Abort
    * @throws IOException
    * @since 0.90.0
    */
-  public void enableTableAsync(final byte [] tableName)
+  public void enableTableAsync(final TableName tableName)
   throws IOException {
-    HTableDescriptor.isLegalTableName(tableName);
+    TableName.isLegalFullyQualifiedTableName(tableName.getName());
     executeCallable(new MasterAdminCallable<Void>(getConnection()) {
       @Override
       public Void call() throws ServiceException {
-        LOG.info("Started enable of " + Bytes.toString(tableName));
+        LOG.info("Started enable of " + tableName);
         EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName);
         masterAdmin.enableTable(null,req);
         return null;
@@ -768,6 +775,16 @@ public class HBaseAdmin implements Abort
     });
   }
 
+  public void enableTableAsync(final byte[] tableName)
+  throws IOException {
+    enableTable(TableName.valueOf(tableName));
+  }
+
+  public void enableTableAsync(final String tableName)
+  throws IOException {
+    enableTableAsync(TableName.valueOf(tableName));
+  }
+
   /**
    * Enable tables matching the passed in pattern and wait on completion.
    *
@@ -797,11 +814,11 @@ public class HBaseAdmin implements Abort
   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
     for (HTableDescriptor table : listTables(pattern)) {
-      if (isTableDisabled(table.getName())) {
+      if (isTableDisabled(table.getTableName())) {
         try {
-          enableTable(table.getName());
+          enableTable(table.getTableName());
         } catch (IOException ex) {
-          LOG.info("Failed to enable table " + table.getNameAsString(), ex);
+          LOG.info("Failed to enable table " + table.getTableName(), ex);
           failed.add(table);
         }
       }
@@ -809,10 +826,6 @@ public class HBaseAdmin implements Abort
     return failed.toArray(new HTableDescriptor[failed.size()]);
   }
 
-  public void disableTableAsync(final String tableName) throws IOException {
-    disableTableAsync(Bytes.toBytes(tableName));
-  }
-
   /**
    * Starts the disable of a table.  If it is being served, the master
    * will tell the servers to stop serving it.  This method returns immediately.
@@ -826,12 +839,12 @@ public class HBaseAdmin implements Abort
    * @see #isTableEnabled(byte[])
    * @since 0.90.0
    */
-  public void disableTableAsync(final byte [] tableName) throws IOException {
-    HTableDescriptor.isLegalTableName(tableName);
+  public void disableTableAsync(final TableName tableName) throws IOException {
+    TableName.isLegalFullyQualifiedTableName(tableName.getName());
     executeCallable(new MasterAdminCallable<Void>(getConnection()) {
       @Override
       public Void call() throws ServiceException {
-        LOG.info("Started disable of " + Bytes.toString(tableName));
+        LOG.info("Started disable of " + tableName);
         DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName);
         masterAdmin.disableTable(null,req);
         return null;
@@ -839,9 +852,12 @@ public class HBaseAdmin implements Abort
     });
   }
 
-  public void disableTable(final String tableName)
-  throws IOException {
-    disableTable(Bytes.toBytes(tableName));
+  public void disableTableAsync(final byte[] tableName) throws IOException {
+    disableTableAsync(TableName.valueOf(tableName));
+  }
+
+  public void disableTableAsync(final String tableName) throws IOException {
+    disableTableAsync(TableName.valueOf(tableName));
   }
 
   /**
@@ -855,7 +871,7 @@ public class HBaseAdmin implements Abort
    * TableNotFoundException means the table doesn't exist.
    * TableNotEnabledException means the table isn't in enabled state.
    */
-  public void disableTable(final byte [] tableName)
+  public void disableTable(final TableName tableName)
   throws IOException {
     disableTableAsync(tableName);
     // Wait until table is disabled
@@ -868,7 +884,7 @@ public class HBaseAdmin implements Abort
       long sleep = getPauseTime(tries);
       if (LOG.isDebugEnabled()) {
         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
-          "disabled in " + Bytes.toString(tableName));
+          "disabled in " + tableName);
       }
       try {
         Thread.sleep(sleep);
@@ -881,9 +897,19 @@ public class HBaseAdmin implements Abort
     }
     if (!disabled) {
       throw new RegionException("Retries exhausted, it took too long to wait"+
-        " for the table " + Bytes.toString(tableName) + " to be disabled.");
+        " for the table " + tableName + " to be disabled.");
     }
-    LOG.info("Disabled " + Bytes.toString(tableName));
+    LOG.info("Disabled " + tableName);
+  }
+
+  public void disableTable(final byte[] tableName)
+  throws IOException {
+    disableTable(TableName.valueOf(tableName));
+  }
+
+  public void disableTable(final String tableName)
+  throws IOException {
+    disableTable(TableName.valueOf(tableName));
   }
 
   /**
@@ -917,11 +943,11 @@ public class HBaseAdmin implements Abort
   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
     for (HTableDescriptor table : listTables(pattern)) {
-      if (isTableEnabled(table.getName())) {
+      if (isTableEnabled(table.getTableName())) {
         try {
-          disableTable(table.getName());
+          disableTable(table.getTableName());
         } catch (IOException ex) {
-          LOG.info("Failed to disable table " + table.getNameAsString(), ex);
+          LOG.info("Failed to disable table " + table.getTableName(), ex);
           failed.add(table);
         }
       }
@@ -934,74 +960,52 @@ public class HBaseAdmin implements Abort
    * @return true if table is on-line
    * @throws IOException if a remote or network exception occurs
    */
-  public boolean isTableEnabled(String tableName) throws IOException {
-    return isTableEnabled(Bytes.toBytes(tableName));
+  public boolean isTableEnabled(TableName tableName) throws IOException {
+    return connection.isTableEnabled(tableName);
   }
-  /**
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException if a remote or network exception occurs
-   */
+
   public boolean isTableEnabled(byte[] tableName) throws IOException {
-    if (!HTableDescriptor.isMetaTable(tableName)) {
-      HTableDescriptor.isLegalTableName(tableName);
-    }
-    return connection.isTableEnabled(tableName);
+    return isTableEnabled(TableName.valueOf(tableName));
   }
 
-  /**
-   * @param tableName name of table to check
-   * @return true if table is off-line
-   * @throws IOException if a remote or network exception occurs
-   */
-  public boolean isTableDisabled(final String tableName) throws IOException {
-    return isTableDisabled(Bytes.toBytes(tableName));
+  public boolean isTableEnabled(String tableName) throws IOException {
+    return isTableEnabled(TableName.valueOf(tableName));
   }
 
+
+
   /**
    * @param tableName name of table to check
    * @return true if table is off-line
    * @throws IOException if a remote or network exception occurs
    */
-  public boolean isTableDisabled(byte[] tableName) throws IOException {
-    if (!HTableDescriptor.isMetaTable(tableName)) {
-      HTableDescriptor.isLegalTableName(tableName);
-    }
+  public boolean isTableDisabled(TableName tableName) throws IOException {
     return connection.isTableDisabled(tableName);
   }
 
+  public boolean isTableDisabled(byte[] tableName) throws IOException {
+    return isTableDisabled(TableName.valueOf(tableName));
+  }
+
+  public boolean isTableDisabled(String tableName) throws IOException {
+    return isTableDisabled(TableName.valueOf(tableName));
+  }
+
   /**
    * @param tableName name of table to check
    * @return true if all regions of the table are available
    * @throws IOException if a remote or network exception occurs
    */
-  public boolean isTableAvailable(byte[] tableName) throws IOException {
+  public boolean isTableAvailable(TableName tableName) throws IOException {
     return connection.isTableAvailable(tableName);
   }
 
-  /**
-   * @param tableName name of table to check
-   * @return true if all regions of the table are available
-   * @throws IOException if a remote or network exception occurs
-   */
-  public boolean isTableAvailable(String tableName) throws IOException {
-    return connection.isTableAvailable(Bytes.toBytes(tableName));
+  public boolean isTableAvailable(byte[] tableName) throws IOException {
+    return isTableAvailable(TableName.valueOf(tableName));
   }
-  
-  /**
-   * Use this api to check if the table has been created with the specified number of 
-   * splitkeys which was used while creating the given table.
-   * Note : If this api is used after a table's region gets splitted, the api may return
-   * false.
-   * @param tableName
-   *          name of table to check
-   * @param splitKeys
-   *          keys to check if the table has been created with all split keys
-   * @throws IOException
-   *           if a remote or network excpetion occurs
-   */
-  public boolean isTableAvailable(String tableName, byte[][] splitKeys) throws IOException {
-    return connection.isTableAvailable(Bytes.toBytes(tableName), splitKeys);
+
+  public boolean isTableAvailable(String tableName) throws IOException {
+    return isTableAvailable(TableName.valueOf(tableName));
   }
   
   /**
@@ -1016,25 +1020,34 @@ public class HBaseAdmin implements Abort
    * @throws IOException
    *           if a remote or network excpetion occurs
    */
-  public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException {
+  public boolean isTableAvailable(TableName tableName,
+                                  byte[][] splitKeys) throws IOException {
     return connection.isTableAvailable(tableName, splitKeys);
   }
 
+  public boolean isTableAvailable(byte[] tableName,
+                                  byte[][] splitKeys) throws IOException {
+    return isTableAvailable(TableName.valueOf(tableName), splitKeys);
+  }
+
+  public boolean isTableAvailable(String tableName,
+                                  byte[][] splitKeys) throws IOException {
+    return isTableAvailable(TableName.valueOf(tableName), splitKeys);
+  }
+
   /**
    * Get the status of alter command - indicates how many regions have received
    * the updated schema Asynchronous operation.
    *
-   * @param tableName
-   *          name of the table to get the status of
+   * @param tableName TableName instance
    * @return Pair indicating the number of regions updated Pair.getFirst() is the
    *         regions that are yet to be updated Pair.getSecond() is the total number
    *         of regions of the table
    * @throws IOException
    *           if a remote or network exception occurs
    */
-  public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
+  public Pair<Integer, Integer> getAlterStatus(final TableName tableName)
   throws IOException {
-    HTableDescriptor.isLegalTableName(tableName);
     return executeCallable(new MasterMonitorCallable<Pair<Integer, Integer>>(getConnection()) {
       @Override
       public Pair<Integer, Integer> call() throws ServiceException {
@@ -1049,6 +1062,37 @@ public class HBaseAdmin implements Abort
   }
 
   /**
+   * Get the status of alter command - indicates how many regions have received
+   * the updated schema Asynchronous operation.
+   *
+   * @param tableName
+   *          name of the table to get the status of
+   * @return Pair indicating the number of regions updated Pair.getFirst() is the
+   *         regions that are yet to be updated Pair.getSecond() is the total number
+   *         of regions of the table
+   * @throws IOException
+   *           if a remote or network exception occurs
+   */
+  public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
+   throws IOException {
+    return getAlterStatus(TableName.valueOf(tableName));
+  }
+
+  /**
+   * Add a column to an existing table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of the table to add column to
+   * @param column column descriptor of column to be added
+   * @throws IOException if a remote or network exception occurs
+   */
+  public void addColumn(final byte[] tableName, HColumnDescriptor column)
+  throws IOException {
+    addColumn(TableName.valueOf(tableName), column);
+  }
+
+
+  /**
    * Add a column to an existing table.
    * Asynchronous operation.
    *
@@ -1058,7 +1102,7 @@ public class HBaseAdmin implements Abort
    */
   public void addColumn(final String tableName, HColumnDescriptor column)
   throws IOException {
-    addColumn(Bytes.toBytes(tableName), column);
+    addColumn(TableName.valueOf(tableName), column);
   }
 
   /**
@@ -1069,7 +1113,7 @@ public class HBaseAdmin implements Abort
    * @param column column descriptor of column to be added
    * @throws IOException if a remote or network exception occurs
    */
-  public void addColumn(final byte [] tableName, final HColumnDescriptor column)
+  public void addColumn(final TableName tableName, final HColumnDescriptor column)
   throws IOException {
     executeCallable(new MasterAdminCallable<Void>(getConnection()) {
       @Override
@@ -1089,9 +1133,22 @@ public class HBaseAdmin implements Abort
    * @param columnName name of column to be deleted
    * @throws IOException if a remote or network exception occurs
    */
+  public void deleteColumn(final byte[] tableName, final String columnName)
+  throws IOException {
+    deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
+  }
+
+  /**
+   * Delete a column from a table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param columnName name of column to be deleted
+   * @throws IOException if a remote or network exception occurs
+   */
   public void deleteColumn(final String tableName, final String columnName)
   throws IOException {
-    deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName));
+    deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
   }
 
   /**
@@ -1102,7 +1159,7 @@ public class HBaseAdmin implements Abort
    * @param columnName name of column to be deleted
    * @throws IOException if a remote or network exception occurs
    */
-  public void deleteColumn(final byte [] tableName, final byte [] columnName)
+  public void deleteColumn(final TableName tableName, final byte [] columnName)
   throws IOException {
     executeCallable(new MasterAdminCallable<Void>(getConnection()) {
       @Override
@@ -1124,7 +1181,20 @@ public class HBaseAdmin implements Abort
    */
   public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
   throws IOException {
-    modifyColumn(Bytes.toBytes(tableName), descriptor);
+    modifyColumn(TableName.valueOf(tableName), descriptor);
+  }
+
+  /**
+   * Modify an existing column family on a table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param descriptor new column descriptor to use
+   * @throws IOException if a remote or network exception occurs
+   */
+  public void modifyColumn(final byte[] tableName, HColumnDescriptor descriptor)
+  throws IOException {
+    modifyColumn(TableName.valueOf(tableName), descriptor);
   }
 
 
@@ -1137,7 +1207,7 @@ public class HBaseAdmin implements Abort
    * @param descriptor new column descriptor to use
    * @throws IOException if a remote or network exception occurs
    */
-  public void modifyColumn(final byte [] tableName, final HColumnDescriptor descriptor)
+  public void modifyColumn(final TableName tableName, final HColumnDescriptor descriptor)
   throws IOException {
     executeCallable(new MasterAdminCallable<Void>(getConnection()) {
       @Override
@@ -1286,7 +1356,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs
    * @throws InterruptedException
    */
-  public void flush(final byte [] tableNameOrRegionName)
+  public void flush(final byte[] tableNameOrRegionName)
   throws IOException, InterruptedException {
     CatalogTracker ct = getCatalogTracker();
     try {
@@ -1299,7 +1369,8 @@ public class HBaseAdmin implements Abort
           flush(regionServerPair.getSecond(), regionServerPair.getFirst());
         }
       } else {
-        final String tableName = tableNameString(tableNameOrRegionName, ct);
+        final TableName tableName = checkTableExists(
+            TableName.valueOf(tableNameOrRegionName), ct);
         List<Pair<HRegionInfo, ServerName>> pairs =
           MetaReader.getTableRegionsAndLocations(ct,
               tableName);
@@ -1354,7 +1425,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs
    * @throws InterruptedException
    */
-  public void compact(final byte [] tableNameOrRegionName)
+  public void compact(final byte[] tableNameOrRegionName)
   throws IOException, InterruptedException {
     compact(tableNameOrRegionName, null, false);
   }
@@ -1382,7 +1453,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs
    * @throws InterruptedException
    */
-  public void compact(final byte [] tableNameOrRegionName, final byte[] columnFamily)
+  public void compact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
   throws IOException, InterruptedException {
     compact(tableNameOrRegionName, columnFamily, false);
   }
@@ -1408,7 +1479,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs
    * @throws InterruptedException
    */
-  public void majorCompact(final byte [] tableNameOrRegionName)
+  public void majorCompact(final byte[] tableNameOrRegionName)
   throws IOException, InterruptedException {
     compact(tableNameOrRegionName, null, true);
   }
@@ -1437,7 +1508,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs
    * @throws InterruptedException
    */
-  public void majorCompact(final byte [] tableNameOrRegionName,
+  public void majorCompact(final byte[] tableNameOrRegionName,
     final byte[] columnFamily) throws IOException, InterruptedException {
     compact(tableNameOrRegionName, columnFamily, true);
   }
@@ -1452,7 +1523,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs
    * @throws InterruptedException
    */
-  private void compact(final byte [] tableNameOrRegionName,
+  private void compact(final byte[] tableNameOrRegionName,
     final byte[] columnFamily,final boolean major)
   throws IOException, InterruptedException {
     CatalogTracker ct = getCatalogTracker();
@@ -1466,7 +1537,8 @@ public class HBaseAdmin implements Abort
           compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
         }
       } else {
-        final String tableName = tableNameString(tableNameOrRegionName, ct);
+        final TableName tableName =
+            checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
         List<Pair<HRegionInfo, ServerName>> pairs =
           MetaReader.getTableRegionsAndLocations(ct,
               tableName);
@@ -1749,7 +1821,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs
    * @throws InterruptedException
    */
-  public void split(final byte [] tableNameOrRegionName)
+  public void split(final byte[] tableNameOrRegionName)
   throws IOException, InterruptedException {
     split(tableNameOrRegionName, null);
   }
@@ -1768,7 +1840,7 @@ public class HBaseAdmin implements Abort
    * @throws IOException if a remote or network exception occurs
    * @throws InterruptedException interrupt exception occurred
    */
-  public void split(final byte [] tableNameOrRegionName,
+  public void split(final byte[] tableNameOrRegionName,
       final byte [] splitPoint) throws IOException, InterruptedException {
     CatalogTracker ct = getCatalogTracker();
     try {
@@ -1781,7 +1853,8 @@ public class HBaseAdmin implements Abort
           split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
         }
       } else {
-        final String tableName = tableNameString(tableNameOrRegionName, ct);
+        final TableName tableName =
+            checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
         List<Pair<HRegionInfo, ServerName>> pairs =
           MetaReader.getTableRegionsAndLocations(ct,
               tableName);
@@ -1817,11 +1890,11 @@ public class HBaseAdmin implements Abort
    * @param htd modified description of the table
    * @throws IOException if a remote or network exception occurs
    */
-  public void modifyTable(final byte [] tableName, final HTableDescriptor htd)
+  public void modifyTable(final TableName tableName, final HTableDescriptor htd)
   throws IOException {
-    if (!Bytes.equals(tableName, htd.getName())) {
-      throw new IllegalArgumentException("the specified table name '" + Bytes.toString(tableName) +
-        "' doesn't match with the HTD one: " + htd.getNameAsString());
+    if (!tableName.equals(htd.getTableName())) {
+      throw new IllegalArgumentException("the specified table name '" + tableName +
+        "' doesn't match with the HTD one: " + htd.getTableName());
     }
 
     executeCallable(new MasterAdminCallable<Void>(getConnection()) {
@@ -1834,6 +1907,16 @@ public class HBaseAdmin implements Abort
     });
   }
 
+  public void modifyTable(final byte[] tableName, final HTableDescriptor htd)
+  throws IOException {
+    modifyTable(TableName.valueOf(tableName), htd);
+  }
+
+  public void modifyTable(final String tableName, final HTableDescriptor htd)
+  throws IOException {
+    modifyTable(TableName.valueOf(tableName), htd);
+  }
+
   /**
    * @param tableNameOrRegionName Name of a table or name of a region.
    * @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
@@ -1875,21 +1958,21 @@ public class HBaseAdmin implements Abort
   }
 
   /**
-   * Convert the table name byte array into a table name string and check if table
-   * exists or not.
-   * @param tableNameBytes Name of a table.
+   * Check if table exists or not
+   * @param tableName Name of a table.
    * @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
-   * @return tableName in string form.
+   * @return tableName instance
    * @throws IOException if a remote or network exception occurs.
    * @throws TableNotFoundException if table does not exist.
    */
-  private String tableNameString(final byte[] tableNameBytes, CatalogTracker ct)
+  //TODO rename this method
+  private TableName checkTableExists(
+      final TableName tableName, CatalogTracker ct)
       throws IOException {
-    String tableNameString = Bytes.toString(tableNameBytes);
-    if (!MetaReader.tableExists(ct, tableNameString)) {
-      throw new TableNotFoundException(tableNameString);
+    if (!MetaReader.tableExists(ct, tableName)) {
+      throw new TableNotFoundException(tableName);
     }
-    return tableNameString;
+    return tableName;
   }
 
   /**
@@ -1958,9 +2041,9 @@ public class HBaseAdmin implements Abort
     });
   }
 
-  private HRegionLocation getFirstMetaServerForTable(final byte [] tableName)
+  private HRegionLocation getFirstMetaServerForTable(final TableName tableName)
   throws IOException {
-    return connection.locateRegion(HConstants.META_TABLE_NAME,
+    return connection.locateRegion(TableName.META_TABLE_NAME,
       HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
   }
 
@@ -1972,6 +2055,127 @@ public class HBaseAdmin implements Abort
   }
 
   /**
+   * Create a new namespace
+   * @param descriptor descriptor which describes the new namespace
+   * @throws IOException
+   */
+  public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
+    executeCallable(new MasterAdminCallable<Void>(getConnection()) {
+      @Override
+      public Void call() throws Exception {
+        masterAdmin.createNamespace(null,
+            MasterAdminProtos.CreateNamespaceRequest.newBuilder()
+                .setNamespaceDescriptor(ProtobufUtil
+                    .toProtoNamespaceDescriptor(descriptor)).build());
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Modify an existing namespace
+   * @param descriptor descriptor which describes the new namespace
+   * @throws IOException
+   */
+  public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
+    executeCallable(new MasterAdminCallable<Void>(getConnection()) {
+      @Override
+      public Void call() throws Exception {
+        masterAdmin.modifyNamespace(null,
+            MasterAdminProtos.ModifyNamespaceRequest.newBuilder()
+                .setNamespaceDescriptor(ProtobufUtil
+                    .toProtoNamespaceDescriptor(descriptor)).build());
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+   * @param name namespace name
+   * @throws IOException
+   */
+  public void deleteNamespace(final String name) throws IOException {
+    executeCallable(new MasterAdminCallable<Void>(getConnection()) {
+      @Override
+      public Void call() throws Exception {
+        masterAdmin.deleteNamespace(null,
+            MasterAdminProtos.DeleteNamespaceRequest.newBuilder()
+                .setNamespaceName(name).build());
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Get a namespace descriptor by name
+   * @param name name of namespace descriptor
+   * @return
+   * @throws IOException
+   */
+  public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
+    return
+        executeCallable(new MasterAdminCallable<NamespaceDescriptor>(getConnection()) {
+          @Override
+          public NamespaceDescriptor call() throws Exception {
+            return ProtobufUtil.toNamespaceDescriptor(
+              masterAdmin.getNamespaceDescriptor(null,
+                  MasterAdminProtos.GetNamespaceDescriptorRequest.newBuilder()
+                    .setNamespaceName(name).build()).getNamespaceDescriptor());
+          }
+        });
+  }
+
+  /**
+   * List available namespace descriptors
+   * @return
+   * @throws IOException
+   */
+  public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
+    return
+        executeCallable(new MasterAdminCallable<NamespaceDescriptor[]>(getConnection()) {
+          @Override
+          public NamespaceDescriptor[] call() throws Exception {
+            List<HBaseProtos.NamespaceDescriptor> list =
+                masterAdmin.listNamespaceDescriptors(null,
+                    MasterAdminProtos.ListNamespaceDescriptorsRequest.newBuilder().build())
+                    .getNamespaceDescriptorList();
+            NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
+            for(int i = 0; i < list.size(); i++) {
+              res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
+            }
+            return res;
+          }
+        });
+  }
+
+  /**
+   * Get list of table descriptors by namespace
+   * @param name namespace name
+   * @return
+   * @throws IOException
+   */
+  public HTableDescriptor[] getTableDescriptorsByNamespace(final String name) throws IOException {
+    return
+        executeCallable(new MasterAdminCallable<HTableDescriptor[]>(getConnection()) {
+          @Override
+          public HTableDescriptor[] call() throws Exception {
+            List<TableSchema> list =
+                masterAdmin.getTableDescriptorsByNamespace(null,
+                    MasterAdminProtos.GetTableDescriptorsByNamespaceRequest.newBuilder()
+                        .setNamespaceName(name).build())
+                            .getTableSchemaList();
+            HTableDescriptor[] res = new HTableDescriptor[list.size()];
+            for(int i=0; i < list.size(); i++) {
+
+              res[i] = HTableDescriptor.convert(list.get(i));
+            }
+            return res;
+          }
+        });
+  }
+
+  /**
    * Check to see if HBase is running. Throw an exception if not.
    * We consider that HBase is running if ZooKeeper and Master are running.
    *
@@ -2029,7 +2233,7 @@ public class HBaseAdmin implements Abort
    * @return Ordered list of {@link HRegionInfo}.
    * @throws IOException
    */
-  public List<HRegionInfo> getTableRegions(final byte[] tableName)
+  public List<HRegionInfo> getTableRegions(final TableName tableName)
   throws IOException {
     CatalogTracker ct = getCatalogTracker();
     List<HRegionInfo> Regions = null;
@@ -2041,6 +2245,11 @@ public class HBaseAdmin implements Abort
     return Regions;
   }
 
+  public List<HRegionInfo> getTableRegions(final byte[] tableName)
+  throws IOException {
+    return getTableRegions(TableName.valueOf(tableName));
+  }
+
   @Override
   public void close() throws IOException {
     if (cleanupConnectionOnClose && this.connection != null) {
@@ -2048,18 +2257,34 @@ public class HBaseAdmin implements Abort
     }
   }
 
- /**
- * Get tableDescriptors
- * @param tableNames List of table names
- * @return HTD[] the tableDescriptor
- * @throws IOException if a remote or network exception occurs
- */
-  public HTableDescriptor[] getTableDescriptors(List<String> tableNames)
+  /**
+   * Get tableDescriptors
+   * @param tableNames List of table names
+   * @return HTD[] the tableDescriptor
+   * @throws IOException if a remote or network exception occurs
+   */
+  public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
   throws IOException {
-    return this.connection.getHTableDescriptors(tableNames);
+    return this.connection.getHTableDescriptorsByTableName(tableNames);
   }
 
   /**
+   * Get tableDescriptors
+   * @param tableNames List of table names
+   * @return HTD[] the tableDescriptor
+   * @throws IOException if a remote or network exception occurs
+   */
+  public HTableDescriptor[] getTableDescriptors(List<String> names)
+  throws IOException {
+    List<TableName> tableNames = new ArrayList<TableName>(names.size());
+    for(String name : names) {
+      tableNames.add(TableName.valueOf(name));
+    }
+    return getTableDescriptorsByTableName(tableNames);
+  }
+
+
+  /**
    * Roll the log writer. That is, start writing log messages to a new file.
    *
    * @param serverName
@@ -2123,7 +2348,7 @@ public class HBaseAdmin implements Abort
    * @throws InterruptedException
    * @return the current compaction state
    */
-  public CompactionState getCompactionState(final byte [] tableNameOrRegionName)
+  public CompactionState getCompactionState(final byte[] tableNameOrRegionName)
       throws IOException, InterruptedException {
     CompactionState state = CompactionState.NONE;
     CatalogTracker ct = getCatalogTracker();
@@ -2142,7 +2367,8 @@ public class HBaseAdmin implements Abort
           return response.getCompactionState();
         }
       } else {
-        final String tableName = tableNameString(tableNameOrRegionName, ct);
+        final TableName tableName =
+            checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
         List<Pair<HRegionInfo, ServerName>> pairs =
           MetaReader.getTableRegionsAndLocations(ct, tableName);
         for (Pair<HRegionInfo, ServerName> pair: pairs) {
@@ -2198,36 +2424,54 @@ public class HBaseAdmin implements Abort
    * a {@link SnapshotCreationException} indicating the duplicate naming.
    * <p>
    * Snapshot names follow the same naming constraints as tables in HBase. See
-   * {@link HTableDescriptor#isLegalTableName(byte[])}.
+   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
    * @param snapshotName name of the snapshot to be created
    * @param tableName name of the table for which snapshot is created
    * @throws IOException if a remote or network exception occurs
    * @throws SnapshotCreationException if snapshot creation failed
    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
    */
-  public void snapshot(final String snapshotName, final String tableName) throws IOException,
+  public void snapshot(final String snapshotName,
+                       final TableName tableName) throws IOException,
       SnapshotCreationException, IllegalArgumentException {
     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
   }
 
+  public void snapshot(final String snapshotName,
+                       final String tableName) throws IOException,
+      SnapshotCreationException, IllegalArgumentException {
+    snapshot(snapshotName, TableName.valueOf(tableName),
+        SnapshotDescription.Type.FLUSH);
+  }
+
   /**
-   * Create a timestamp consistent snapshot for the given table.
-   * <p>
-   * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
-   * snapshot with the same name (even a different type or with different parameters) will fail with
-   * a {@link SnapshotCreationException} indicating the duplicate naming.
-   * <p>
-   * Snapshot names follow the same naming constraints as tables in HBase. See
-   * {@link HTableDescriptor#isLegalTableName(byte[])}.
-   * @param snapshotName name of the snapshot to be created
-   * @param tableName name of the table for which snapshot is created
-   * @throws IOException if a remote or network exception occurs
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   */
-  public void snapshot(final byte[] snapshotName, final byte[] tableName) throws IOException,
+   public void snapshot(final String snapshotName,
+    * Create a timestamp consistent snapshot for the given table.
+                        final byte[] tableName) throws IOException,
+    * <p>
+    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
+    * snapshot with the same name (even a different type or with different parameters) will fail with
+    * a {@link SnapshotCreationException} indicating the duplicate naming.
+    * <p>
+    * Snapshot names follow the same naming constraints as tables in HBase. See
+    * {@link HTableDescriptor#isLegalTableName(byte[])}.
+    * @param snapshotName name of the snapshot to be created
+    * @param tableName name of the table for which snapshot is created
+    * @throws IOException if a remote or network exception occurs
+    * @throws SnapshotCreationException if snapshot creation failed
+    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
+    */
+  public void snapshot(final byte[] snapshotName,
+                       final TableName tableName) throws IOException,
       SnapshotCreationException, IllegalArgumentException {
-    snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
+    snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH);
+  }
+
+  public void snapshot(final byte[] snapshotName,
+                       final byte[] tableName) throws IOException,
+      SnapshotCreationException, IllegalArgumentException {
+    snapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName),
+        SnapshotDescription.Type.FLUSH);
   }
 
   /**
@@ -2238,7 +2482,7 @@ public class HBaseAdmin implements Abort
    * a {@link SnapshotCreationException} indicating the duplicate naming.
    * <p>
    * Snapshot names follow the same naming constraints as tables in HBase. See
-   * {@link HTableDescriptor#isLegalTableName(byte[])}.
+   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
    * <p>
    * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
    *          snapshots stored on the cluster
@@ -2248,16 +2492,31 @@ public class HBaseAdmin implements Abort
    * @throws SnapshotCreationException if snapshot creation failed
    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
    */
-  public void snapshot(final String snapshotName, final String tableName,
-      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
+  public void snapshot(final String snapshotName,
+                       final TableName tableName,
+                      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
       IllegalArgumentException {
     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
-    builder.setTable(tableName);
+    builder.setTable(tableName.getNameAsString());
     builder.setName(snapshotName);
     builder.setType(type);
     snapshot(builder.build());
   }
 
+  public void snapshot(final String snapshotName,
+                       final String tableName,
+                      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
+      IllegalArgumentException {
+    snapshot(snapshotName, TableName.valueOf(tableName), type);
+  }
+
+  public void snapshot(final String snapshotName,
+                       final byte[] tableName,
+                      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
+      IllegalArgumentException {
+    snapshot(snapshotName, TableName.valueOf(tableName), type);
+  }
+
   /**
    * Take a snapshot and wait for the server to complete that snapshot (blocking).
    * <p>
@@ -2270,7 +2529,7 @@ public class HBaseAdmin implements Abort
    * a {@link SnapshotCreationException} indicating the duplicate naming.
    * <p>
    * Snapshot names follow the same naming constraints as tables in HBase. See
-   * {@link HTableDescriptor#isLegalTableName(byte[])}.
+   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
    * <p>
    * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
    * unless you are sure about the type of snapshot that you want to take.
@@ -2406,10 +2665,10 @@ public class HBaseAdmin implements Abort
       throws IOException, RestoreSnapshotException {
     String rollbackSnapshot = snapshotName + "-" + EnvironmentEdgeManager.currentTimeMillis();
 
-    String tableName = null;
+    TableName tableName = null;
     for (SnapshotDescription snapshotInfo: listSnapshots()) {
       if (snapshotInfo.getName().equals(snapshotName)) {
-        tableName = snapshotInfo.getTable();
+        tableName = TableName.valueOf(snapshotInfo.getTable());
         break;
       }
     }
@@ -2453,9 +2712,26 @@ public class HBaseAdmin implements Abort
    */
   public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
-    cloneSnapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
+    cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName));
+  }
+
+  /**
+   * Create a new table by cloning the snapshot content.
+   *
+   * @param snapshotName name of the snapshot to be cloned
+   * @param tableName name of the table where the snapshot will be restored
+   * @throws IOException if a remote or network exception occurs
+   * @throws TableExistsException if table to be created already exists
+   * @throws RestoreSnapshotException if snapshot failed to be cloned
+   * @throws IllegalArgumentException if the specified table has not a valid name
+   */
+  public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
+      throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
+    cloneSnapshot(Bytes.toString(snapshotName), tableName);
   }
 
+
+
   /**
    * Create a new table by cloning the snapshot content.
    *
@@ -2468,11 +2744,26 @@ public class HBaseAdmin implements Abort
    */
   public void cloneSnapshot(final String snapshotName, final String tableName)
       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
+    cloneSnapshot(snapshotName, TableName.valueOf(tableName));
+  }
+
+  /**
+   * Create a new table by cloning the snapshot content.
+   *
+   * @param snapshotName name of the snapshot to be cloned
+   * @param tableName name of the table where the snapshot will be restored
+   * @throws IOException if a remote or network exception occurs
+   * @throws TableExistsException if table to be created already exists
+   * @throws RestoreSnapshotException if snapshot failed to be cloned
+   * @throws IllegalArgumentException if the specified table has not a valid name
+   */
+  public void cloneSnapshot(final String snapshotName, final TableName tableName)
+      throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
     if (tableExists(tableName)) {
-      throw new TableExistsException(tableName);
+      throw new TableExistsException("Table '" + tableName + " already exists");
     }
     internalRestoreSnapshot(snapshotName, tableName);
-    waitUntilTableIsEnabled(Bytes.toBytes(tableName));
+    waitUntilTableIsEnabled(tableName);
   }
 
   /**
@@ -2485,10 +2776,11 @@ public class HBaseAdmin implements Abort
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted incorrectly
    */
-  private void internalRestoreSnapshot(final String snapshotName, final String tableName)
+  private void internalRestoreSnapshot(final String snapshotName, final TableName
+      tableName)
       throws IOException, RestoreSnapshotException {
     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
-        .setName(snapshotName).setTable(tableName).build();
+        .setName(snapshotName).setTable(tableName.getNameAsString()).build();
 
     // actually restore the snapshot
     internalRestoreSnapshotAsync(snapshot);
@@ -2609,7 +2901,7 @@ public class HBaseAdmin implements Abort
    */
   public void deleteSnapshot(final String snapshotName) throws IOException {
     // make sure the snapshot is possibly valid
-    HTableDescriptor.isLegalTableName(Bytes.toBytes(snapshotName));
+    TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
     // do the delete
     executeCallable(new MasterAdminCallable<Void>(getConnection()) {
       @Override
@@ -2652,7 +2944,7 @@ public class HBaseAdmin implements Abort
   }
 
   /**
-   * @see {@link #execute(MasterAdminCallable<V>)}
+   * @see {@link #executeCallable(org.apache.hadoop.hbase.client.HBaseAdmin.MasterCallable)}
    */
   abstract static class MasterAdminCallable<V> extends MasterCallable<V> {
     protected MasterAdminKeepAliveConnection masterAdmin;
@@ -2673,7 +2965,7 @@ public class HBaseAdmin implements Abort
   }
 
   /**
-   * @see {@link #execute(MasterMonitorCallable<V>)}
+   * @see {@link #executeCallable(org.apache.hadoop.hbase.client.HBaseAdmin.MasterCallable)}
    */
   abstract static class MasterMonitorCallable<V> extends MasterCallable<V> {
     protected MasterMonitorKeepAliveConnection masterMonitor;

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.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.Abortable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
@@ -136,6 +137,9 @@ public interface HConnection extends Abo
    * @return true if the table is enabled, false otherwise
    * @throws IOException if a remote or network exception occurs
    */
+  boolean isTableEnabled(TableName tableName) throws IOException;
+
+  @Deprecated
   boolean isTableEnabled(byte[] tableName) throws IOException;
 
   /**
@@ -143,6 +147,9 @@ public interface HConnection extends Abo
    * @return true if the table is disabled, false otherwise
    * @throws IOException if a remote or network exception occurs
    */
+  boolean isTableDisabled(TableName tableName) throws IOException;
+
+  @Deprecated
   boolean isTableDisabled(byte[] tableName) throws IOException;
 
   /**
@@ -150,6 +157,9 @@ public interface HConnection extends Abo
    * @return true if all regions of the table are available, false otherwise
    * @throws IOException if a remote or network exception occurs
    */
+  boolean isTableAvailable(TableName tableName) throws IOException;
+
+  @Deprecated
   boolean isTableAvailable(byte[] tableName) throws IOException;
 
   /**
@@ -164,7 +174,12 @@ public interface HConnection extends Abo
    * @throws IOException
    *           if a remote or network exception occurs
    */
-  boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException;
+  boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws
+      IOException;
+
+  @Deprecated
+  boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws
+      IOException;
 
   /**
    * List all the userspace tables.  In other words, scan the META table.
@@ -183,6 +198,10 @@ public interface HConnection extends Abo
    * @return table metadata
    * @throws IOException if a remote or network exception occurs
    */
+  HTableDescriptor getHTableDescriptor(TableName tableName)
+  throws IOException;
+
+  @Deprecated
   HTableDescriptor getHTableDescriptor(byte[] tableName)
   throws IOException;
 
@@ -195,10 +214,12 @@ public interface HConnection extends Abo
    * question
    * @throws IOException if a remote or network exception occurs
    */
-  HRegionLocation locateRegion(
-    final byte[] tableName, final byte[] row
-  )
-  throws IOException;
+  public HRegionLocation locateRegion(final TableName tableName,
+      final byte [] row) throws IOException;
+
+  @Deprecated
+  public HRegionLocation locateRegion(final byte[] tableName,
+      final byte [] row) throws IOException;
 
   /**
    * Allows flushing the region cache.
@@ -211,6 +232,9 @@ public interface HConnection extends Abo
    * @param tableName Name of the table whose regions we are to remove from
    * cache.
    */
+  void clearRegionCache(final TableName tableName);
+
+  @Deprecated
   void clearRegionCache(final byte[] tableName);
 
   /**
@@ -228,10 +252,12 @@ public interface HConnection extends Abo
    * question
    * @throws IOException if a remote or network exception occurs
    */
-  HRegionLocation relocateRegion(
-    final byte[] tableName, final byte[] row
-  )
-  throws IOException;
+  HRegionLocation relocateRegion(final TableName tableName,
+      final byte [] row) throws IOException;
+
+  @Deprecated
+  HRegionLocation relocateRegion(final byte[] tableName,
+      final byte [] row) throws IOException;
 
   /**
    * Update the location cache. This is used internally by HBase, in most cases it should not be
@@ -241,9 +267,12 @@ public interface HConnection extends Abo
    * @param exception the exception if any. Can be null.
    * @param source the previous location
    */
-  void updateCachedLocations(
-    byte[] tableName, byte[] rowkey, Object exception, HRegionLocation source
-  );
+  void updateCachedLocations(TableName tableName, byte[] rowkey,
+                                    Object exception, HRegionLocation source);
+
+  @Deprecated
+  void updateCachedLocations(byte[] tableName, byte[] rowkey,
+                                    Object exception, HRegionLocation source);
 
   /**
    * Gets the location of the region of <i>regionName</i>.
@@ -261,8 +290,10 @@ public interface HConnection extends Abo
    * @return list of region locations for all regions of table
    * @throws IOException
    */
-  List<HRegionLocation> locateRegions(final byte[] tableName)
-  throws IOException;
+  List<HRegionLocation> locateRegions(final TableName tableName) throws IOException;
+
+  @Deprecated
+  List<HRegionLocation> locateRegions(final byte[] tableName) throws IOException;
 
   /**
    * Gets the locations of all regions in the specified table, <i>tableName</i>.
@@ -273,9 +304,14 @@ public interface HConnection extends Abo
    * @return list of region locations for all regions of table
    * @throws IOException
    */
-  List<HRegionLocation> locateRegions(
-    final byte[] tableName, final boolean useCache, final boolean offlined
-  ) throws IOException;
+  public List<HRegionLocation> locateRegions(final TableName tableName,
+      final boolean useCache,
+      final boolean offlined) throws IOException;
+
+  @Deprecated
+  public List<HRegionLocation> locateRegions(final byte[] tableName,
+      final boolean useCache,
+      final boolean offlined) throws IOException;
 
   /**
    * Returns a {@link MasterAdminKeepAliveConnection} to the active master
@@ -325,7 +361,12 @@ public interface HConnection extends Abo
    * @return Location of row.
    * @throws IOException if a remote or network exception occurs
    */
-  HRegionLocation getRegionLocation(byte [] tableName, byte [] row,
+  HRegionLocation getRegionLocation(TableName tableName, byte [] row,
+    boolean reload)
+  throws IOException;
+
+  @Deprecated
+  HRegionLocation getRegionLocation(byte[] tableName, byte [] row,
     boolean reload)
   throws IOException;
 
@@ -345,10 +386,12 @@ public interface HConnection extends Abo
    * @deprecated since 0.96 - Use {@link HTableInterface#batch} instead
    */
   @Deprecated
-  void processBatch(
-    List<? extends Row> actions, final byte[] tableName, ExecutorService pool, Object[] results
-  )
-      throws IOException, InterruptedException;
+  void processBatch(List<? extends Row> actions, final TableName tableName,
+      ExecutorService pool, Object[] results) throws IOException, InterruptedException;
+
+  @Deprecated
+  void processBatch(List<? extends Row> actions, final byte[] tableName,
+      ExecutorService pool, Object[] results) throws IOException, InterruptedException;
 
   /**
    * Parameterized batch processing, allowing varying return types for different
@@ -356,13 +399,18 @@ public interface HConnection extends Abo
    * @deprecated since 0.96 - Use {@link HTableInterface#batchCallback} instead
    */
   @Deprecated
-  <R> void processBatchCallback(
-    List<? extends Row> list,
-    byte[] tableName,
-    ExecutorService pool,
-    Object[] results,
-    Batch.Callback<R> callback
-  ) throws IOException, InterruptedException;
+  public <R> void processBatchCallback(List<? extends Row> list,
+      final TableName tableName,
+      ExecutorService pool,
+      Object[] results,
+      Batch.Callback<R> callback) throws IOException, InterruptedException;
+
+  @Deprecated
+  public <R> void processBatchCallback(List<? extends Row> list,
+      final byte[] tableName,
+      ExecutorService pool,
+      Object[] results,
+      Batch.Callback<R> callback) throws IOException, InterruptedException;
 
   /**
    * Enable or disable region cache prefetch for the table. It will be
@@ -371,9 +419,11 @@ public interface HConnection extends Abo
    * @param tableName name of table to configure.
    * @param enable Set to true to enable region cache prefetch.
    */
-  void setRegionCachePrefetch(
-    final byte[] tableName, final boolean enable
-  );
+  public void setRegionCachePrefetch(final TableName tableName,
+      final boolean enable);
+
+  public void setRegionCachePrefetch(final byte[] tableName,
+      final boolean enable);
 
   /**
    * Check whether region cache prefetch is enabled or not.
@@ -381,6 +431,8 @@ public interface HConnection extends Abo
    * @return true if table's region cache prefetch is enabled. Otherwise
    * it is disabled.
    */
+  boolean getRegionCachePrefetch(final TableName tableName);
+
   boolean getRegionCachePrefetch(final byte[] tableName);
 
   /**
@@ -395,8 +447,11 @@ public interface HConnection extends Abo
    * @return HTD[] table metadata
    * @throws IOException if a remote or network exception occurs
    */
-  HTableDescriptor[] getHTableDescriptors(List<String> tableNames)
-  throws IOException;
+  HTableDescriptor[] getHTableDescriptorsByTableName(List<TableName> tableNames) throws IOException;
+
+  @Deprecated
+  HTableDescriptor[] getHTableDescriptors(List<String> tableNames) throws
+      IOException;
 
   /**
    * @return true if this connection is closed