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 2015/12/01 20:30:43 UTC

[2/3] hbase git commit: Revert "HBASE-14769 Removing unused functions from HBaseAdmin. Removing redundant javadocs from HBaseAdmin as they will be automatically inhertited from Admin.java. (Apekshit)" Reverting. The conversation on issues to do with this

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 7a50458..66079dd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -221,7 +221,7 @@ public class HBaseAdmin implements Admin {
     return operationTimeout;
   }
 
-  HBaseAdmin(ClusterConnection connection) throws IOException {
+  HBaseAdmin(ClusterConnection connection) {
     this.conf = connection.getConfiguration();
     this.connection = connection;
 
@@ -253,6 +253,13 @@ public class HBaseAdmin implements Admin {
     return this.aborted;
   }
 
+  /**
+   * Abort a procedure
+   * @param procId ID of the procedure to abort
+   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
+   * @return true if aborted, false if procedure already completed or does not exist
+   * @throws IOException
+   */
   @Override
   public boolean abortProcedure(
       final long procId,
@@ -273,6 +280,18 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Abort a procedure but does not block and wait for it be completely removed.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   *
+   * @param procId ID of the procedure to abort
+   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
+   * @return true if aborted, false if procedure already completed or does not exist
+   * @throws IOException
+   */
   @Override
   public Future<Boolean> abortProcedureAsync(
     final long procId,
@@ -320,6 +339,23 @@ public class HBaseAdmin implements Admin {
     return connection;
   }
 
+  /** @return - true if the master server is running. Throws an exception
+   *  otherwise.
+   * @throws ZooKeeperConnectionException
+   * @throws MasterNotRunningException
+   * @deprecated this has been deprecated without a replacement
+   */
+  @Deprecated
+  public boolean isMasterRunning()
+  throws MasterNotRunningException, ZooKeeperConnectionException {
+    return connection.isMasterRunning();
+  }
+
+  /**
+   * @param tableName Table to check.
+   * @return True if table exists already.
+   * @throws IOException
+   */
   @Override
   public boolean tableExists(final TableName tableName) throws IOException {
     return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
@@ -330,6 +366,16 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  public boolean tableExists(final byte[] tableName)
+  throws IOException {
+    return tableExists(TableName.valueOf(tableName));
+  }
+
+  public boolean tableExists(final String tableName)
+  throws IOException {
+    return tableExists(TableName.valueOf(tableName));
+  }
+
   @Override
   public HTableDescriptor[] listTables() throws IOException {
     return listTables((Pattern)null, false);
@@ -364,6 +410,51 @@ public class HBaseAdmin implements Admin {
     return listTables(Pattern.compile(regex), includeSysTables);
   }
 
+  /**
+   * List all of the names of userspace tables.
+   * @return String[] table names
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated Use {@link Admin#listTableNames()} instead
+   */
+  @Deprecated
+  public String[] getTableNames() throws IOException {
+    TableName[] tableNames = listTableNames();
+    String[] result = new String[tableNames.length];
+    for (int i = 0; i < tableNames.length; i++) {
+      result[i] = tableNames[i].getNameAsString();
+    }
+    return result;
+  }
+
+  /**
+   * List all of the names of userspace tables matching the given regular expression.
+   * @param pattern The regular expression to match against
+   * @return String[] table names
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated Use {@link Admin#listTableNames(Pattern)} instead.
+   */
+  @Deprecated
+  public String[] getTableNames(Pattern pattern) throws IOException {
+    TableName[] tableNames = listTableNames(pattern);
+    String[] result = new String[tableNames.length];
+    for (int i = 0; i < tableNames.length; i++) {
+      result[i] = tableNames[i].getNameAsString();
+    }
+    return result;
+  }
+
+  /**
+   * List all of the names of userspace tables matching the given regular expression.
+   * @param regex The regular expression to match against
+   * @return String[] table names
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated Use {@link Admin#listTableNames(Pattern)} instead.
+   */
+  @Deprecated
+  public String[] getTableNames(String regex) throws IOException {
+    return getTableNames(Pattern.compile(regex));
+  }
+
   @Override
   public TableName[] listTableNames() throws IOException {
     return listTableNames((Pattern)null, false);
@@ -399,13 +490,23 @@ public class HBaseAdmin implements Admin {
     return listTableNames(Pattern.compile(regex), includeSysTables);
   }
 
+  /**
+   * Method for getting the tableDescriptor
+   * @param tableName as a byte []
+   * @return the tableDescriptor
+   * @throws TableNotFoundException
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
-  public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
+  public HTableDescriptor getTableDescriptor(final TableName tableName)
+  throws TableNotFoundException, IOException {
      return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, operationTimeout);
   }
 
-  static HTableDescriptor getTableDescriptor(final TableName tableName, HConnection connection,
-      RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout) throws IOException {
+  static HTableDescriptor getTableDescriptor(final TableName tableName,
+         HConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
+         int operationTimeout) throws TableNotFoundException, IOException {
+
       if (tableName == null) return null;
       HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
         @Override
@@ -427,6 +528,11 @@ public class HBaseAdmin implements Admin {
       throw new TableNotFoundException(tableName.getNameAsString());
   }
 
+  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) {
@@ -435,12 +541,49 @@ public class HBaseAdmin implements Admin {
     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
   }
 
+  /**
+   * Creates a new table.
+   * Synchronous operation.
+   *
+   * @param desc table descriptor for table
+   *
+   * @throws IllegalArgumentException if the table name is reserved
+   * @throws MasterNotRunningException if master is not running
+   * @throws TableExistsException if table already exists (If concurrent
+   * threads, the table may have been created between test-for-existence
+   * and attempt-at-creation).
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void createTable(HTableDescriptor desc)
   throws IOException {
     createTable(desc, null);
   }
 
+  /**
+   * Creates a new table with the specified number of regions.  The start key
+   * specified will become the end key of the first region of the table, and
+   * the end key specified will become the start key of the last region of the
+   * table (the first region has a null start key and the last region has a
+   * null end key).
+   *
+   * BigInteger math will be used to divide the key range specified into
+   * enough segments to make the required number of total regions.
+   *
+   * Synchronous operation.
+   *
+   * @param desc table descriptor for table
+   * @param startKey beginning of key range
+   * @param endKey end of key range
+   * @param numRegions the total number of regions to create
+   *
+   * @throws IllegalArgumentException if the table name is reserved
+   * @throws MasterNotRunningException if master is not running
+   * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
+   * threads, the table may have been created between test-for-existence
+   * and attempt-at-creation).
+   * @throws IOException
+   */
   @Override
   public void createTable(HTableDescriptor desc, byte [] startKey,
       byte [] endKey, int numRegions)
@@ -461,6 +604,23 @@ public class HBaseAdmin implements Admin {
     createTable(desc, splitKeys);
   }
 
+  /**
+   * Creates a new table with an initial set of empty regions defined by the
+   * specified split keys.  The total number of regions created will be the
+   * number of split keys plus one. Synchronous operation.
+   * Note : Avoid passing empty split key.
+   *
+   * @param desc table descriptor for table
+   * @param splitKeys array of split keys for the initial regions of the table
+   *
+   * @throws IllegalArgumentException if the table name is reserved, if the split keys
+   * are repeated and if the split key has empty byte array.
+   * @throws MasterNotRunningException if master is not running
+   * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
+   * threads, the table may have been created between test-for-existence
+   * and attempt-at-creation).
+   * @throws IOException
+   */
   @Override
   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
       throws IOException {
@@ -482,6 +642,21 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Creates a new table but does not block and wait for it to come online.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   *
+   * @param desc table descriptor for table
+   * @param splitKeys keys to check if the table has been created with all split keys
+   * @throws IllegalArgumentException Bad table name, if the split keys
+   *    are repeated and if the split key has empty byte array.
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async creation. You can use Future.get(long, TimeUnit)
+   *    to wait on the operation to complete.
+   */
   @Override
   public Future<Void> createTableAsync(final HTableDescriptor desc, final byte[][] splitKeys)
       throws IOException {
@@ -548,6 +723,21 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  public void deleteTable(final String tableName) throws IOException {
+    deleteTable(TableName.valueOf(tableName));
+  }
+
+  public void deleteTable(final byte[] tableName) throws IOException {
+    deleteTable(TableName.valueOf(tableName));
+  }
+
+  /**
+   * Deletes a table.
+   * Synchronous operation.
+   *
+   * @param tableName name of table to delete
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void deleteTable(final TableName tableName) throws IOException {
     Future<Void> future = deleteTableAsync(tableName);
@@ -566,6 +756,18 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Deletes the table but does not block and wait for it be completely removed.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   *
+   * @param tableName name of table to delete
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async delete. You can use Future.get(long, TimeUnit)
+   *    to wait on the operation to complete.
+   */
   @Override
   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
     DeleteTableResponse response = executeCallable(
@@ -608,6 +810,19 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Deletes tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.lang.String)} and
+   * {@link #deleteTable(byte[])}
+   *
+   * @param regex The regular expression to match table names against
+   * @return Table descriptors for tables that couldn't be deleted
+   * @throws IOException
+   * @see #deleteTables(java.util.regex.Pattern)
+   * @see #deleteTable(java.lang.String)
+   */
   @Override
   public HTableDescriptor[] deleteTables(String regex) throws IOException {
     return deleteTables(Pattern.compile(regex));
@@ -618,7 +833,7 @@ public class HBaseAdmin implements Admin {
    *
    * Warning: Use this method carefully, there is no prompting and the effect is
    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
-   * {@link #deleteTable(TableName)}
+   * {@link #deleteTable(byte[])}
    *
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be deleted
@@ -638,6 +853,12 @@ public class HBaseAdmin implements Admin {
     return failed.toArray(new HTableDescriptor[failed.size()]);
   }
 
+  /**
+   * Truncate a table. Synchronous operation.
+   * @param tableName name of table to truncate
+   * @param preserveSplits True if the splits should be preserved
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void truncateTable(final TableName tableName, final boolean preserveSplits)
       throws IOException {
@@ -658,6 +879,18 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Truncate the table but does not block and wait for it be completely enabled. You can use
+   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete. Asynchronous
+   * operation.
+   * @param tableName name of table to delete
+   * @param preserveSplits true if the splits should be preserved
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the
+   *         operation to complete.
+   */
   @Override
   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
       throws IOException {
@@ -714,6 +947,20 @@ public class HBaseAdmin implements Admin {
     return splits;
   }
 
+
+  /**
+   * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
+   * and {@link #isTableEnabled(byte[])} instead.
+   * The table has to be in disabled state for it to be enabled.
+   * @param tableName name of the table
+   * @throws IOException if a remote or network exception occurs
+   * There could be couple types of IOException
+   * TableNotFoundException means the table doesn't exist.
+   * TableNotDisabledException means the table isn't in disabled state.
+   * @see #isTableEnabled(byte[])
+   * @see #disableTable(byte[])
+   * @see #enableTableAsync(byte[])
+   */
   @Override
   public void enableTable(final TableName tableName)
   throws IOException {
@@ -733,6 +980,16 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  public void enableTable(final byte[] tableName)
+  throws IOException {
+    enableTable(TableName.valueOf(tableName));
+  }
+
+  public void enableTable(final String tableName)
+  throws IOException {
+    enableTable(TableName.valueOf(tableName));
+  }
+
   /**
    * Wait for the table to be enabled and available
    * If enabling the table exceeds the retry period, an exception is thrown.
@@ -774,6 +1031,28 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  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 the table but does not block and wait for it be completely enabled.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   *
+   * @param tableName name of table to delete
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async enable. You can use Future.get(long, TimeUnit)
+   *    to wait on the operation to complete.
+   */
   @Override
   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
@@ -809,11 +1088,33 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Enable tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.lang.String)} and
+   * {@link #enableTable(byte[])}
+   *
+   * @param regex The regular expression to match table names against
+   * @throws IOException
+   * @see #enableTables(java.util.regex.Pattern)
+   * @see #enableTable(java.lang.String)
+   */
   @Override
   public HTableDescriptor[] enableTables(String regex) throws IOException {
     return enableTables(Pattern.compile(regex));
   }
 
+  /**
+   * Enable tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
+   * {@link #enableTable(byte[])}
+   *
+   * @param pattern The pattern to match table names against
+   * @throws IOException
+   */
   @Override
   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
@@ -830,6 +1131,25 @@ public class HBaseAdmin implements Admin {
     return failed.toArray(new HTableDescriptor[failed.size()]);
   }
 
+  public void disableTableAsync(final byte[] tableName) throws IOException {
+    disableTableAsync(TableName.valueOf(tableName));
+  }
+
+  public void disableTableAsync(final String tableName) throws IOException {
+    disableTableAsync(TableName.valueOf(tableName));
+  }
+
+  /**
+   * Disable table and wait on completion.  May timeout eventually.  Use
+   * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
+   * instead.
+   * The table has to be in enabled state for it to be disabled.
+   * @param tableName
+   * @throws IOException
+   * There could be couple types of IOException
+   * TableNotFoundException means the table doesn't exist.
+   * TableNotEnabledException means the table isn't in enabled state.
+   */
   @Override
   public void disableTable(final TableName tableName)
   throws IOException {
@@ -849,6 +1169,28 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  public void disableTable(final byte[] tableName)
+  throws IOException {
+    disableTable(TableName.valueOf(tableName));
+  }
+
+  public void disableTable(final String tableName)
+  throws IOException {
+    disableTable(TableName.valueOf(tableName));
+  }
+
+  /**
+   * Disable the table but does not block and wait for it be completely disabled.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   *
+   * @param tableName name of table to delete
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async disable. You can use Future.get(long, TimeUnit)
+   *    to wait on the operation to complete.
+   */
   @Override
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
@@ -878,17 +1220,42 @@ public class HBaseAdmin implements Admin {
     }
 
     @Override
-    protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException {
+    protected Void waitOperationResult(final long deadlineTs)
+        throws IOException, TimeoutException {
       waitForTableDisabled(deadlineTs);
       return null;
     }
   }
 
+  /**
+   * Disable tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.lang.String)} and
+   * {@link #disableTable(byte[])}
+   *
+   * @param regex The regular expression to match table names against
+   * @return Table descriptors for tables that couldn't be disabled
+   * @throws IOException
+   * @see #disableTables(java.util.regex.Pattern)
+   * @see #disableTable(java.lang.String)
+   */
   @Override
   public HTableDescriptor[] disableTables(String regex) throws IOException {
     return disableTables(Pattern.compile(regex));
   }
 
+  /**
+   * Disable tables matching the passed in pattern and wait on completion.
+   *
+   * Warning: Use this method carefully, there is no prompting and the effect is
+   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
+   * {@link #disableTable(byte[])}
+   *
+   * @param pattern The pattern to match table names against
+   * @return Table descriptors for tables that couldn't be disabled
+   * @throws IOException
+   */
   @Override
   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
@@ -905,9 +1272,24 @@ public class HBaseAdmin implements Admin {
     return failed.toArray(new HTableDescriptor[failed.size()]);
   }
 
+  /*
+   * Checks whether table exists. If not, throws TableNotFoundException
+   * @param tableName
+   */
+  private void checkTableExistence(TableName tableName) throws IOException {
+    if (!tableExists(tableName)) {
+      throw new TableNotFoundException(tableName);
+    }
+  }
+
+  /**
+   * @param tableName name of table to check
+   * @return true if table is on-line
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
-    checkTableExists(tableName);
+    checkTableExistence(tableName);
     return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException, IOException {
@@ -919,46 +1301,173 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  public boolean isTableEnabled(byte[] tableName) throws IOException {
+    return isTableEnabled(TableName.valueOf(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
+   */
   @Override
   public boolean isTableDisabled(TableName tableName) throws IOException {
-    checkTableExists(tableName);
+    checkTableExistence(tableName);
     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
+   */
   @Override
   public boolean isTableAvailable(TableName tableName) throws IOException {
     return connection.isTableAvailable(tableName);
   }
 
+  public boolean isTableAvailable(byte[] tableName) throws IOException {
+    return isTableAvailable(TableName.valueOf(tableName));
+  }
+
+  public boolean isTableAvailable(String 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
+   */
   @Override
-  public boolean isTableAvailable(TableName 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 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
+   */
   @Override
-  public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
+  public Pair<Integer, Integer> getAlterStatus(final TableName tableName)
+  throws IOException {
     return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
       @Override
       public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
         GetSchemaAlterStatusRequest req = RequestConverter
             .buildGetSchemaAlterStatusRequest(tableName);
         GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null, req);
-        Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
-            ret.getTotalRegions());
+        Pair<Integer, Integer> pair = new Pair<Integer, Integer>(Integer.valueOf(ret
+            .getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions()));
         return pair;
       }
     });
   }
 
+  /**
+   * 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
+   */
   @Override
-  public Pair<Integer, Integer> getAlterStatus(final byte[] tableName) throws IOException {
+  public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
+   throws IOException {
     return getAlterStatus(TableName.valueOf(tableName));
   }
 
   /**
-   * {@inheritDoc}
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #addColumnFamily(TableName, HColumnDescriptor)} instead.
+   * Add a column family to an existing table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of the table to add column family to
+   * @param columnFamily column family descriptor of column family to be added
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
+   */
+  @Deprecated
+  public void addColumn(final byte[] tableName, HColumnDescriptor columnFamily)
+  throws IOException {
+    addColumnFamily(TableName.valueOf(tableName), columnFamily);
+  }
+
+  /**
+   * Add a column family to an existing table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of the table to add column family to
+   * @param columnFamily column family descriptor of column family to be added
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
+   */
+  @Deprecated
+  public void addColumn(final String tableName, HColumnDescriptor columnFamily)
+  throws IOException {
+    addColumnFamily(TableName.valueOf(tableName), columnFamily);
+  }
+
+  /**
+   * Add a column family to an existing table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of the table to add column family to
+   * @param columnFamily column family descriptor of column family to be added
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
    */
   @Override
   @Deprecated
@@ -997,9 +1506,52 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * {@inheritDoc}
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #deleteColumnFamily(TableName, byte[])} instead.
+   * Delete a column family from a table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param columnFamily name of column family to be deleted
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #deleteColumnFamily(TableName, byte[])}.
+   */
+  @Deprecated
+  public void deleteColumn(final byte[] tableName, final String columnFamily)
+  throws IOException {
+    deleteColumnFamily(TableName.valueOf(tableName), Bytes.toBytes(columnFamily));
+  }
+
+  /**
+   * Delete a column family from a table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param columnFamily name of column family to be deleted
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #deleteColumnFamily(TableName, byte[])}.
+   */
+  @Deprecated
+  public void deleteColumn(final String tableName, final String columnFamily)
+  throws IOException {
+    deleteColumnFamily(TableName.valueOf(tableName), Bytes.toBytes(columnFamily));
+  }
+
+  /**
+   * Delete a column family from a table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param columnFamily name of column family to be deleted
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #deleteColumnFamily(TableName, byte[])}.
    */
   @Override
   @Deprecated
@@ -1039,9 +1591,52 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * {@inheritDoc}
-   * @deprecated As of 2.0. Will be removed in 3.0. Use
-   *     {@link #modifyColumnFamily(TableName, HColumnDescriptor)} instead.
+   * Modify an existing column family on a table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param columnFamily new column family descriptor to use
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
+   */
+  @Deprecated
+  public void modifyColumn(final String tableName, HColumnDescriptor columnFamily)
+  throws IOException {
+    modifyColumnFamily(TableName.valueOf(tableName), columnFamily);
+  }
+
+  /**
+   * Modify an existing column family on a table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param columnFamily new column family descriptor to use
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
+   */
+  @Deprecated
+  public void modifyColumn(final byte[] tableName, HColumnDescriptor columnFamily)
+  throws IOException {
+    modifyColumnFamily(TableName.valueOf(tableName), columnFamily);
+  }
+
+  /**
+   * Modify an existing column family on a table.
+   * Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param columnFamily new column family descriptor to use
+   * @throws IOException if a remote or network exception occurs
+   * @deprecated As of release 2.0.0.
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
+   *             This will be removed in HBase 3.0.0.
+   *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
    */
   @Override
   @Deprecated
@@ -1080,13 +1675,33 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Close a region. For expert-admins.  Runs close on the regionserver.  The
+   * master will not be informed of the close.
+   * @param regionname region name to close
+   * @param serverName If supplied, we'll use this location rather than
+   * the one currently in <code>hbase:meta</code>
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
-  public void closeRegion(final String regionname, final String serverName) throws IOException {
+  public void closeRegion(final String regionname, final String serverName)
+  throws IOException {
     closeRegion(Bytes.toBytes(regionname), serverName);
   }
 
+  /**
+   * Close a region.  For expert-admins  Runs close on the regionserver.  The
+   * master will not be informed of the close.
+   * @param regionname region name to close
+   * @param serverName The servername of the regionserver.  If passed null we
+   * will use servername found in the hbase:meta table. A server name
+   * is made of host, port and startcode.  Here is an example:
+   * <code> host187.example.com,60020,1289493121758</code>
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
-  public void closeRegion(final byte [] regionname, final String serverName) throws IOException {
+  public void closeRegion(final byte [] regionname, final String serverName)
+      throws IOException {
     if (serverName != null) {
       Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
       if (pair == null || pair.getFirst() == null) {
@@ -1106,6 +1721,27 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * For expert-admins. Runs close on the regionserver. Closes a region based on
+   * the encoded region name. The region server name is mandatory. If the
+   * servername is provided then based on the online regions in the specified
+   * regionserver the specified region will be closed. The master will not be
+   * informed of the close. Note that the regionname is the encoded regionname.
+   *
+   * @param encodedRegionName
+   *          The encoded region name; i.e. the hash that makes up the region
+   *          name suffix: e.g. if regionname is
+   *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
+   *          , then the encoded region name is:
+   *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
+   * @param serverName
+   *          The servername of the regionserver. A server name is made of host,
+   *          port and startcode. This is mandatory. Here is an example:
+   *          <code> host187.example.com,60020,1289493121758</code>
+   * @return true if the region was closed, false if not.
+   * @throws IOException
+   *           if a remote or network exception occurs
+   */
   @Override
   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
       final String serverName) throws IOException {
@@ -1130,19 +1766,33 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Close a region.  For expert-admins  Runs close on the regionserver.  The
+   * master will not be informed of the close.
+   * @param sn
+   * @param hri
+   * @throws IOException
+   */
   @Override
-  public void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException {
+  public void closeRegion(final ServerName sn, final HRegionInfo hri)
+  throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     ProtobufUtil.closeRegion(admin, sn, hri.getRegionName());
   }
 
+  /**
+   * Get all the online regions on a region server.
+   */
   @Override
   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     return ProtobufUtil.getOnlineRegions(admin);
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public void flush(final TableName tableName) throws IOException {
     checkTableExists(tableName);
@@ -1154,6 +1804,9 @@ public class HBaseAdmin implements Admin {
       new HashMap<String, String>());
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public void flushRegion(final byte[] regionName) throws IOException {
     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
@@ -1163,16 +1816,7 @@ public class HBaseAdmin implements Admin {
     if (regionServerPair.getSecond() == null) {
       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
-    HRegionInfo hRegionInfo = regionServerPair.getFirst();
-    ServerName serverName = regionServerPair.getSecond();
-    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
-    FlushRegionRequest request =
-        RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
-    try {
-      admin.flushRegion(null, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    flush(regionServerPair.getSecond(), regionServerPair.getFirst());
   }
 
   private void flush(final ServerName sn, final HRegionInfo hri)
@@ -1196,6 +1840,9 @@ public class HBaseAdmin implements Admin {
     compact(tableName, null, false, CompactType.NORMAL);
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public void compactRegion(final byte[] regionName)
     throws IOException {
@@ -1231,12 +1878,18 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public void majorCompact(final TableName tableName)
   throws IOException {
     compact(tableName, null, true, CompactType.NORMAL);
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public void majorCompactRegion(final byte[] regionName)
   throws IOException {
@@ -1252,6 +1905,9 @@ public class HBaseAdmin implements Admin {
     compact(tableName, columnFamily, true, CompactType.NORMAL);
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
   throws IOException {
@@ -1345,6 +2001,19 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Move the region <code>r</code> to <code>dest</code>.
+   * @param encodedRegionName The encoded region name; i.e. the hash that makes
+   * up the region name suffix: e.g. if regionname is
+   * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
+   * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
+   * @param destServerName The servername of the destination regionserver.  If
+   * passed the empty byte array we'll assign to a random server.  A server name
+   * is made of host, port and startcode.  Here is an example:
+   * <code> host187.example.com,60020,1289493121758</code>
+   * @throws UnknownRegionException Thrown if we can't find a region named
+   * <code>encodedRegionName</code>
+   */
   @Override
   public void move(final byte [] encodedRegionName, final byte [] destServerName)
       throws IOException {
@@ -1365,6 +2034,13 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * @param regionName
+   *          Region name to assign.
+   * @throws MasterNotRunningException
+   * @throws ZooKeeperConnectionException
+   * @throws IOException
+   */
   @Override
   public void assign(final byte[] regionName) throws MasterNotRunningException,
       ZooKeeperConnectionException, IOException {
@@ -1380,6 +2056,20 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Unassign a region from current hosting regionserver.  Region will then be
+   * assigned to a regionserver chosen at random.  Region could be reassigned
+   * back to the same server.  Use {@link #move(byte[], byte[])} if you want
+   * to control the region movement.
+   * @param regionName Region to unassign. Will clear any existing RegionPlan
+   * if one found.
+   * @param force If true, force unassign (Will remove region from
+   * regions-in-transition too if present. If results in double assignment
+   * use hbck -fix to resolve. To be used by experts).
+   * @throws MasterNotRunningException
+   * @throws ZooKeeperConnectionException
+   * @throws IOException
+   */
   @Override
   public void unassign(final byte [] regionName, final boolean force)
   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
@@ -1395,6 +2085,18 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Offline specified region from master's in-memory state. It will not attempt to reassign the
+   * region as in unassign. This API can be used when a region not served by any region server and
+   * still online as per Master's in memory state. If this API is incorrectly used on active region
+   * then master will loose track of that region.
+   *
+   * This is a special method that should be used by experts or hbck.
+   *
+   * @param regionName
+   *          Region to offline.
+   * @throws IOException
+   */
   @Override
   public void offline(final byte [] regionName)
   throws IOException {
@@ -1407,6 +2109,12 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Turn the load balancer on or off.
+   * @param on If true, enable balancer. If false, disable balancer.
+   * @param synchronous If true, it waits until current balance() call, if outstanding, to return.
+   * @return Previous balancer value
+   */
   @Override
   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
   throws IOException {
@@ -1420,6 +2128,12 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Invoke the balancer.  Will run the balancer and if regions to move, it will
+   * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
+   * logs.
+   * @return True if balancer ran, false otherwise.
+   */
   @Override
   public boolean balancer() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@@ -1440,6 +2154,12 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Query the state of the balancer from the Master. It's not a guarantee that the balancer is
+   * actually running this very moment, but that it will run.
+   *
+   * @return True if the balancer is enabled, false otherwise.
+   */
   @Override
   public boolean isBalancerEnabled() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@@ -1451,6 +2171,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Invoke region normalizer. Can NOT run for various reasons.  Check logs.
+   *
+   * @return True if region normalizer ran, false otherwise.
+   */
   @Override
   public boolean normalize() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@@ -1462,7 +2187,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  @Override
+  /**
+   * Query the current state of the region normalizer
+   *
+   * @return true if region normalizer is enabled, false otherwise.
+   */
   public boolean isNormalizerEnabled() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
@@ -1473,7 +2202,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  @Override
+  /**
+   * Turn region normalizer on or off.
+   *
+   * @return Previous normalizer value
+   */
   public boolean setNormalizerRunning(final boolean on) throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
@@ -1485,8 +2218,15 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Enable/Disable the catalog janitor
+   * @param enable if true enables the catalog janitor
+   * @return the previous state
+   * @throws MasterNotRunningException
+   */
   @Override
-  public boolean enableCatalogJanitor(final boolean enable) throws IOException {
+  public boolean enableCatalogJanitor(final boolean enable)
+      throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
@@ -1496,6 +2236,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Ask for a scan of the catalog table
+   * @return the number of entries cleaned
+   * @throws MasterNotRunningException
+   */
   @Override
   public int runCatalogScan() throws IOException {
     return executeCallable(new MasterCallable<Integer>(getConnection()) {
@@ -1507,6 +2252,10 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Query on the catalog janitor state (Enabled/Disabled?)
+   * @throws org.apache.hadoop.hbase.MasterNotRunningException
+   */
   @Override
   public boolean isCatalogJanitorEnabled() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@@ -1570,13 +2319,21 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
-  public void split(final TableName tableName) throws IOException {
+  public void split(final TableName tableName)
+    throws IOException {
     split(tableName, null);
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
-  public void splitRegion(final byte[] regionName) throws IOException {
+  public void splitRegion(final byte[] regionName)
+    throws IOException {
     splitRegion(regionName, null);
   }
 
@@ -1584,7 +2341,8 @@ public class HBaseAdmin implements Admin {
    * {@inheritDoc}
    */
   @Override
-  public void split(final TableName tableName, final byte [] splitPoint) throws IOException {
+  public void split(final TableName tableName, final byte [] splitPoint)
+  throws IOException {
     ZooKeeperWatcher zookeeper = null;
     try {
       checkTableExists(tableName);
@@ -1615,8 +2373,12 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
-  public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
+  public void splitRegion(final byte[] regionName, final byte [] splitPoint)
+  throws IOException {
     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
     if (regionServerPair == null) {
       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
@@ -1644,6 +2406,20 @@ public class HBaseAdmin implements Admin {
     ProtobufUtil.split(admin, hri, splitPoint);
   }
 
+  /**
+   * Modify an existing table, more IRB friendly version. Asynchronous operation.
+   * This means that it may be a while before your schema change is updated across all of the
+   * table. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   *
+   * @param tableName name of table.
+   * @param htd modified description of the table
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
+   *     operation to complete.
+   */
   @Override
   public Future<Void> modifyTable(final TableName tableName, final HTableDescriptor htd)
   throws IOException {
@@ -1692,11 +2468,21 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  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 regionName Name of a region.
    * @return a pair of HRegionInfo and ServerName if <code>regionName</code> is
    *  a verified region name (we call {@link
-   *  MetaTableAccessor#getRegionLocation(HConnection, byte[])}
+   *  MetaTableAccessor#getRegion(HConnection, byte[])}
    *  else null.
    * Throw IllegalArgumentException if <code>regionName</code> is null.
    * @throws IOException
@@ -1787,6 +2573,10 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Shuts down the HBase cluster
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public synchronized void shutdown() throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -1798,6 +2588,12 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Shuts down the current HBase master only.
+   * Does not shutdown the cluster.
+   * @see #shutdown()
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public synchronized void stopMaster() throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -1809,6 +2605,12 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Stop the designated regionserver
+   * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
+   * <code>example.org:1234</code>
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public synchronized void stopRegionServer(final String hostnamePort)
   throws IOException {
@@ -1825,6 +2627,11 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+
+  /**
+   * @return cluster status
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public ClusterStatus getClusterStatus() throws IOException {
     return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
@@ -1836,11 +2643,19 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * @return Configuration used by the instance.
+   */
   @Override
   public Configuration getConfiguration() {
     return this.conf;
   }
 
+  /**
+   * Create a new namespace
+   * @param descriptor descriptor which describes the new namespace
+   * @throws IOException
+   */
   @Override
   public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -1856,6 +2671,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Modify an existing namespace
+   * @param descriptor descriptor which describes the new namespace
+   * @throws IOException
+   */
   @Override
   public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -1868,6 +2688,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+   * @param name namespace name
+   * @throws IOException
+   */
   @Override
   public void deleteNamespace(final String name) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -1880,6 +2705,12 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Get a namespace descriptor by name
+   * @param name name of namespace descriptor
+   * @return A descriptor
+   * @throws IOException
+   */
   @Override
   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
     return
@@ -1893,6 +2724,11 @@ public class HBaseAdmin implements Admin {
         });
   }
 
+  /**
+   * List available namespace descriptors
+   * @return List of descriptors
+   * @throws IOException
+   */
   @Override
   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
     return
@@ -1911,6 +2747,11 @@ public class HBaseAdmin implements Admin {
         });
   }
 
+  /**
+   * List procedures
+   * @return procedure list
+   * @throws IOException
+   */
   @Override
   public ProcedureInfo[] listProcedures() throws IOException {
     return
@@ -1928,6 +2769,12 @@ public class HBaseAdmin implements Admin {
         });
   }
 
+  /**
+   * Get list of table descriptors by namespace
+   * @param name namespace name
+   * @return A descriptor
+   * @throws IOException
+   */
   @Override
   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
     return
@@ -1947,6 +2794,12 @@ public class HBaseAdmin implements Admin {
         });
   }
 
+  /**
+   * Get list of table names by namespace
+   * @param name namespace name
+   * @return The list of table names in the namespace
+   * @throws IOException
+   */
   @Override
   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
     return
@@ -2005,6 +2858,13 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * get the regions of a given table.
+   *
+   * @param tableName the name of the table
+   * @return Ordered list of {@link HRegionInfo}.
+   * @throws IOException
+   */
   @Override
   public List<HRegionInfo> getTableRegions(final TableName tableName)
   throws IOException {
@@ -2024,10 +2884,21 @@ public class HBaseAdmin implements Admin {
     return regions;
   }
 
+  public List<HRegionInfo> getTableRegions(final byte[] tableName)
+  throws IOException {
+    return getTableRegions(TableName.valueOf(tableName));
+  }
+
   @Override
   public synchronized void close() throws IOException {
   }
 
+  /**
+   * Get tableDescriptors
+   * @param tableNames List of table names
+   * @return HTD[] the tableDescriptor
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
   throws IOException {
@@ -2062,6 +2933,12 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Get tableDescriptors
+   * @param names List of table names
+   * @return HTD[] the tableDescriptor
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public HTableDescriptor[] getTableDescriptors(List<String> names)
   throws IOException {
@@ -2138,12 +3015,18 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public CompactionState getCompactionState(final TableName tableName)
   throws IOException {
     return getCompactionState(tableName, CompactType.NORMAL);
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public CompactionState getCompactionStateForRegion(final byte[] regionName)
   throws IOException {
@@ -2166,6 +3049,22 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
+   * taken. If the table is disabled, an offline snapshot is taken.
+   * <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 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
+   */
   @Override
   public void snapshot(final String snapshotName,
                        final TableName tableName) throws IOException,
@@ -2173,16 +3072,86 @@ public class HBaseAdmin implements Admin {
     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
   }
 
-  @Override
-  public void snapshot(final byte[] snapshotName, final TableName tableName)
-      throws IOException, SnapshotCreationException, IllegalArgumentException {
+  public void snapshot(final String snapshotName,
+                       final String tableName) throws IOException,
+      SnapshotCreationException, IllegalArgumentException {
+    snapshot(snapshotName, TableName.valueOf(tableName),
+        SnapshotDescription.Type.FLUSH);
+  }
+
+  /**
+   * Create snapshot for the given table of given flush type.
+   * <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.
+   * @param snapshotName name of the snapshot to be created
+   * @param tableName name of the table for which snapshot is created
+   * @param flushType if the snapshot should be taken without flush memstore first
+   * @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,
+                       final SnapshotDescription.Type flushType) throws
+      IOException, SnapshotCreationException, IllegalArgumentException {
+      snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName), flushType);
+  }
+  /**
+   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.
+    * @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
+    */
+  @Override
+  public void snapshot(final byte[] snapshotName,
+                       final TableName tableName) throws IOException,
+      SnapshotCreationException, IllegalArgumentException {
     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);
+  }
+
+  /**
+   * Create typed snapshot of the 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 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
+   * @param tableName name of the table to snapshot
+   * @param type type of snapshot to take
+   * @throws IOException we fail to reach the master
+   * @throws SnapshotCreationException if snapshot creation failed
+   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
+   */
   @Override
-  public void snapshot(final String snapshotName, final TableName tableName,
-      SnapshotDescription.Type type)
-      throws IOException, SnapshotCreationException, IllegalArgumentException {
+  public void snapshot(final String snapshotName,
+                       final TableName tableName,
+                      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
+      IllegalArgumentException {
     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
     builder.setTable(tableName.getNameAsString());
     builder.setName(snapshotName);
@@ -2190,6 +3159,41 @@ public class HBaseAdmin implements Admin {
     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>
+   * Only a single snapshot should be taken at a time for an instance of HBase, or results may be
+   * undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
+   * time for a single cluster).
+   * <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 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.
+   * @param snapshot snapshot to take
+   * @throws IOException or we lose contact with the master.
+   * @throws SnapshotCreationException if snapshot failed to be taken
+   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
+   */
   @Override
   public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
       IllegalArgumentException {
@@ -2231,6 +3235,16 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
+   * <p>
+   * Only a single snapshot should be taken at a time, or results may be undefined.
+   * @param snapshot snapshot to take
+   * @return response from the server indicating the max time to wait for the snapshot
+   * @throws IOException if the snapshot did not succeed or we lose contact with the master.
+   * @throws SnapshotCreationException if snapshot creation failed
+   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
+   */
   @Override
   public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
       SnapshotCreationException {
@@ -2246,6 +3260,26 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Check the current state of the passed snapshot.
+   * <p>
+   * There are three possible states:
+   * <ol>
+   * <li>running - returns <tt>false</tt></li>
+   * <li>finished - returns <tt>true</tt></li>
+   * <li>finished with error - throws the exception that caused the snapshot to fail</li>
+   * </ol>
+   * <p>
+   * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
+   * run/started since the snapshot your are checking, you will recieve an
+   * {@link UnknownSnapshotException}.
+   * @param snapshot description of the snapshot to check
+   * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
+   *         running
+   * @throws IOException if we have a network issue
+   * @throws HBaseSnapshotException if the snapshot failed
+   * @throws UnknownSnapshotException if the requested snapshot is unknown
+   */
   @Override
   public boolean isSnapshotFinished(final SnapshotDescription snapshot)
       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
@@ -2259,12 +3293,38 @@ public class HBaseAdmin implements Admin {
     }).getDone();
   }
 
+  /**
+   * Restore the specified snapshot on the original table. (The table must be disabled)
+   * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
+   * is set to true, a snapshot of the current table is taken
+   * before executing the restore operation.
+   * In case of restore failure, the failsafe snapshot will be restored.
+   * If the restore completes without problem the failsafe snapshot is deleted.
+   *
+   * @param snapshotName name of the snapshot to restore
+   * @throws IOException if a remote or network exception occurs
+   * @throws RestoreSnapshotException if snapshot failed to be restored
+   * @throws IllegalArgumentException if the restore request is formatted incorrectly
+   */
   @Override
   public void restoreSnapshot(final byte[] snapshotName)
       throws IOException, RestoreSnapshotException {
     restoreSnapshot(Bytes.toString(snapshotName));
   }
 
+  /**
+   * Restore the specified snapshot on the original table. (The table must be disabled)
+   * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
+   * is set to true, a snapshot of the current table is taken
+   * before executing the restore operation.
+   * In case of restore failure, the failsafe snapshot will be restored.
+   * If the restore completes without problem the failsafe snapshot is deleted.
+   *
+   * @param snapshotName name of the snapshot to restore
+   * @throws IOException if a remote or network exception occurs
+   * @throws RestoreSnapshotException if snapshot failed to be restored
+   * @throws IllegalArgumentException if the restore request is formatted incorrectly
+   */
   @Override
   public void restoreSnapshot(final String snapshotName)
       throws IOException, RestoreSnapshotException {
@@ -2273,12 +3333,44 @@ public class HBaseAdmin implements Admin {
     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
   }
 
+  /**
+   * Restore the specified snapshot on the original table. (The table must be disabled)
+   * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
+   * before executing the restore operation.
+   * In case of restore failure, the failsafe snapshot will be restored.
+   * If the restore completes without problem the failsafe snapshot is deleted.
+   *
+   * The failsafe snapshot name is configurable by using the property
+   * "hbase.snapshot.restore.failsafe.name".
+   *
+   * @param snapshotName name of the snapshot to restore
+   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
+   * @throws IOException if a remote or network exception occurs
+   * @throws RestoreSnapshotException if snapshot failed to be restored
+   * @throws IllegalArgumentException if the restore request is formatted incorrectly
+   */
   @Override
   public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
       throws IOException, RestoreSnapshotException {
     restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
   }
 
+  /**
+   * Restore the specified snapshot on the original table. (The table must be disabled)
+   * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
+   * before executing the restore operation.
+   * In case of restore failure, the failsafe snapshot will be restored.
+   * If the restore completes without problem the failsafe snapshot is deleted.
+   *
+   * The failsafe snapshot name is configurable by using the property
+   * "hbase.snapshot.restore.failsafe.name".
+   *
+   * @param snapshotName name of the snapshot to restore
+   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
+   * @throws IOException if a remote or network exception occurs
+   * @throws RestoreSnapshotException if snapshot failed to be restored
+   * @throws IllegalArgumentException if the restore request is formatted incorrectly
+   */
   @Override
   public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot)
       throws IOException, RestoreSnapshotException {
@@ -2353,12 +3445,64 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * 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 byte[] tableName)
+      throws IOException, TableExistsException, RestoreSnapshotException {
+    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
+   */
   @Override
   public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
       throws IOException, TableExistsException, RestoreSnapshotException {
     cloneSnapshot(Bytes.toString(snapshotName), 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 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
+   */
   @Override
   public void cloneSnapshot(final String snapshotName, final TableName tableName)
       throws IOException, TableExistsException, RestoreSnapshotException {
@@ -2369,9 +3513,20 @@ public class HBaseAdmin implements Admin {
     waitUntilTableIsEnabled(tableName);
   }
 
+  /**
+   * Execute a distributed procedure on a cluster synchronously with return data
+   *
+   * @param signature A distributed procedure is uniquely identified
+   * by its signature (default the root ZK node name of the procedure).
+   * @param instance The instance name of the procedure. For some procedures, this parameter is
+   * optional.
+   * @param props Property/Value pairs of properties passing to the procedure
+   * @return data returned after procedure execution. null if no return data.
+   * @throws IOException
+   */
   @Override
-  public byte[] execProcedureWithRet(String signature, String instance, Map<String, String> props)
-      throws IOException {
+  public byte[] execProcedureWithRet(String signature, String instance,
+      Map<String, String> props) throws IOException {
     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
     builder.setSignature(signature).setInstance(instance);
     for (Entry<String, String> entry : props.entrySet()) {
@@ -2393,10 +3548,19 @@ public class HBaseAdmin implements Admin {
 
     return response.hasReturnData() ? response.getReturnData().toByteArray() : null;
   }
-
+  /**
+   * Execute a distributed procedure on a cluster.
+   *
+   * @param signature A distributed procedure is uniquely identified
+   * by its signature (default the root ZK node name of the procedure).
+   * @param instance The instance name of the procedure. For some procedures, this parameter is
+   * optional.
+   * @param props Property/Value pairs of properties passing to the procedure
+   * @throws IOException
+   */
   @Override
-  public void execProcedure(String signature, String instance, Map<String, String> props)
-      throws IOException {
+  public void execProcedure(String signature, String instance,
+      Map<String, String> props) throws IOException {
     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
     builder.setSignature(signature).setInstance(instance);
     for (Entry<String, String> entry : props.entrySet()) {
@@ -2444,6 +3608,23 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Check the current state of the specified procedure.
+   * <p>
+   * There are three possible states:
+   * <ol>
+   * <li>running - returns <tt>false</tt></li>
+   * <li>finished - returns <tt>true</tt></li>
+   * <li>finished with error - throws the exception that caused the procedure to fail</li>
+   * </ol>
+   * <p>
+   *
+   * @param signature The signature that uniquely identifies a procedure
+   * @param instance The instance name of the procedure
+   * @param props Property/Value pairs of properties passing to the procedure
+   * @return true if the specified procedure is finished successfully, false if it is still running
+   * @throws IOException if the specified procedure finished with error
+   */
   @Override
   public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
       throws IOException {
@@ -2475,7 +3656,8 @@ public class HBaseAdmin implements Admin {
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted incorrectly
    */
-  private void internalRestoreSnapshot(final String snapshotName, final TableName tableName)
+  private void internalRestoreSnapshot(final String snapshotName, final TableName
+      tableName)
       throws IOException, RestoreSnapshotException {
     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
         .setName(snapshotName).setTable(tableName.getNameAsString()).build();
@@ -2540,6 +3722,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * List completed snapshots.
+   * @return a list of snapshot descriptors for completed snapshots
+   * @throws IOException if a network error occurs
+   */
   @Override
   public List<SnapshotDescription> listSnapshots() throws IOException {
     return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
@@ -2551,11 +3738,25 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * List all the completed snapshots matching the given regular expression.
+   *
+   * @param regex The regular expression to match against
+   * @return - returns a List of SnapshotDescription
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
     return listSnapshots(Pattern.compile(regex));
   }
 
+  /**
+   * List all the completed snapshots matching the given pattern.
+   *
+   * @param pattern The compiled regular expression to match against
+   * @return - returns a List of SnapshotDescription
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
     List<SnapshotDescription> matched = new LinkedList<SnapshotDescription>();
@@ -2568,12 +3769,28 @@ public class HBaseAdmin implements Admin {
     return matched;
   }
 
+  /**
+   * List all the completed snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNameRegex The table name regular expression to match against
+   * @param snapshotNameRegex The snapshot name regular expression to match against
+   * @return returns a List of completed SnapshotDescription
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
       String snapshotNameRegex) throws IOException {
     return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
   }
 
+  /**
+   * List all the completed snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNamePattern The compiled table name regular expression to match against
+   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
+   * @return returns a List of completed SnapshotDescription
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
       Pattern snapshotNamePattern) throws IOException {
@@ -2591,11 +3808,21 @@ public class HBaseAdmin implements Admin {
     return tableSnapshots;
   }
 
+  /**
+   * Delete an existing snapshot.
+   * @param snapshotName name of the snapshot
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
     deleteSnapshot(Bytes.toString(snapshotName));
   }
 
+  /**
+   * Delete an existing snapshot.
+   * @param snapshotName name of the snapshot
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void deleteSnapshot(final String snapshotName) throws IOException {
     // make sure the snapshot is possibly valid
@@ -2613,11 +3840,21 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Delete existing snapshots whose names match the pattern passed.
+   * @param regex The regular expression to match against
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void deleteSnapshots(final String regex) throws IOException {
     deleteSnapshots(Pattern.compile(regex));
   }
 
+  /**
+   * Delete existing snapshots whose names match the pattern passed.
+   * @param pattern pattern for names of the snapshot to match
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void deleteSnapshots(final Pattern pattern) throws IOException {
     List<SnapshotDescription> snapshots = listSnapshots(pattern);
@@ -2643,12 +3880,26 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Delete all existing snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNameRegex The table name regular expression to match against
+   * @param snapshotNameRegex The snapshot name regular expression to match against
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex)
       throws IOException {
     deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
   }
 
+  /**
+   * Delete all existing snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNamePattern The compiled table name regular expression to match against
+   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
       throws IOException {
@@ -2663,6 +3914,12 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Apply the new quota settings.
+   *
+   * @param quota the quota settings
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public void setQuota(final QuotaSettings quota) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -2674,6 +3931,13 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * Return a Quota Scanner to list the quotas based on the filter.
+   *
+   * @param filter the quota settings filter
+   * @return the quota scanner
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
     return QuotaRetriever.open(conf, filter);
@@ -2694,6 +3958,27 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Creates and returns a {@link com.google.protobuf.RpcChannel} instance
+   * connected to the active master.
+   *
+   * <p>
+   * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
+   * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
+   * </p>
+   *
+   * <div style="background-color: #cccccc; padding: 2px">
+   * <blockquote><pre>
+   * CoprocessorRpcChannel channel = myAdmin.coprocessorService();
+   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
+   * MyCallRequest request = MyCallRequest.newBuilder()
+   *     ...
+   *     .build();
+   * MyCallResponse response = service.myCall(null, request);
+   * </pre></blockquote></div>
+   *
+   * @return A MasterCoprocessorRpcChannel instance
+   */
   @Override
   public CoprocessorRpcChannel coprocessorService() {
     return new MasterCoprocessorRpcChannel(connection);
@@ -2715,6 +4000,28 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   * Creates and returns a {@link com.google.protobuf.RpcChannel} instance
+   * connected to the passed region server.
+   *
+   * <p>
+   * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
+   * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
+   * </p>
+   *
+   * <div style="background-color: #cccccc; padding: 2px">
+   * <blockquote><pre>
+   * CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName);
+   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
+   * MyCallRequest request = MyCallRequest.newBuilder()
+   *     ...
+   *     .build();
+   * MyCallResponse response = service.myCall(null, request);
+   * </pre></blockquote></div>
+   *
+   * @param sn the server name to which the endpoint call is made
+   * @return A RegionServerCoprocessorRpcChannel instance
+   */
   @Override
   public CoprocessorRpcChannel coprocessorService(ServerName sn) {
     return new RegionServerCoprocessorRpcChannel(connection, sn);

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
index d9c7caa..dffde55 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
@@ -718,7 +718,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         }
         TableName tableName = selected.getTableName();
         LOG.info("Deleting column family: " + cfd + " from table: " + tableName);
-        admin.deleteColumnFamily(tableName, cfd.getName());
+        admin.deleteColumn(tableName, cfd.getName());
         // assertion
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
         Assert.assertFalse("Column family: " + cfd + " was not added",