HBASE-18234 Revisit the async admin api

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

Branch: refs/heads/branch-2
Commit: 289938337b6427a15f8877c0a05b02766ed2bca3
Parents: d9dd319
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Jun 26 10:12:38 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Mon Jun 26 17:30:47 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/AsyncMetaTableAccessor.java    |  116 +-
 .../apache/hadoop/hbase/MetaTableAccessor.java  |    9 +-
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  373 ++----
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    | 1211 ++++++++----------
 .../hbase/shaded/protobuf/ProtobufUtil.java     |   84 +-
 .../hbase/shaded/protobuf/RequestConverter.java |  115 +-
 ...gionServerBulkLoadWithOldSecureEndpoint.java |    3 +-
 .../hbase/client/TestAsyncBalancerAdminApi.java |   10 +-
 .../client/TestAsyncNamespaceAdminApi.java      |    4 +-
 .../client/TestAsyncProcedureAdminApi.java      |    5 +-
 .../hbase/client/TestAsyncRegionAdminApi.java   |   88 +-
 .../hbase/client/TestAsyncSnapshotAdminApi.java |   37 +-
 .../hbase/client/TestAsyncTableAdminApi.java    |   40 +-
 .../regionserver/TestHRegionServerBulkLoad.java |    3 +-
 .../TestHRegionServerBulkLoadWithOldClient.java |    3 +-
 15 files changed, 981 insertions(+), 1120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/28993833/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
index 6988047..18fca6b 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
@@ -22,6 +22,7 @@ import static 
org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -30,6 +31,7 @@ import java.util.SortedMap;
 import java.util.concurrent.CompletableFuture;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,6 +39,7 @@ import 
org.apache.hadoop.hbase.MetaTableAccessor.CollectingVisitor;
 import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
 import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.RawAsyncTable;
@@ -45,6 +48,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.client.Scan.ReadType;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -98,34 +102,72 @@ public class AsyncMetaTableAccessor {
     return future;
   }
 
-  public static CompletableFuture<Pair<HRegionInfo, ServerName>> 
getRegion(RawAsyncTable metaTable,
-      byte[] regionName) {
-    CompletableFuture<Pair<HRegionInfo, ServerName>> future = new 
CompletableFuture<>();
-    byte[] row = regionName;
-    HRegionInfo parsedInfo = null;
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param metaTable
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static CompletableFuture<Optional<HRegionLocation>> getRegionLocation(
+      RawAsyncTable metaTable, byte[] regionName) {
+    CompletableFuture<Optional<HRegionLocation>> future = new 
CompletableFuture<>();
     try {
-      parsedInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
-      row = MetaTableAccessor.getMetaKeyForRegion(parsedInfo);
-    } catch (Exception parseEx) {
-      // Ignore if regionName is a encoded region name.
+      HRegionInfo parsedRegionInfo = 
MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
+      metaTable.get(
+        new Get(MetaTableAccessor.getMetaKeyForRegion(parsedRegionInfo))
+            .addFamily(HConstants.CATALOG_FAMILY)).whenComplete(
+        (r, err) -> {
+          if (err != null) {
+            future.completeExceptionally(err);
+            return;
+          }
+          future.complete(getRegionLocations(r).map(
+            locations -> 
locations.getRegionLocation(parsedRegionInfo.getReplicaId())));
+        });
+    } catch (IOException parseEx) {
+      LOG.warn("Failed to parse the passed region name: " + 
Bytes.toStringBinary(regionName));
+      future.completeExceptionally(parseEx);
     }
+    return future;
+  }
 
-    final HRegionInfo finalHRI = parsedInfo;
-    metaTable.get(new 
Get(row).addFamily(HConstants.CATALOG_FAMILY)).whenComplete((r, err) -> {
-      if (err != null) {
-        future.completeExceptionally(err);
-        return;
-      }
-      RegionLocations locations = MetaTableAccessor.getRegionLocations(r);
-      HRegionLocation hrl = locations == null ? null
-          : locations.getRegionLocation(finalHRI == null ? 0 : 
finalHRI.getReplicaId());
-      if (hrl == null) {
-        future.complete(null);
-      } else {
-        future.complete(new Pair<>(hrl.getRegionInfo(), hrl.getServerName()));
-      }
-    });
-
+  /**
+   * Returns the HRegionLocation from meta for the given encoded region name
+   * @param metaTable
+   * @param encodedRegionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static CompletableFuture<Optional<HRegionLocation>> 
getRegionLocationWithEncodedName(
+      RawAsyncTable metaTable, byte[] encodedRegionName) {
+    CompletableFuture<Optional<HRegionLocation>> future = new 
CompletableFuture<>();
+    metaTable.scanAll(new 
Scan().setReadType(ReadType.PREAD).addFamily(HConstants.CATALOG_FAMILY))
+        .whenComplete(
+          (results, err) -> {
+            if (err != null) {
+              future.completeExceptionally(err);
+              return;
+            }
+            String encodedRegionNameStr = Bytes.toString(encodedRegionName);
+            results
+                .stream()
+                .filter(result -> !result.isEmpty())
+                .filter(result -> MetaTableAccessor.getHRegionInfo(result) != 
null)
+                .forEach(
+                  result -> {
+                    getRegionLocations(result).ifPresent(
+                      locations -> {
+                        for (HRegionLocation location : 
locations.getRegionLocations()) {
+                          if (location != null
+                              && 
encodedRegionNameStr.equals(location.getRegionInfo()
+                                  .getEncodedName())) {
+                            future.complete(Optional.of(location));
+                            return;
+                          }
+                        }
+                      });
+                  });
+            future.complete(Optional.empty());
+          });
     return future;
   }
 
@@ -143,15 +185,29 @@ public class AsyncMetaTableAccessor {
   }
 
   /**
-   * Used to get table regions' info and server.
+   * Used to get all region locations for the specific table.
    * @param metaTable
    * @param tableName table we're looking for, can be null for getting all 
regions
-   * @return the list of regioninfos and server. The return value will be 
wrapped by a
+   * @return the list of region locations. The return value will be wrapped by 
a
    *         {@link CompletableFuture}.
    */
-  public static CompletableFuture<List<Pair<HRegionInfo, ServerName>>> 
getTableRegionsAndLocations(
+  public static CompletableFuture<List<HRegionLocation>> 
getTableHRegionLocations(
       RawAsyncTable metaTable, final Optional<TableName> tableName) {
-    return getTableRegionsAndLocations(metaTable, tableName, true);
+    CompletableFuture<List<HRegionLocation>> future = new 
CompletableFuture<>();
+    getTableRegionsAndLocations(metaTable, tableName, true).whenComplete(
+      (locations, err) -> {
+        if (err != null) {
+          future.completeExceptionally(err);
+        } else if (locations == null || locations.isEmpty()) {
+          future.complete(Collections.emptyList());
+        } else {
+          List<HRegionLocation> regionLocations = locations.stream()
+              .map(loc -> new HRegionLocation(loc.getFirst(), loc.getSecond()))
+              .collect(Collectors.toList());
+          future.complete(regionLocations);
+        }
+      });
+    return future;
   }
 
   /**
@@ -162,7 +218,7 @@ public class AsyncMetaTableAccessor {
    * @return the list of regioninfos and server. The return value will be 
wrapped by a
    *         {@link CompletableFuture}.
    */
-  public static CompletableFuture<List<Pair<HRegionInfo, ServerName>>> 
getTableRegionsAndLocations(
+  private static CompletableFuture<List<Pair<HRegionInfo, ServerName>>> 
getTableRegionsAndLocations(
       RawAsyncTable metaTable, final Optional<TableName> tableName,
       final boolean excludeOfflinedSplitParents) {
     CompletableFuture<List<Pair<HRegionInfo, ServerName>>> future = new 
CompletableFuture<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/28993833/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 9eb5111..3870fd1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -364,13 +364,12 @@ public class MetaTableAccessor {
    * is stored in the name, so the returned object should only be used for the 
fields
    * in the regionName.
    */
-  protected static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName)
-    throws IOException {
+  public static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName) 
throws IOException {
     byte[][] fields = HRegionInfo.parseRegionName(regionName);
-    long regionId =  Long.parseLong(Bytes.toString(fields[2]));
+    long regionId = Long.parseLong(Bytes.toString(fields[2]));
     int replicaId = fields.length > 3 ? 
Integer.parseInt(Bytes.toString(fields[3]), 16) : 0;
-    return new HRegionInfo(
-      TableName.valueOf(fields[0]), fields[1], fields[1], false, regionId, 
replicaId);
+    return new HRegionInfo(TableName.valueOf(fields[0]), fields[1], fields[1], 
false, regionId,
+        replicaId);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/28993833/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 1a3cae2..3b022f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
 import java.util.List;
 import java.util.Collection;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.regex.Pattern;
 
@@ -42,7 +43,7 @@ import org.apache.hadoop.hbase.util.Pair;
  * This feature is still under development, so marked as IA.Private. Will 
change to public when
  * done. Use it with caution.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 public interface AsyncAdmin {
 
   /**
@@ -55,63 +56,50 @@ public interface AsyncAdmin {
    * @return True if table exists already. The return value will be wrapped by 
a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> tableExists(final TableName tableName);
+  CompletableFuture<Boolean> tableExists(TableName tableName);
 
   /**
    * List all the userspace tables.
-   * @return - returns an array of TableDescriptors wrapped by a {@link 
CompletableFuture}.
-   * @see #listTables(Pattern, boolean)
+   * @return - returns a list of TableDescriptors wrapped by a {@link 
CompletableFuture}.
+   * @see #listTables(Optional, boolean)
    */
-  CompletableFuture<TableDescriptor[]> listTables();
-
-  /**
-   * List all the tables matching the given pattern.
-   * @param regex The regular expression to match against
-   * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of TableDescriptors wrapped by a {@link 
CompletableFuture}.
-   * @see #listTables(Pattern, boolean)
-   */
-  CompletableFuture<TableDescriptor[]> listTables(String regex, boolean 
includeSysTables);
+  default CompletableFuture<List<TableDescriptor>> listTables() {
+    return listTables(Optional.empty(), false);
+  }
 
   /**
    * List all the tables matching the given pattern.
    * @param pattern The compiled regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of TableDescriptors wrapped by a {@link 
CompletableFuture}.
-   */
-  CompletableFuture<TableDescriptor[]> listTables(Pattern pattern, boolean 
includeSysTables);
-
-  /**
-   * List all of the names of userspace tables.
-   * @return TableName[] an array of table names wrapped by a {@link 
CompletableFuture}.
-   * @see #listTableNames(Pattern, boolean)
+   * @return - returns a list of TableDescriptors wrapped by a {@link 
CompletableFuture}.
    */
-  CompletableFuture<TableName[]> listTableNames();
+  CompletableFuture<List<TableDescriptor>> listTables(Optional<Pattern> 
pattern,
+      boolean includeSysTables);
 
   /**
    * List all of the names of userspace tables.
-   * @param regex The regular expression to match against
-   * @param includeSysTables False to match only against userspace tables
-   * @return TableName[] an array of table names wrapped by a {@link 
CompletableFuture}.
-   * @see #listTableNames(Pattern, boolean)
+   * @return a list of table names wrapped by a {@link CompletableFuture}.
+   * @see #listTableNames(Optional, boolean)
    */
-  CompletableFuture<TableName[]> listTableNames(final String regex, final 
boolean includeSysTables);
+  default CompletableFuture<List<TableName>> listTableNames() {
+    return listTableNames(Optional.empty(), false);
+  }
 
   /**
    * List all of the names of userspace tables.
    * @param pattern The regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return TableName[] an array of table names wrapped by a {@link 
CompletableFuture}.
+   * @return a list of table names wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<TableName[]> listTableNames(final Pattern pattern,
-      final boolean includeSysTables);
+  CompletableFuture<List<TableName>> listTableNames(Optional<Pattern> pattern,
+      boolean includeSysTables);
 
   /**
    * Method for getting the tableDescriptor
    * @param tableName as a {@link TableName}
    * @return the read-only tableDescriptor wrapped by a {@link 
CompletableFuture}.
    */
-  CompletableFuture<TableDescriptor> getTableDescriptor(final TableName 
tableName);
+  CompletableFuture<TableDescriptor> getTableDescriptor(TableName tableName);
 
   /**
    * Creates a new table.
@@ -140,94 +128,63 @@ public interface AsyncAdmin {
    * @param desc table descriptor for table
    * @param splitKeys array of split keys for the initial regions of the table
    */
-  CompletableFuture<Void> createTable(final TableDescriptor desc, byte[][] 
splitKeys);
+  CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] 
splitKeys);
 
   /**
    * Deletes a table.
    * @param tableName name of table to delete
    */
-  CompletableFuture<Void> deleteTable(final TableName tableName);
-
-  /**
-   * 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(String, boolean)} and
-   * {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be deleted. The return 
value will be wrapped
-   *         by a {@link CompletableFuture}. The return HTDs are read-only.
-   */
-  CompletableFuture<TableDescriptor[]> deleteTables(String regex);
+  CompletableFuture<Void> deleteTable(TableName tableName);
 
   /**
    * Delete 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(Pattern, boolean) } and
+   * {@link #listTableNames(Optional, boolean) } and
    * {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be deleted. The return 
value will be wrapped
    *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<TableDescriptor[]> deleteTables(Pattern pattern);
+  CompletableFuture<List<TableDescriptor>> deleteTables(Pattern pattern);
 
   /**
    * Truncate a table.
    * @param tableName name of table to truncate
    * @param preserveSplits True if the splits should be preserved
    */
-  CompletableFuture<Void> truncateTable(final TableName tableName, final 
boolean preserveSplits);
+  CompletableFuture<Void> truncateTable(TableName tableName, boolean 
preserveSplits);
 
   /**
    * Enable a table. The table has to be in disabled state for it to be 
enabled.
    * @param tableName name of the table
    */
-  CompletableFuture<Void> enableTable(final TableName tableName);
+  CompletableFuture<Void> enableTable(TableName tableName);
 
   /**
    * Enable tables matching the passed in pattern. Warning: Use this method 
carefully, there is no
-   * prompting and the effect is immediate. Consider using {@link 
#listTables(Pattern, boolean)} and
-   * {@link #enableTable(TableName)}
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be enabled. The return 
value will be wrapped
-   *         by a {@link CompletableFuture}. The return HTDs are read-only.
-   */
-  CompletableFuture<TableDescriptor[]> enableTables(String regex);
-
-  /**
-   * Enable tables matching the passed in pattern. Warning: Use this method 
carefully, there is no
-   * prompting and the effect is immediate. Consider using {@link 
#listTables(Pattern, boolean)} and
+   * prompting and the effect is immediate. Consider using {@link 
#listTables(Optional, boolean)} and
    * {@link #enableTable(TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be enabled. The return 
value will be wrapped
    *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<TableDescriptor[]> enableTables(Pattern pattern);
+  CompletableFuture<List<TableDescriptor>> enableTables(Pattern pattern);
 
   /**
    * Disable a table. The table has to be in enabled state for it to be 
disabled.
    * @param tableName
    */
-  CompletableFuture<Void> disableTable(final TableName tableName);
+  CompletableFuture<Void> disableTable(TableName tableName);
 
   /**
    * Disable tables matching the passed in pattern. Warning: Use this method 
carefully, there is no
-   * prompting and the effect is immediate. Consider using {@link 
#listTables(Pattern, boolean)} and
-   * {@link #disableTable(TableName)}
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be disabled. The 
return value will be wrapped by a
-   *         {@link CompletableFuture}. The return HTDs are read-only.
-   */
-  CompletableFuture<TableDescriptor[]> disableTables(String regex);
-
-  /**
-   * Disable tables matching the passed in pattern. Warning: Use this method 
carefully, there is no
-   * prompting and the effect is immediate. Consider using {@link 
#listTables(Pattern, boolean)} and
+   * prompting and the effect is immediate. Consider using {@link 
#listTables(Optional, boolean)} and
    * {@link #disableTable(TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be disabled. The 
return value will be wrapped by a
    *         {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<TableDescriptor[]> disableTables(Pattern pattern);
+  CompletableFuture<List<TableDescriptor>> disableTables(Pattern pattern);
 
   /**
    * @param tableName name of table to check
@@ -261,61 +218,61 @@ public interface AsyncAdmin {
    *         yet to be updated Pair.getSecond() is the total number of regions 
of the table. The
    *         return value will be wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<Pair<Integer, Integer>> getAlterStatus(final TableName 
tableName);
+  CompletableFuture<Pair<Integer, Integer>> getAlterStatus(TableName 
tableName);
 
   /**
    * Add a column family to an existing table.
    * @param tableName name of the table to add column family to
    * @param columnFamily column family descriptor of column family to be added
    */
-  CompletableFuture<Void> addColumnFamily(final TableName tableName,
-      final ColumnFamilyDescriptor columnFamily);
+  CompletableFuture<Void> addColumnFamily(TableName tableName,
+      ColumnFamilyDescriptor columnFamily);
 
   /**
    * Delete a column family from a table.
    * @param tableName name of table
    * @param columnFamily name of column family to be deleted
    */
-  CompletableFuture<Void> deleteColumnFamily(final TableName tableName, final 
byte[] columnFamily);
+  CompletableFuture<Void> deleteColumnFamily(TableName tableName, byte[] 
columnFamily);
 
   /**
    * Modify an existing column family on a table.
    * @param tableName name of table
    * @param columnFamily new column family descriptor to use
    */
-  CompletableFuture<Void> modifyColumnFamily(final TableName tableName,
-      final ColumnFamilyDescriptor columnFamily);
+  CompletableFuture<Void> modifyColumnFamily(TableName tableName,
+      ColumnFamilyDescriptor columnFamily);
 
   /**
    * Create a new namespace.
    * @param descriptor descriptor which describes the new namespace
    */
-  CompletableFuture<Void> createNamespace(final NamespaceDescriptor 
descriptor);
+  CompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor);
 
   /**
    * Modify an existing namespace.
    * @param descriptor descriptor which describes the new namespace
    */
-  CompletableFuture<Void> modifyNamespace(final NamespaceDescriptor 
descriptor);
+  CompletableFuture<Void> modifyNamespace(NamespaceDescriptor descriptor);
 
   /**
    * Delete an existing namespace. Only empty namespaces (no tables) can be 
removed.
    * @param name namespace name
    */
-  CompletableFuture<Void> deleteNamespace(final String name);
+  CompletableFuture<Void> deleteNamespace(String name);
 
   /**
    * Get a namespace descriptor by name
    * @param name name of namespace descriptor
    * @return A descriptor wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(final String 
name);
+  CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name);
 
   /**
    * List available namespace descriptors
    * @return List of descriptors wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<NamespaceDescriptor[]> listNamespaceDescriptors();
+  CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors();
 
   /**
    * @param tableName name of table to check
@@ -329,7 +286,7 @@ public interface AsyncAdmin {
    * @param on
    * @return Previous balancer value wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> setBalancerRunning(final boolean on);
+  CompletableFuture<Boolean> setBalancerOn(boolean on);
 
   /**
    * Invoke the balancer. Will run the balancer and if regions to move, it 
will go ahead and do the
@@ -337,72 +294,38 @@ public interface AsyncAdmin {
    * @return True if balancer ran, false otherwise. The return value will be 
wrapped by a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> balancer();
+  default CompletableFuture<Boolean> balance() {
+    return balance(false);
+  }
 
   /**
    * Invoke the balancer. Will run the balancer and if regions to move, it 
will go ahead and do the
    * reassignments. If there is region in transition, force parameter of true 
would still run
    * balancer. Can *not* run for other reasons. Check logs.
-   * @param force whether we should force balance even if there is region in 
transition.
+   * @param forcible whether we should force balance even if there is region 
in transition.
    * @return True if balancer ran, false otherwise. The return value will be 
wrapped by a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> balancer(boolean force);
+  CompletableFuture<Boolean> balance(boolean forcible);
 
   /**
    * Query the current state of the balancer.
-   * @return true if the balancer is enabled, false otherwise.
-   *         The return value will be wrapped by a {@link CompletableFuture}.
-   */
-  CompletableFuture<Boolean> isBalancerEnabled();
-
-  /**
-   * 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>
+   * @return true if the balance switch is on, false otherwise The return 
value will be wrapped by a
+   *         {@link CompletableFuture}.
    */
-  CompletableFuture<Void> closeRegion(String regionname, String serverName);
+  CompletableFuture<Boolean> isBalancerOn();
 
   /**
-   * Close a region.  For expert-admins  Runs close on the regionserver.  The 
master will not be
+   * 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>
-   */
-  CompletableFuture<Void> closeRegion(byte[] regionname, String serverName);
-
-  /**
-   * 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>
+   * @param regionName region name to close
+   * @param serverName The servername of the regionserver. If not present, 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>
    * @return true if the region was closed, false if not. The return value 
will be wrapped by a
-   * {@link CompletableFuture}.
-   */
-  CompletableFuture<Boolean> closeRegionWithEncodedRegionName(String 
encodedRegionName, String serverName);
-
-  /**
-   * Close a region.  For expert-admins  Runs close on the regionserver.  The 
master will not be
-   * informed of the close.
-   *
-   * @param sn
-   * @param hri
+   *         {@link CompletableFuture}.
    */
-  CompletableFuture<Void> closeRegion(ServerName sn, HRegionInfo hri);
+  CompletableFuture<Boolean> closeRegion(byte[] regionName, 
Optional<ServerName> serverName);
 
   /**
    * Get all the online regions on a region server.
@@ -422,60 +345,80 @@ public interface AsyncAdmin {
   CompletableFuture<Void> flushRegion(byte[] regionName);
 
   /**
-   * Compact a table. Asynchronous operation even if CompletableFuture.get().
+   * Compact a table. When the returned CompletableFuture is done, it only 
means the compact request
+   * was sent to HBase and may need some time to finish the compact operation.
    * @param tableName table to compact
    */
-  CompletableFuture<Void> compact(TableName tableName);
+  default CompletableFuture<Void> compact(TableName tableName) {
+    return compact(tableName, Optional.empty());
+  }
 
   /**
-   * Compact a column family within a table. Asynchronous operation even if 
CompletableFuture.get().
+   * Compact a column family within a table. When the returned 
CompletableFuture is done, it only
+   * means the compact request was sent to HBase and may need some time to 
finish the compact
+   * operation.
    * @param tableName table to compact
-   * @param columnFamily column family within a table
+   * @param columnFamily column family within a table. If not present, compact 
the table's all
+   *          column families.
    */
-  CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily);
+  CompletableFuture<Void> compact(TableName tableName, Optional<byte[]> 
columnFamily);
 
   /**
-   * Compact an individual region. Asynchronous operation even if 
CompletableFuture.get().
+   * Compact an individual region. When the returned CompletableFuture is 
done, it only means the
+   * compact request was sent to HBase and may need some time to finish the 
compact operation.
    * @param regionName region to compact
    */
-  CompletableFuture<Void> compactRegion(byte[] regionName);
+  default CompletableFuture<Void> compactRegion(byte[] regionName) {
+    return compactRegion(regionName, Optional.empty());
+  }
 
   /**
-   * Compact a column family within a region. Asynchronous operation even if
-   * CompletableFuture.get().
+   * Compact a column family within a region. When the returned 
CompletableFuture is done, it only
+   * means the compact request was sent to HBase and may need some time to 
finish the compact
+   * operation.
    * @param regionName region to compact
-   * @param columnFamily column family within a region
+   * @param columnFamily column family within a region. If not present, 
compact the region's all
+   *          column families.
    */
-  CompletableFuture<Void> compactRegion(byte[] regionName, byte[] 
columnFamily);
+  CompletableFuture<Void> compactRegion(byte[] regionName, Optional<byte[]> 
columnFamily);
 
   /**
-   * Major compact a table. Asynchronous operation even if 
CompletableFuture.get().
+   * Major compact a table. When the returned CompletableFuture is done, it 
only means the compact
+   * request was sent to HBase and may need some time to finish the compact 
operation.
    * @param tableName table to major compact
    */
-  CompletableFuture<Void> majorCompact(TableName tableName);
+  default CompletableFuture<Void> majorCompact(TableName tableName) {
+    return majorCompact(tableName, Optional.empty());
+  }
 
   /**
-   * Major compact a column family within a table. Asynchronous operation even 
if
-   * CompletableFuture.get().
+   * Major compact a column family within a table. When the returned 
CompletableFuture is done, it
+   * only means the compact request was sent to HBase and may need some time 
to finish the compact
+   * operation.
    * @param tableName table to major compact
-   * @param columnFamily column family within a table
+   * @param columnFamily column family within a table. If not present, major 
compact the table's all
+   *          column families.
    */
-  CompletableFuture<Void> majorCompact(TableName tableName, byte[] 
columnFamily);
+  CompletableFuture<Void> majorCompact(TableName tableName, Optional<byte[]> 
columnFamily);
 
   /**
-   * Major compact a table or an individual region. Asynchronous operation 
even if
-   * CompletableFuture.get().
+   * Major compact a region. When the returned CompletableFuture is done, it 
only means the compact
+   * request was sent to HBase and may need some time to finish the compact 
operation.
    * @param regionName region to major compact
    */
-  CompletableFuture<Void> majorCompactRegion(byte[] regionName);
+  default CompletableFuture<Void> majorCompactRegion(byte[] regionName) {
+    return majorCompactRegion(regionName, Optional.empty());
+  }
 
   /**
-   * Major compact a column family within region. Asynchronous operation even 
if
-   * CompletableFuture.get().
-   * @param regionName egion to major compact
-   * @param columnFamily column family within a region
+   * Major compact a column family within region. When the returned 
CompletableFuture is done, it
+   * only means the compact request was sent to HBase and may need some time 
to finish the compact
+   * operation.
+   * @param regionName region to major compact
+   * @param columnFamily column family within a region. If not present, major 
compact the region's
+   *          all column families.
    */
-  CompletableFuture<Void> majorCompactRegion(byte[] regionName, byte[] 
columnFamily);
+  CompletableFuture<Void> majorCompactRegion(byte[] regionName, 
Optional<byte[]> columnFamily);
 
   /**
    * Compact all regions on the region server.
@@ -496,51 +439,54 @@ public interface AsyncAdmin {
    * @param forcible true if do a compulsory merge, otherwise we will only 
merge two adjacent
    *          regions
    */
-  CompletableFuture<Void> mergeRegions(final byte[] nameOfRegionA, final 
byte[] nameOfRegionB,
-      final boolean forcible);
+  CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] 
nameOfRegionB,
+      boolean forcible);
 
   /**
    * Split a table. The method will execute split action for each region in 
table.
    * @param tableName table to split
    */
-  CompletableFuture<Void> split(final TableName tableName);
+  CompletableFuture<Void> split(TableName tableName);
 
   /**
    * Split an individual region.
    * @param regionName region to split
    */
-  CompletableFuture<Void> splitRegion(final byte[] regionName);
+  default CompletableFuture<Void> splitRegion(byte[] regionName) {
+    return splitRegion(regionName, Optional.empty());
+  }
 
   /**
    * Split a table.
    * @param tableName table to split
    * @param splitPoint the explicit position to split on
    */
-  CompletableFuture<Void> split(final TableName tableName, final byte[] 
splitPoint);
+  CompletableFuture<Void> split(TableName tableName, byte[] splitPoint);
 
   /**
    * Split an individual region.
    * @param regionName region to split
-   * @param splitPoint the explicit position to split on
+   * @param splitPoint the explicit position to split on. If not present, it 
will decide by region
+   *          server.
    */
-  CompletableFuture<Void> splitRegion(final byte[] regionName, final byte[] 
splitPoint);
+  CompletableFuture<Void> splitRegion(byte[] regionName, Optional<byte[]> 
splitPoint);
 
   /**
    * @param regionName Encoded or full name of region to assign.
    */
-  CompletableFuture<Void> assign(final byte[] regionName);
+  CompletableFuture<Void> assign(byte[] regionName);
 
   /**
    * 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.
+   * {@link #move(byte[], Optional)} if you want to control the region 
movement.
    * @param regionName Encoded or full name of 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
+   * @param forcible 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).
    */
-  CompletableFuture<Void> unassign(final byte[] regionName, final boolean 
force);
+  CompletableFuture<Void> unassign(byte[] regionName, boolean forcible);
 
   /**
    * Offline specified region from master's in-memory state. It will not 
attempt to reassign the
@@ -550,22 +496,22 @@ public interface AsyncAdmin {
    * experts or hbck.
    * @param regionName Encoded or full name of region to offline
    */
-  CompletableFuture<Void> offline(final byte[] regionName);
+  CompletableFuture<Void> offline(byte[] regionName);
 
   /**
    * Move the region <code>r</code> to <code>dest</code>.
    * @param regionName Encoded or full name of region to move.
-   * @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>
+   * @param destServerName The servername of the destination regionserver. If 
not present, 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>
    */
-  CompletableFuture<Void> move(final byte[] regionName, final byte[] 
destServerName);
+  CompletableFuture<Void> move(byte[] regionName, Optional<ServerName> 
destServerName);
 
   /**
    * Apply the new quota settings.
    * @param quota the quota settings
    */
-  CompletableFuture<Void> setQuota(final QuotaSettings quota);
+  CompletableFuture<Void> setQuota(QuotaSettings quota);
 
   /**
    * List the quotas based on the filter.
@@ -579,41 +525,41 @@ public interface AsyncAdmin {
    * @param peerId a short name that identifies the peer
    * @param peerConfig configuration for the replication slave cluster
    */
-  CompletableFuture<Void> addReplicationPeer(final String peerId,
-      final ReplicationPeerConfig peerConfig);
+  CompletableFuture<Void> addReplicationPeer(String peerId,
+      ReplicationPeerConfig peerConfig);
 
   /**
    * Remove a peer and stop the replication
    * @param peerId a short name that identifies the peer
    */
-  CompletableFuture<Void> removeReplicationPeer(final String peerId);
+  CompletableFuture<Void> removeReplicationPeer(String peerId);
 
   /**
    * Restart the replication stream to the specified peer
    * @param peerId a short name that identifies the peer
    */
-  CompletableFuture<Void> enableReplicationPeer(final String peerId);
+  CompletableFuture<Void> enableReplicationPeer(String peerId);
 
   /**
    * Stop the replication stream to the specified peer
    * @param peerId a short name that identifies the peer
    */
-  CompletableFuture<Void> disableReplicationPeer(final String peerId);
+  CompletableFuture<Void> disableReplicationPeer(String peerId);
 
   /**
    * Returns the configured ReplicationPeerConfig for the specified peer
    * @param peerId a short name that identifies the peer
    * @return ReplicationPeerConfig for the peer wrapped by a {@link 
CompletableFuture}.
    */
-  CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(final 
String peerId);
+  CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String 
peerId);
 
   /**
    * Update the peerConfig for the specified peer
    * @param peerId a short name that identifies the peer
    * @param peerConfig new config for the peer
    */
-  CompletableFuture<Void> updateReplicationPeerConfig(final String peerId,
-      final ReplicationPeerConfig peerConfig);
+  CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
+      ReplicationPeerConfig peerConfig);
 
   /**
    * Append the replicable table-cf config of the specified peer
@@ -636,15 +582,9 @@ public interface AsyncAdmin {
    * @return a list of replication peers description. The return value will be 
wrapped by a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers();
-
-  /**
-   * Return a list of replication peers.
-   * @param regex The regular expression to match peer id
-   * @return a list of replication peers description. The return value will be 
wrapped by a
-   *         {@link CompletableFuture}.
-   */
-  CompletableFuture<List<ReplicationPeerDescription>> 
listReplicationPeers(String regex);
+  default CompletableFuture<List<ReplicationPeerDescription>> 
listReplicationPeers() {
+    return listReplicationPeers(Optional.empty());
+  }
 
   /**
    * Return a list of replication peers.
@@ -652,7 +592,8 @@ public interface AsyncAdmin {
    * @return a list of replication peers description. The return value will be 
wrapped by a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<List<ReplicationPeerDescription>> 
listReplicationPeers(Pattern pattern);
+  CompletableFuture<List<ReplicationPeerDescription>>
+      listReplicationPeers(Optional<Pattern> pattern);
 
   /**
    * Find all table and column families that are replicated from this cluster
@@ -686,7 +627,7 @@ public interface AsyncAdmin {
    * @param tableName name of the table to snapshot
    * @param type type of snapshot to take
    */
-  CompletableFuture<Void> snapshot(final String snapshotName, final TableName 
tableName,
+  CompletableFuture<Void> snapshot(String snapshotName, TableName tableName,
       SnapshotType type);
 
   /**
@@ -718,7 +659,7 @@ public interface AsyncAdmin {
    * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the 
snapshot is still
    *         running
    */
-  CompletableFuture<Boolean> isSnapshotFinished(final SnapshotDescription 
snapshot);
+  CompletableFuture<Boolean> isSnapshotFinished(SnapshotDescription snapshot);
 
   /**
    * Restore the specified snapshot on the original table. (The table must be 
disabled) If the
@@ -747,7 +688,7 @@ public interface AsyncAdmin {
    * @param snapshotName name of the snapshot to be cloned
    * @param tableName name of the table where the snapshot will be restored
    */
-  CompletableFuture<Void> cloneSnapshot(final String snapshotName, final 
TableName tableName);
+  CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName 
tableName);
 
   /**
    * List completed snapshots.
@@ -757,13 +698,6 @@ public interface AsyncAdmin {
   CompletableFuture<List<SnapshotDescription>> listSnapshots();
 
   /**
-   * List all the completed snapshots matching the given regular expression.
-   * @param regex The regular expression to match against
-   * @return - returns a List of SnapshotDescription wrapped by a {@link 
CompletableFuture}
-   */
-  CompletableFuture<List<SnapshotDescription>> listSnapshots(String 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 wrapped by a {@link 
CompletableFuture}
@@ -773,17 +707,6 @@ public interface AsyncAdmin {
   /**
    * 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 wrapped by a
-   *         {@link CompletableFuture}
-   */
-  CompletableFuture<List<SnapshotDescription>> listTableSnapshots(String 
tableNameRegex,
-      String 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 wrapped by a
@@ -800,12 +723,6 @@ public interface AsyncAdmin {
 
   /**
    * Delete existing snapshots whose names match the pattern passed.
-   * @param regex The regular expression to match against
-   */
-  CompletableFuture<Void> deleteSnapshots(String regex);
-
-  /**
-   * Delete existing snapshots whose names match the pattern passed.
    * @param pattern pattern for names of the snapshot to match
    */
   CompletableFuture<Void> deleteSnapshots(Pattern pattern);
@@ -813,14 +730,6 @@ public interface AsyncAdmin {
   /**
    * 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
-   */
-  CompletableFuture<Void> deleteTableSnapshots(String tableNameRegex, String 
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
    */
@@ -861,7 +770,7 @@ public interface AsyncAdmin {
    * @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.
-   *         The value is vrapped by {@link CompletableFuture}
+   *         The value is wrapped by {@link CompletableFuture}
    */
   CompletableFuture<Boolean> isProcedureFinished(String signature, String 
instance,
       Map<String, String> props);
@@ -879,5 +788,5 @@ public interface AsyncAdmin {
    * List procedures
    * @return procedure list wrapped by {@link CompletableFuture}
    */
-  CompletableFuture<ProcedureInfo[]> listProcedures();
+  CompletableFuture<List<ProcedureInfo>> listProcedures();
 }

Reply via email to