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:42 UTC

[1/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

Repository: hbase
Updated Branches:
  refs/heads/master ed4e00119 -> 92e178df2


http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
index 13a5936..82a599c 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
@@ -121,7 +121,7 @@ public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
       if(Arrays.equals(columnDescriptor.getName(), mobColumnFamily)) {
         columnDescriptor.setMobEnabled(true);
         columnDescriptor.setMobThreshold((long) threshold);
-        admin.modifyColumnFamily(tableName, columnDescriptor);
+        admin.modifyColumn(tableName, columnDescriptor);
       }
     }
     LOG.info("Enabling table " + getTablename());

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 5333c15..5bb25db 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Consistency;
@@ -2680,15 +2681,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
 
   /**
    * Returns a Admin instance.
-   * This instance is shared between HBaseTestingUtility instance users. Closing it has no effect,
-   * it will be closed automatically when the cluster shutdowns
+   * This instance is shared between HBaseTestingUtility instance users.
+   * Closing it has no effect, it will be closed automatically when the
+   * cluster shutdowns
    *
-   * @return HBaseAdmin instance which is guaranteed to support only {@link Admin} interface.
-   *     Functions in HBaseAdmin not provided by {@link Admin} interface can be changed/deleted
-   *     anytime.
-   * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #getAdmin()} instead.
+   * @return An Admin instance.
+   * @throws IOException
    */
-  @Deprecated
   public synchronized HBaseAdmin getHBaseAdmin()
   throws IOException {
     if (hbaseAdmin == null){
@@ -2697,19 +2696,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     return hbaseAdmin;
   }
 
-  /**
-   * Returns an Admin instance which is shared between HBaseTestingUtility instance users.
-   * Closing it has no effect, it will be closed automatically when the cluster shutdowns
-   */
-  public synchronized Admin getAdmin() throws IOException {
-    if (hbaseAdmin == null){
-      this.hbaseAdmin = (HBaseAdmin) getConnection().getAdmin();
-    }
-    return hbaseAdmin;
-  }
-
   private HBaseAdmin hbaseAdmin = null;
 
+
   /**
    * Returns a ZooKeeperWatcher instance.
    * This instance is shared between HBaseTestingUtility instance users.

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
index 989192d..5450148 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
@@ -88,7 +88,7 @@ public class TestAcidGuarantees implements Tool {
       // force mob enabled such that all data is mob data
       hcd.setMobEnabled(true);
       hcd.setMobThreshold(4);
-      util.getHBaseAdmin().modifyColumnFamily(TABLE_NAME, hcd);
+      util.getHBaseAdmin().modifyColumn(TABLE_NAME, hcd);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index e510d28..0f286b6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -465,8 +465,8 @@ public class TestAdmin2 {
         onlineRegions.contains(info));
   }
 
-  private HBaseAdmin createTable(TableName tableName) throws IOException {
-    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+  private Admin createTable(TableName tableName) throws IOException {
+    Admin admin = TEST_UTIL.getHBaseAdmin();
 
     HTableDescriptor htd = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor("value");

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 5e302d2..7278892 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -400,8 +400,8 @@ public class TestMetaWithReplicas {
       }
     }
     assert(moveToServer != null);
-    TableName tableName = TableName.valueOf("randomTable5678");
-    TEST_UTIL.createTable(tableName, "f");
+    String tableName = "randomTable5678";
+    TEST_UTIL.createTable(TableName.valueOf(tableName), "f");
     assertTrue(TEST_UTIL.getHBaseAdmin().tableExists(tableName));
     TEST_UTIL.getHBaseAdmin().move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
         Bytes.toBytes(moveToServer.getServerName()));
@@ -413,8 +413,8 @@ public class TestMetaWithReplicas {
       i++;
     } while (!moveToServer.equals(currentServer) && i < 1000); //wait for 10 seconds overall
     assert(i != 1000);
-    TEST_UTIL.getHBaseAdmin().disableTable(tableName);
-    assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled(tableName));
+    TEST_UTIL.getHBaseAdmin().disableTable("randomTable5678");
+    assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled("randomTable5678"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
index 267201f..fff6f44 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
@@ -100,7 +100,7 @@ public class TestExpiredMobFileCleaner {
     int timeToLive = expireDays * secondsOfDay();
     hcd.setTimeToLive(timeToLive);
 
-    admin.modifyColumnFamily(tableName, hcd);
+    admin.modifyColumn(tableName, hcd);
   }
 
   private void putKVAndFlush(BufferedMutator table, byte[] row, byte[] value, long ts)

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
index f3cfc39..e556a58 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 import java.util.Random;
 
@@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -149,7 +151,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-TTL");
+    String tableName = this.tableName.getNameAsString()+"-TTL";
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
@@ -178,7 +180,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-MinVersion");
+    String tableName = this.tableName.getNameAsString()+"-MinVersion";
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
@@ -209,7 +211,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-BlockingStoreFiles");
+    String tableName = this.tableName.getNameAsString()+"-MinVersion";
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 0b42f7c..ac201cd 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -201,7 +201,7 @@ module Hbase
     def enable(table_name)
       tableExists(table_name)
       return if enabled?(table_name)
-      @admin.enableTable(TableName.valueOf(table_name))
+      @admin.enableTable(table_name)
     end
 
     #----------------------------------------------------------------------------------------------
@@ -216,7 +216,7 @@ module Hbase
     def disable(table_name)
       tableExists(table_name)
       return if disabled?(table_name)
-      @admin.disableTable(TableName.valueOf(table_name))
+      @admin.disableTable(table_name)
     end
 
     #----------------------------------------------------------------------------------------------
@@ -235,15 +235,14 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Is table disabled?
     def disabled?(table_name)
-      @admin.isTableDisabled(TableName.valueOf(table_name))
+      @admin.isTableDisabled(table_name)
     end
 
     #----------------------------------------------------------------------------------------------
     # Drops a table
     def drop(table_name)
       tableExists(table_name)
-      raise ArgumentError, "Table #{table_name} is enabled. Disable it first." if enabled?(
-          table_name)
+      raise ArgumentError, "Table #{table_name} is enabled. Disable it first." if enabled?(table_name)
 
       @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name))
     end
@@ -432,17 +431,15 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Truncates table (deletes all records by recreating the table)
-    def truncate(table_name_str, conf = @conf)
-      table_name = TableName.valueOf(table_name_str)
-      table_description = @admin.getTableDescriptor(table_name)
-      raise ArgumentError, "Table #{table_name_str} is not enabled. Enable it first." unless
-          enabled?(table_name_str)
+    def truncate(table_name, conf = @conf)
+      table_description = @admin.getTableDescriptor(TableName.valueOf(table_name))
+      raise ArgumentError, "Table #{table_name} is not enabled. Enable it first." unless enabled?(table_name)
       yield 'Disabling table...' if block_given?
       @admin.disableTable(table_name)
 
       begin
         yield 'Truncating table...' if block_given?
-        @admin.truncateTable(table_name, false)
+        @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), false)
       rescue => e
         # Handle the compatibility case, where the truncate method doesn't exists on the Master
         raise e unless e.respond_to?(:cause) && e.cause != nil
@@ -450,7 +447,7 @@ module Hbase
         if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then
           # Handle the compatibility case, where the truncate method doesn't exists on the Master
           yield 'Dropping table...' if block_given?
-          @admin.deleteTable(table_name)
+          @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name))
 
           yield 'Creating table...' if block_given?
           @admin.createTable(table_description)
@@ -462,22 +459,21 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Truncates table while maintaing region boundaries (deletes all records by recreating the table)
-    def truncate_preserve(table_name_str, conf = @conf)
-      table_name = TableName.valueOf(table_name_str)
-      h_table = @connection.getTable(table_name)
-      locator = @connection.getRegionLocator(table_name)
+    def truncate_preserve(table_name, conf = @conf)
+      h_table = @connection.getTable(TableName.valueOf(table_name))
+      locator = @connection.getRegionLocator(TableName.valueOf(table_name))
       splits = locator.getAllRegionLocations().
           map{|i| Bytes.toString(i.getRegionInfo().getStartKey)}.
           delete_if{|k| k == ""}.to_java :String
       locator.close()
 
-      table_description = @admin.getTableDescriptor(table_name)
+      table_description = @admin.getTableDescriptor(TableName.valueOf(table_name))
       yield 'Disabling table...' if block_given?
-      disable(table_name_str)
+      disable(table_name)
 
       begin
         yield 'Truncating table...' if block_given?
-        @admin.truncateTable(table_name, true)
+        @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), true)
       rescue => e
         # Handle the compatibility case, where the truncate method doesn't exists on the Master
         raise e unless e.respond_to?(:cause) && e.cause != nil
@@ -485,7 +481,7 @@ module Hbase
         if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then
           # Handle the compatibility case, where the truncate method doesn't exists on the Master
           yield 'Dropping table...' if block_given?
-          @admin.deleteTable(table_name)
+          @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name))
 
           yield 'Creating table with region boundaries...' if block_given?
           @admin.createTable(table_description, splits)
@@ -519,21 +515,18 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Change table structure or table options
-    def alter(table_name_str, wait = true, *args)
+    def alter(table_name, wait = true, *args)
       # Table name should be a string
-      raise(ArgumentError, "Table name must be of type String") unless
-          table_name_str.kind_of?(String)
+      raise(ArgumentError, "Table name must be of type String") unless table_name.kind_of?(String)
 
       # Table should exist
-      raise(ArgumentError, "Can't find a table: #{table_name_str}") unless exists?(table_name_str)
+      raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
       # There should be at least one argument
       raise(ArgumentError, "There should be at least one argument but the table name") if args.empty?
 
-      table_name = TableName.valueOf(table_name_str)
-
       # Get table descriptor
-      htd = @admin.getTableDescriptor(table_name)
+      htd = @admin.getTableDescriptor(TableName.valueOf(table_name))
 
       # Process all args
       args.each do |arg|
@@ -561,11 +554,11 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name_str)
+            alter_status(table_name)
           end
 
           # We bypass descriptor when adding column families; refresh it to apply other args correctly.
-          htd = @admin.getTableDescriptor(table_name)
+          htd = @admin.getTableDescriptor(TableName.valueOf(table_name))
           next
         end
 
@@ -575,7 +568,7 @@ module Hbase
           # Delete column family
           if method == "delete"
             raise(ArgumentError, "NAME parameter missing for delete method") unless name
-            @admin.deleteColumn(table_name, name.to_java_bytes)
+            @admin.deleteColumn(table_name, name)
           # Unset table attributes
           elsif method == "table_att_unset"
             raise(ArgumentError, "NAME parameter missing for table_att_unset method") unless name
@@ -592,7 +585,7 @@ module Hbase
               end
               htd.remove(name)
             end
-            @admin.modifyTable(table_name, htd)
+            @admin.modifyTable(table_name.to_java_bytes, htd)
           # Unknown method
           else
             raise ArgumentError, "Unknown method: #{method}"
@@ -604,12 +597,12 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name_str)
+            alter_status(table_name)
           end
 
           if method == "delete"
             # We bypass descriptor when deleting column families; refresh it to apply other args correctly.
-            htd = @admin.getTableDescriptor(table_name)
+            htd = @admin.getTableDescriptor(TableName.valueOf(table_name))
           end
           next
         end
@@ -656,7 +649,7 @@ module Hbase
             arg.delete(key)
           end
 
-          @admin.modifyTable(table_name, htd)
+          @admin.modifyTable(table_name.to_java_bytes, htd)
 
           arg.each_key do |unknown_key|
             puts("Unknown argument ignored: %s" % [unknown_key])
@@ -664,7 +657,7 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name_str)
+            alter_status(table_name)
           end
           next
         end
@@ -781,13 +774,13 @@ module Hbase
 
     # Does table exist?
     def exists?(table_name)
-      @admin.tableExists(TableName.valueOf(table_name))
+      @admin.tableExists(table_name)
     end
 
     #----------------------------------------------------------------------------------------------
     # Is table enabled
     def enabled?(table_name)
-      @admin.isTableEnabled(TableName.valueOf(table_name))
+      @admin.isTableEnabled(table_name)
     end
 
     #----------------------------------------------------------------------------------------------
@@ -899,23 +892,14 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Take a snapshot of specified table
     def snapshot(table, snapshot_name, *args)
-      # Table name should be a string
-      raise(ArgumentError, "Table name must be of type String") unless table.kind_of?(String)
-
-      # Snapshot name should be a string
-      raise(ArgumentError, "Snapshot name must be of type String") unless
-          snapshot_name.kind_of?(String)
-
-      table_name = TableName.valueOf(table)
       if args.empty?
-         @admin.snapshot(snapshot_name, table_name)
+         @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
       else
          args.each do |arg|
             if arg[SKIP_FLUSH] == true
-              @admin.snapshot(snapshot_name, table_name,
-                              SnapshotDescription::Type::SKIPFLUSH)
+              @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes, SnapshotDescription::Type::SKIPFLUSH)
             else
-               @admin.snapshot(snapshot_name, table_name)
+               @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
             end
          end
       end
@@ -924,19 +908,19 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Restore specified snapshot
     def restore_snapshot(snapshot_name)
-      @admin.restoreSnapshot(snapshot_name)
+      @admin.restoreSnapshot(snapshot_name.to_java_bytes)
     end
 
     #----------------------------------------------------------------------------------------------
     # Create a new table by cloning the snapshot content
     def clone_snapshot(snapshot_name, table)
-      @admin.cloneSnapshot(snapshot_name, TableName.valueOf(table))
+      @admin.cloneSnapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
     end
 
     #----------------------------------------------------------------------------------------------
     # Delete specified snapshot
     def delete_snapshot(snapshot_name)
-      @admin.deleteSnapshot(snapshot_name)
+      @admin.deleteSnapshot(snapshot_name.to_java_bytes)
     end
 
     #----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-shell/src/main/ruby/hbase/security.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/security.rb b/hbase-shell/src/main/ruby/hbase/security.rb
index 20f3298..c7b94e8 100644
--- a/hbase-shell/src/main/ruby/hbase/security.rb
+++ b/hbase-shell/src/main/ruby/hbase/security.rb
@@ -64,7 +64,7 @@ module Hbase
             # Table should exist
             raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
-            tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
+            tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes)
             htd = @admin.getTableDescriptor(tableName)
 
             if (family != nil)
@@ -106,7 +106,7 @@ module Hbase
              # Table should exist
              raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
-             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
+             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes)
              htd = @admin.getTableDescriptor(tableName)
 
              if (family != nil)
@@ -165,7 +165,7 @@ module Hbase
 
     # Does table exist?
     def exists?(table_name)
-      @admin.tableExists(TableName.valueOf(table_name))
+      @admin.tableExists(table_name)
     end
 
     def isNamespace?(table_name)

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
index 98bfb2c..0d50d8a 100644
--- a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
+++ b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
@@ -153,7 +153,7 @@ module Hbase
 
     # Does table exist?
     def exists?(table_name)
-      @admin.tableExists(TableName.valueOf(table_name))
+      @admin.tableExists(table_name)
     end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index 0270037..0a1a92e 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -405,21 +405,21 @@ module Hbase
     end
 
     #-------------------------------------------------------------------------------
-    define_test "Snapshot should fail with non-string table name" do
-      assert_raise(ArgumentError) do
+    define_test "Snapshot should fail with non-string snapshot name" do
+      assert_raise(NoMethodError) do
         admin.snapshot(123, 'xxx')
       end
     end
 
-    define_test "Snapshot should fail with non-string snapshot name" do
-      assert_raise(ArgumentError) do
-        admin.snapshot(@test_name, 123)
+    define_test "Snapshot should fail with non-string table name" do
+      assert_raise(NoMethodError) do
+        admin.snapshot(@create_test_snapshot, 123)
       end
     end
 
-    define_test "Snapshot should fail without snapshot name" do
+    define_test "Snapshot should fail without table name" do
       assert_raise(ArgumentError) do
-        admin.snapshot(@test_name)
+        admin.snapshot("hbase_create_test_snapshot")
       end
     end
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/src/main/asciidoc/_chapters/cp.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/cp.adoc b/src/main/asciidoc/_chapters/cp.adoc
index a4587ec..aa2ac0f 100644
--- a/src/main/asciidoc/_chapters/cp.adoc
+++ b/src/main/asciidoc/_chapters/cp.adoc
@@ -421,7 +421,7 @@ onwards.
 +
 [source,java]
 ----
-TableName tableName = TableName.valueOf("users");
+String tableName = "users";
 String path = "hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar";
 Configuration conf = HBaseConfiguration.create();
 HBaseAdmin admin = new HBaseAdmin(conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/src/main/asciidoc/_chapters/external_apis.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/external_apis.adoc b/src/main/asciidoc/_chapters/external_apis.adoc
index 43a428a..4aa5815 100644
--- a/src/main/asciidoc/_chapters/external_apis.adoc
+++ b/src/main/asciidoc/_chapters/external_apis.adoc
@@ -741,7 +741,7 @@ the data, and deletes the table.
 [source,jython]
 ----
 import java.lang
-from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, HConstants, TableName
+from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, HConstants
 from org.apache.hadoop.hbase.client import HBaseAdmin, HTable, Get
 from org.apache.hadoop.hbase.io import Cell, RowResult
 
@@ -753,7 +753,7 @@ conf = HBaseConfiguration()
 # Create a table named 'test' that has two column families,
 # one named 'content, and the other 'anchor'.  The colons
 # are required for column family names.
-tablename = TableName.valueOf("test")
+tablename = "test"
 
 desc = HTableDescriptor(tablename)
 desc.addFamily(HColumnDescriptor("content:"))

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index e8d44eb..db255aa 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -2214,7 +2214,7 @@ or in code it would be as follows:
 
 [source,java]
 ----
-void rename(Admin admin, String oldTableName, TableName newTableName) {
+void rename(Admin admin, String oldTableName, String newTableName) {
   String snapshotName = randomName();
   admin.disableTable(oldTableName);
   admin.snapshot(snapshotName, oldTableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/src/main/asciidoc/_chapters/schema_design.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/schema_design.adoc b/src/main/asciidoc/_chapters/schema_design.adoc
index e2dcbad..926df71 100644
--- a/src/main/asciidoc/_chapters/schema_design.adoc
+++ b/src/main/asciidoc/_chapters/schema_design.adoc
@@ -53,7 +53,7 @@ Tables must be disabled when making ColumnFamily modifications, for example:
 
 Configuration config = HBaseConfiguration.create();
 Admin admin = new Admin(conf);
-TableName table = TableName.valueOf("myTable");
+String table = "myTable";
 
 admin.disableTable(table);
 


[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

Posted by st...@apache.org.
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",


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

Posted by st...@apache.org.
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 patch was not done (Appy brought it to my attention).

This reverts commit 6adce67572a502a067c74f78cf83683dda4d67e4.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/92e178df
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/92e178df
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/92e178df

Branch: refs/heads/master
Commit: 92e178df28120fcae2aad47f3e4b1cc6e06a8404
Parents: ed4e001
Author: stack <st...@apache.org>
Authored: Tue Dec 1 11:29:59 2015 -0800
Committer: stack <st...@apache.org>
Committed: Tue Dec 1 11:30:39 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |    8 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 1417 +++++++++++++++++-
 .../hbase/IntegrationTestDDLMasterFailover.java |    2 +-
 .../hbase/IntegrationTestIngestWithMOB.java     |    2 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   25 +-
 .../apache/hadoop/hbase/TestAcidGuarantees.java |    2 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |    4 +-
 .../hbase/client/TestMetaWithReplicas.java      |    8 +-
 .../hbase/mob/TestExpiredMobFileCleaner.java    |    2 +-
 .../compactions/TestFIFOCompactionPolicy.java   |    8 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |   88 +-
 hbase-shell/src/main/ruby/hbase/security.rb     |    6 +-
 .../src/main/ruby/hbase/visibility_labels.rb    |    2 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   14 +-
 src/main/asciidoc/_chapters/cp.adoc             |    2 +-
 src/main/asciidoc/_chapters/external_apis.adoc  |    4 +-
 src/main/asciidoc/_chapters/ops_mgt.adoc        |    2 +-
 src/main/asciidoc/_chapters/schema_design.adoc  |    2 +-
 18 files changed, 1439 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/92e178df/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index b06902a..290ecb5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -487,10 +487,7 @@ public interface Admin extends Abortable, Closeable {
    * @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
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #getAlterStatus(TableName)}
-   *     instead.
    */
-  @Deprecated
   Pair<Integer, Integer> getAlterStatus(final byte[] tableName) throws IOException;
 
   /**
@@ -1184,8 +1181,9 @@ public interface Admin extends Abortable, Closeable {
       throws IOException, SnapshotCreationException, IllegalArgumentException;
 
   /**
-   * Create a timestamp consistent snapshot for the given table. Snapshots are considered unique
-   * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even
+   * public void snapshot(final String snapshotName, Create a timestamp consistent snapshot for the
+   * given table. final byte[] tableName) throws IOException, 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. Snapshot names follow the same naming constraints as tables in
    * HBase.