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/11 00:02:33 UTC

[1/3] hbase git commit: HBASE-14769 Remove unused functions and duplicate javadocs from HBaseAdmin

Repository: hbase
Updated Branches:
  refs/heads/master 9511150bd -> bebcc09fb


http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 82a599c..13a5936 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.modifyColumn(tableName, columnDescriptor);
+        admin.modifyColumnFamily(tableName, columnDescriptor);
       }
     }
     LOG.info("Enabling table " + getTablename());

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 006c3e7..71a3344 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
@@ -2682,13 +2682,15 @@ 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 An Admin instance.
-   * @throws IOException
+   * @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.
    */
+  @Deprecated
   public synchronized HBaseAdmin getHBaseAdmin()
   throws IOException {
     if (hbaseAdmin == null){
@@ -2697,8 +2699,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     return hbaseAdmin;
   }
 
-  private HBaseAdmin hbaseAdmin = null;
+  /**
+   * 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.

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 5450148..989192d 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().modifyColumn(TABLE_NAME, hcd);
+      util.getHBaseAdmin().modifyColumnFamily(TABLE_NAME, hcd);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 0f286b6..e510d28 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 Admin createTable(TableName tableName) throws IOException {
-    Admin admin = TEST_UTIL.getHBaseAdmin();
+  private HBaseAdmin createTable(TableName tableName) throws IOException {
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
 
     HTableDescriptor htd = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor("value");

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 7278892..5e302d2 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);
-    String tableName = "randomTable5678";
-    TEST_UTIL.createTable(TableName.valueOf(tableName), "f");
+    TableName tableName = TableName.valueOf("randomTable5678");
+    TEST_UTIL.createTable(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("randomTable5678");
-    assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled("randomTable5678"));
+    TEST_UTIL.getHBaseAdmin().disableTable(tableName);
+    assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled(tableName));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 fff6f44..267201f 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.modifyColumn(tableName, hcd);
+    admin.modifyColumnFamily(tableName, hcd);
   }
 
   private void putKVAndFlush(BufferedMutator table, byte[] row, byte[] value, long ts)

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 e556a58..f3cfc39 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,7 +21,6 @@ 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;
 
@@ -41,7 +40,6 @@ 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;
@@ -151,7 +149,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    String tableName = this.tableName.getNameAsString()+"-TTL";
+    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-TTL");
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
@@ -180,7 +178,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    String tableName = this.tableName.getNameAsString()+"-MinVersion";
+    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-MinVersion");
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
@@ -211,7 +209,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    String tableName = this.tableName.getNameAsString()+"-MinVersion";
+    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-BlockingStoreFiles");
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 9a32b0e..c61b598 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -210,7 +210,7 @@ module Hbase
     def enable(table_name)
       tableExists(table_name)
       return if enabled?(table_name)
-      @admin.enableTable(table_name)
+      @admin.enableTable(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -225,7 +225,7 @@ module Hbase
     def disable(table_name)
       tableExists(table_name)
       return if disabled?(table_name)
-      @admin.disableTable(table_name)
+      @admin.disableTable(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -244,14 +244,15 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Is table disabled?
     def disabled?(table_name)
-      @admin.isTableDisabled(table_name)
+      @admin.isTableDisabled(TableName.valueOf(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
@@ -447,15 +448,17 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Truncates table (deletes all records by recreating the table)
-    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)
+    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)
       yield 'Disabling table...' if block_given?
       @admin.disableTable(table_name)
 
       begin
         yield 'Truncating table...' if block_given?
-        @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), false)
+        @admin.truncateTable(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
@@ -463,7 +466,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(org.apache.hadoop.hbase.TableName.valueOf(table_name))
+          @admin.deleteTable(table_name)
 
           yield 'Creating table...' if block_given?
           @admin.createTable(table_description)
@@ -475,9 +478,10 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Truncates table while maintaing region boundaries (deletes all records by recreating the table)
-    def truncate_preserve(table_name, conf = @conf)
-      h_table = @connection.getTable(TableName.valueOf(table_name))
-      locator = @connection.getRegionLocator(TableName.valueOf(table_name))
+    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)
       begin
         splits = locator.getAllRegionLocations().
             map{|i| Bytes.toString(i.getRegionInfo().getStartKey)}.
@@ -486,13 +490,13 @@ module Hbase
         locator.close()
       end
 
-      table_description = @admin.getTableDescriptor(TableName.valueOf(table_name))
+      table_description = @admin.getTableDescriptor(table_name)
       yield 'Disabling table...' if block_given?
-      disable(table_name)
+      disable(table_name_str)
 
       begin
         yield 'Truncating table...' if block_given?
-        @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), true)
+        @admin.truncateTable(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
@@ -500,7 +504,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(org.apache.hadoop.hbase.TableName.valueOf(table_name))
+          @admin.deleteTable(table_name)
 
           yield 'Creating table with region boundaries...' if block_given?
           @admin.createTable(table_description, splits)
@@ -534,18 +538,21 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Change table structure or table options
-    def alter(table_name, wait = true, *args)
+    def alter(table_name_str, wait = true, *args)
       # Table name should be a string
-      raise(ArgumentError, "Table name must be of type String") unless table_name.kind_of?(String)
+      raise(ArgumentError, "Table name must be of type String") unless
+          table_name_str.kind_of?(String)
 
       # Table should exist
-      raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
+      raise(ArgumentError, "Can't find a table: #{table_name_str}") unless exists?(table_name_str)
 
       # 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(TableName.valueOf(table_name))
+      htd = @admin.getTableDescriptor(table_name)
 
       # Process all args
       args.each do |arg|
@@ -573,11 +580,11 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name)
+            alter_status(table_name_str)
           end
 
           # We bypass descriptor when adding column families; refresh it to apply other args correctly.
-          htd = @admin.getTableDescriptor(TableName.valueOf(table_name))
+          htd = @admin.getTableDescriptor(table_name)
           next
         end
 
@@ -587,7 +594,7 @@ module Hbase
           # Delete column family
           if method == "delete"
             raise(ArgumentError, "NAME parameter missing for delete method") unless name
-            @admin.deleteColumn(table_name, name)
+            @admin.deleteColumn(table_name, name.to_java_bytes)
           # Unset table attributes
           elsif method == "table_att_unset"
             raise(ArgumentError, "NAME parameter missing for table_att_unset method") unless name
@@ -604,7 +611,7 @@ module Hbase
               end
               htd.remove(name)
             end
-            @admin.modifyTable(table_name.to_java_bytes, htd)
+            @admin.modifyTable(table_name, htd)
           # Unknown method
           else
             raise ArgumentError, "Unknown method: #{method}"
@@ -616,12 +623,12 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name)
+            alter_status(table_name_str)
           end
 
           if method == "delete"
             # We bypass descriptor when deleting column families; refresh it to apply other args correctly.
-            htd = @admin.getTableDescriptor(TableName.valueOf(table_name))
+            htd = @admin.getTableDescriptor(table_name)
           end
           next
         end
@@ -667,7 +674,7 @@ module Hbase
             arg.delete(key)
           end
 
-          @admin.modifyTable(table_name.to_java_bytes, htd)
+          @admin.modifyTable(table_name, htd)
 
           arg.each_key do |unknown_key|
             puts("Unknown argument ignored: %s" % [unknown_key])
@@ -675,7 +682,7 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name)
+            alter_status(table_name_str)
           end
           next
         end
@@ -792,13 +799,13 @@ module Hbase
 
     # Does table exist?
     def exists?(table_name)
-      @admin.tableExists(table_name)
+      @admin.tableExists(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
     # Is table enabled
     def enabled?(table_name)
-      @admin.isTableEnabled(table_name)
+      @admin.isTableEnabled(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -910,14 +917,23 @@ 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.to_java_bytes, table.to_java_bytes)
+         @admin.snapshot(snapshot_name, table_name)
       else
          args.each do |arg|
             if arg[SKIP_FLUSH] == true
-              @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes, SnapshotDescription::Type::SKIPFLUSH)
+              @admin.snapshot(snapshot_name, table_name,
+                              SnapshotDescription::Type::SKIPFLUSH)
             else
-               @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
+               @admin.snapshot(snapshot_name, table_name)
             end
          end
       end
@@ -926,19 +942,19 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Restore specified snapshot
     def restore_snapshot(snapshot_name)
-      @admin.restoreSnapshot(snapshot_name.to_java_bytes)
+      @admin.restoreSnapshot(snapshot_name)
     end
 
     #----------------------------------------------------------------------------------------------
     # Create a new table by cloning the snapshot content
     def clone_snapshot(snapshot_name, table)
-      @admin.cloneSnapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
+      @admin.cloneSnapshot(snapshot_name, TableName.valueOf(table))
     end
 
     #----------------------------------------------------------------------------------------------
     # Delete specified snapshot
     def delete_snapshot(snapshot_name)
-      @admin.deleteSnapshot(snapshot_name.to_java_bytes)
+      @admin.deleteSnapshot(snapshot_name)
     end
 
     #----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 c7b94e8..20f3298 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.to_java_bytes)
+            tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
             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.to_java_bytes)
+             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
              htd = @admin.getTableDescriptor(tableName)
 
              if (family != nil)
@@ -165,7 +165,7 @@ module Hbase
 
     # Does table exist?
     def exists?(table_name)
-      @admin.tableExists(table_name)
+      @admin.tableExists(TableName.valueOf(table_name))
     end
 
     def isNamespace?(table_name)

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 0d50d8a..98bfb2c 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(table_name)
+      @admin.tableExists(TableName.valueOf(table_name))
     end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 0a1a92e..0270037 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 snapshot name" do
-      assert_raise(NoMethodError) do
+    define_test "Snapshot should fail with non-string table name" do
+      assert_raise(ArgumentError) do
         admin.snapshot(123, 'xxx')
       end
     end
 
-    define_test "Snapshot should fail with non-string table name" do
-      assert_raise(NoMethodError) do
-        admin.snapshot(@create_test_snapshot, 123)
+    define_test "Snapshot should fail with non-string snapshot name" do
+      assert_raise(ArgumentError) do
+        admin.snapshot(@test_name, 123)
       end
     end
 
-    define_test "Snapshot should fail without table name" do
+    define_test "Snapshot should fail without snapshot name" do
       assert_raise(ArgumentError) do
-        admin.snapshot("hbase_create_test_snapshot")
+        admin.snapshot(@test_name)
       end
     end
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/src/main/asciidoc/_chapters/cp.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/cp.adoc b/src/main/asciidoc/_chapters/cp.adoc
index aa2ac0f..a4587ec 100644
--- a/src/main/asciidoc/_chapters/cp.adoc
+++ b/src/main/asciidoc/_chapters/cp.adoc
@@ -421,7 +421,7 @@ onwards.
 +
 [source,java]
 ----
-String tableName = "users";
+TableName tableName = TableName.valueOf("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/bebcc09f/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 4aa5815..43a428a 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
+from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, HConstants, TableName
 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 = "test"
+tablename = TableName.valueOf("test")
 
 desc = HTableDescriptor(tablename)
 desc.addFamily(HColumnDescriptor("content:"))

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 db255aa..e8d44eb 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, String newTableName) {
+void rename(Admin admin, String oldTableName, TableName newTableName) {
   String snapshotName = randomName();
   admin.disableTable(oldTableName);
   admin.snapshot(snapshotName, oldTableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 926df71..e2dcbad 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);
-String table = "myTable";
+TableName table = TableName.valueOf("myTable");
 
 admin.disableTable(table);
 


[2/3] hbase git commit: HBASE-14769 Remove unused functions and duplicate javadocs from HBaseAdmin

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 66079dd..7a50458 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) {
+  HBaseAdmin(ClusterConnection connection) throws IOException {
     this.conf = connection.getConfiguration();
     this.connection = connection;
 
@@ -253,13 +253,6 @@ 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,
@@ -280,18 +273,6 @@ 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,
@@ -339,23 +320,6 @@ 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()) {
@@ -366,16 +330,6 @@ 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);
@@ -410,51 +364,6 @@ 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);
@@ -490,23 +399,13 @@ 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 TableNotFoundException, IOException {
+  public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
      return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, operationTimeout);
   }
 
-  static HTableDescriptor getTableDescriptor(final TableName tableName,
-         HConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
-         int operationTimeout) throws TableNotFoundException, IOException {
-
+  static HTableDescriptor getTableDescriptor(final TableName tableName, HConnection connection,
+      RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout) throws IOException {
       if (tableName == null) return null;
       HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
         @Override
@@ -528,11 +427,6 @@ 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) {
@@ -541,49 +435,12 @@ 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)
@@ -604,23 +461,6 @@ 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 {
@@ -642,21 +482,6 @@ 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 {
@@ -723,21 +548,6 @@ 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);
@@ -756,18 +566,6 @@ 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(
@@ -810,19 +608,6 @@ 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));
@@ -833,7 +618,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(byte[])}
+   * {@link #deleteTable(TableName)}
    *
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be deleted
@@ -853,12 +638,6 @@ 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 {
@@ -879,18 +658,6 @@ 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 {
@@ -947,20 +714,6 @@ 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 {
@@ -980,16 +733,6 @@ 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.
@@ -1031,28 +774,6 @@ 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());
@@ -1088,33 +809,11 @@ 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>();
@@ -1131,25 +830,6 @@ 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 {
@@ -1169,28 +849,6 @@ 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());
@@ -1220,42 +878,17 @@ public class HBaseAdmin implements Admin {
     }
 
     @Override
-    protected Void waitOperationResult(final long deadlineTs)
-        throws IOException, TimeoutException {
+    protected Void waitOperationResult(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>();
@@ -1272,24 +905,9 @@ 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 {
-    checkTableExistence(tableName);
+    checkTableExists(tableName);
     return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException, IOException {
@@ -1301,173 +919,46 @@ 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 {
-    checkTableExistence(tableName);
+    checkTableExists(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<Integer, Integer>(Integer.valueOf(ret
-            .getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions()));
+        Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
+            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));
   }
 
   /**
-   * 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)}.
+   * {@inheritDoc}
+   * @deprecated Since 2.0. Will be removed in 3.0. Use
+   *     {@link #addColumnFamily(TableName, HColumnDescriptor)} instead.
    */
   @Override
   @Deprecated
@@ -1506,52 +997,9 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * 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[])}.
+   * {@inheritDoc}
+   * @deprecated Since 2.0. Will be removed in 3.0. Use
+   *     {@link #deleteColumnFamily(TableName, byte[])} instead.
    */
   @Override
   @Deprecated
@@ -1591,52 +1039,9 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * 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)}.
+   * {@inheritDoc}
+   * @deprecated As of 2.0. Will be removed in 3.0. Use
+   *     {@link #modifyColumnFamily(TableName, HColumnDescriptor)} instead.
    */
   @Override
   @Deprecated
@@ -1675,33 +1080,13 @@ 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) {
@@ -1721,27 +1106,6 @@ 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 {
@@ -1766,33 +1130,19 @@ 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);
@@ -1804,9 +1154,6 @@ 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);
@@ -1816,7 +1163,16 @@ public class HBaseAdmin implements Admin {
     if (regionServerPair.getSecond() == null) {
       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
-    flush(regionServerPair.getSecond(), regionServerPair.getFirst());
+    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);
+    }
   }
 
   private void flush(final ServerName sn, final HRegionInfo hri)
@@ -1840,9 +1196,6 @@ public class HBaseAdmin implements Admin {
     compact(tableName, null, false, CompactType.NORMAL);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void compactRegion(final byte[] regionName)
     throws IOException {
@@ -1878,18 +1231,12 @@ 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 {
@@ -1905,9 +1252,6 @@ public class HBaseAdmin implements Admin {
     compact(tableName, columnFamily, true, CompactType.NORMAL);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
   throws IOException {
@@ -2001,19 +1345,6 @@ 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 {
@@ -2034,13 +1365,6 @@ 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 {
@@ -2056,20 +1380,6 @@ 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 {
@@ -2085,18 +1395,6 @@ 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 {
@@ -2109,12 +1407,6 @@ 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 {
@@ -2128,12 +1420,6 @@ 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()) {
@@ -2154,12 +1440,6 @@ 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()) {
@@ -2171,11 +1451,6 @@ 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()) {
@@ -2187,11 +1462,7 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Query the current state of the region normalizer
-   *
-   * @return true if region normalizer is enabled, false otherwise.
-   */
+  @Override
   public boolean isNormalizerEnabled() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
@@ -2202,11 +1473,7 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Turn region normalizer on or off.
-   *
-   * @return Previous normalizer value
-   */
+  @Override
   public boolean setNormalizerRunning(final boolean on) throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
@@ -2218,15 +1485,8 @@ 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 {
@@ -2236,11 +1496,6 @@ 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()) {
@@ -2252,10 +1507,6 @@ 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()) {
@@ -2319,21 +1570,13 @@ 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);
   }
 
@@ -2341,8 +1584,7 @@ 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);
@@ -2373,12 +1615,8 @@ 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));
@@ -2406,20 +1644,6 @@ 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 {
@@ -2468,21 +1692,11 @@ 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#getRegion(HConnection, byte[])}
+   *  MetaTableAccessor#getRegionLocation(HConnection, byte[])}
    *  else null.
    * Throw IllegalArgumentException if <code>regionName</code> is null.
    * @throws IOException
@@ -2573,10 +1787,6 @@ 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()) {
@@ -2588,12 +1798,6 @@ 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()) {
@@ -2605,12 +1809,6 @@ 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 {
@@ -2627,11 +1825,6 @@ 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()) {
@@ -2643,19 +1836,11 @@ 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()) {
@@ -2671,11 +1856,6 @@ 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()) {
@@ -2688,11 +1868,6 @@ 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()) {
@@ -2705,12 +1880,6 @@ 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
@@ -2724,11 +1893,6 @@ public class HBaseAdmin implements Admin {
         });
   }
 
-  /**
-   * List available namespace descriptors
-   * @return List of descriptors
-   * @throws IOException
-   */
   @Override
   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
     return
@@ -2747,11 +1911,6 @@ public class HBaseAdmin implements Admin {
         });
   }
 
-  /**
-   * List procedures
-   * @return procedure list
-   * @throws IOException
-   */
   @Override
   public ProcedureInfo[] listProcedures() throws IOException {
     return
@@ -2769,12 +1928,6 @@ 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
@@ -2794,12 +1947,6 @@ 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
@@ -2858,13 +2005,6 @@ 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 {
@@ -2884,21 +2024,10 @@ 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 {
@@ -2933,12 +2062,6 @@ 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 {
@@ -3015,18 +2138,12 @@ 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 {
@@ -3049,22 +2166,6 @@ 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,
@@ -3072,86 +2173,16 @@ public class HBaseAdmin implements Admin {
     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
   }
 
-  public void snapshot(final String snapshotName,
-                       final String tableName) throws IOException,
-      SnapshotCreationException, IllegalArgumentException {
-    snapshot(snapshotName, TableName.valueOf(tableName),
-        SnapshotDescription.Type.FLUSH);
-  }
-
-  /**
-   * Create 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 {
+  @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);
@@ -3159,41 +2190,6 @@ 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 {
@@ -3235,16 +2231,6 @@ 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 {
@@ -3260,26 +2246,6 @@ 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 {
@@ -3293,38 +2259,12 @@ 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 {
@@ -3333,44 +2273,12 @@ 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 {
@@ -3445,64 +2353,12 @@ 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 {
@@ -3513,20 +2369,9 @@ 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()) {
@@ -3548,19 +2393,10 @@ 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()) {
@@ -3608,23 +2444,6 @@ 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 {
@@ -3656,8 +2475,7 @@ 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();
@@ -3722,11 +2540,6 @@ 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()) {
@@ -3738,25 +2551,11 @@ 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>();
@@ -3769,28 +2568,12 @@ 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 {
@@ -3808,21 +2591,11 @@ 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
@@ -3840,21 +2613,11 @@ 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);
@@ -3880,26 +2643,12 @@ 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 {
@@ -3914,12 +2663,6 @@ 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()) {
@@ -3931,13 +2674,6 @@ 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);
@@ -3958,27 +2694,6 @@ 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);
@@ -4000,28 +2715,6 @@ 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/bebcc09f/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 dffde55..d9c7caa 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.deleteColumn(tableName, cfd.getName());
+        admin.deleteColumnFamily(tableName, cfd.getName());
         // assertion
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
         Assert.assertFalse("Column family: " + cfd + " was not added",


[3/3] hbase git commit: HBASE-14769 Remove unused functions and duplicate javadocs from HBaseAdmin

Posted by st...@apache.org.
HBASE-14769 Remove unused functions and duplicate javadocs from HBaseAdmin


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

Branch: refs/heads/master
Commit: bebcc09fb392b3494131c792520406c001dbd511
Parents: 9511150
Author: stack <st...@apache.org>
Authored: Thu Dec 10 15:02:21 2015 -0800
Committer: stack <st...@apache.org>
Committed: Thu Dec 10 15:02:21 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       |   24 +-
 .../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, 159 insertions(+), 1438 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/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 290ecb5..b06902a 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,7 +487,10 @@ 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;
 
   /**
@@ -1181,9 +1184,8 @@ public interface Admin extends Abortable, Closeable {
       throws IOException, SnapshotCreationException, IllegalArgumentException;
 
   /**
-   * 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
+   * 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
    * 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.