You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2017/07/04 02:14:08 UTC

[5/5] hbase git commit: HBASE-18283 Provide a construct method which accept a thread pool for AsyncAdmin

HBASE-18283 Provide a construct method which accept a thread pool for AsyncAdmin


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

Branch: refs/heads/master
Commit: 14f0423b58256d872a1ddde1712a1fc60b4c5671
Parents: 8318a09
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Jul 4 09:51:41 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Tue Jul 4 09:51:41 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |   67 +-
 .../hadoop/hbase/client/AsyncConnection.java    |   14 +-
 .../hbase/client/AsyncConnectionImpl.java       |    7 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    | 2085 +---------------
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 2278 ++++++++++++++++++
 .../hbase/shaded/protobuf/RequestConverter.java |   21 +-
 .../hadoop/hbase/client/TestAsyncAdminBase.java |   80 +-
 .../hbase/client/TestAsyncBalancerAdminApi.java |    3 +
 .../client/TestAsyncNamespaceAdminApi.java      |    3 +
 .../client/TestAsyncProcedureAdminApi.java      |    4 +-
 .../hbase/client/TestAsyncQuotaAdminApi.java    |   37 +-
 .../hbase/client/TestAsyncRegionAdminApi.java   |  737 +++---
 .../client/TestAsyncReplicationAdminApi.java    |   30 +-
 .../hbase/client/TestAsyncSnapshotAdminApi.java |   24 +-
 .../hbase/client/TestAsyncTableAdminApi.java    |  818 +++----
 15 files changed, 3288 insertions(+), 2920 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/14f0423b/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 3b022f4..ff35d46 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
@@ -47,11 +47,6 @@ import org.apache.hadoop.hbase.util.Pair;
 public interface AsyncAdmin {
 
   /**
-   * @return Async Connection used by this object.
-   */
-  AsyncConnectionImpl getConnection();
-
-  /**
    * @param tableName Table to check.
    * @return True if table exists already. The return value will be wrapped by a
    *         {@link CompletableFuture}.
@@ -105,7 +100,9 @@ public interface AsyncAdmin {
    * Creates a new table.
    * @param desc table descriptor for table
    */
-  CompletableFuture<Void> createTable(TableDescriptor desc);
+  default CompletableFuture<Void> createTable(TableDescriptor desc) {
+    return createTable(desc, Optional.empty());
+  }
 
   /**
    * Creates a new table with the specified number of regions. The start key specified will become
@@ -128,7 +125,7 @@ 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(TableDescriptor desc, byte[][] splitKeys);
+  CompletableFuture<Void> createTable(TableDescriptor desc, Optional<byte[][]> splitKeys);
 
   /**
    * Deletes a table.
@@ -188,6 +185,13 @@ public interface AsyncAdmin {
 
   /**
    * @param tableName name of table to check
+   * @return true if table is on-line. The return value will be wrapped by a
+   *         {@link CompletableFuture}.
+   */
+  CompletableFuture<Boolean> isTableEnabled(TableName tableName);
+
+  /**
+   * @param tableName name of table to check
    * @return true if table is off-line. The return value will be wrapped by a
    *         {@link CompletableFuture}.
    */
@@ -198,7 +202,9 @@ public interface AsyncAdmin {
    * @return true if all regions of the table are available. The return value will be wrapped by a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> isTableAvailable(TableName tableName);
+  default CompletableFuture<Boolean> isTableAvailable(TableName tableName) {
+    return isTableAvailable(tableName, null);
+  }
 
   /**
    * Use this api to check if the table has been created with the specified number of splitkeys
@@ -275,13 +281,6 @@ public interface AsyncAdmin {
   CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors();
 
   /**
-   * @param tableName name of table to check
-   * @return true if table is on-line. The return value will be wrapped by a
-   *         {@link CompletableFuture}.
-   */
-  CompletableFuture<Boolean> isTableEnabled(TableName tableName);
-
-  /**
    * Turn the load balancer on or off.
    * @param on
    * @return Previous balancer value wrapped by a {@link CompletableFuture}.
@@ -330,7 +329,7 @@ public interface AsyncAdmin {
   /**
    * Get all the online regions on a region server.
    */
-  CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName sn);
+  CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName);
 
   /**
    * Flush a table.
@@ -422,15 +421,15 @@ public interface AsyncAdmin {
 
   /**
    * Compact all regions on the region server.
-   * @param sn the region server name
+   * @param serverName the region server name
    */
-  CompletableFuture<Void> compactRegionServer(ServerName sn);
+  CompletableFuture<Void> compactRegionServer(ServerName serverName);
 
   /**
    * Compact all regions on the region server.
-   * @param sn the region server name
+   * @param serverName the region server name
    */
-  CompletableFuture<Void> majorCompactRegionServer(ServerName sn);
+  CompletableFuture<Void> majorCompactRegionServer(ServerName serverName);
 
   /**
    * Merge two regions.
@@ -563,18 +562,18 @@ public interface AsyncAdmin {
 
   /**
    * Append the replicable table-cf config of the specified peer
-   * @param id a short that identifies the cluster
+   * @param peerId a short that identifies the cluster
    * @param tableCfs A map from tableName to column family names
    */
-  CompletableFuture<Void> appendReplicationPeerTableCFs(String id,
+  CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
       Map<TableName, ? extends Collection<String>> tableCfs);
 
   /**
    * Remove some table-cfs from config of the specified peer
-   * @param id a short name that identifies the cluster
+   * @param peerId a short name that identifies the cluster
    * @param tableCfs A map from tableName to column family names
    */
-  CompletableFuture<Void> removeReplicationPeerTableCFs(String id,
+  CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId,
       Map<TableName, ? extends Collection<String>> tableCfs);
 
   /**
@@ -613,7 +612,9 @@ public interface AsyncAdmin {
    * @param snapshotName name of the snapshot to be created
    * @param tableName name of the table for which snapshot is created
    */
-  CompletableFuture<Void> snapshot(String snapshotName, TableName tableName);
+  default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName) {
+    return snapshot(snapshotName, tableName, SnapshotType.FLUSH);
+  }
 
   /**
    * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
@@ -627,8 +628,10 @@ public interface AsyncAdmin {
    * @param tableName name of the table to snapshot
    * @param type type of snapshot to take
    */
-  CompletableFuture<Void> snapshot(String snapshotName, TableName tableName,
-      SnapshotType type);
+  default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName,
+      SnapshotType type) {
+    return snapshot(new SnapshotDescription(snapshotName, tableName, type));
+  }
 
   /**
    * Take a snapshot and wait for the server to complete that snapshot asynchronously. Only a single
@@ -695,14 +698,16 @@ public interface AsyncAdmin {
    * @return a list of snapshot descriptors for completed snapshots wrapped by a
    *         {@link CompletableFuture}
    */
-  CompletableFuture<List<SnapshotDescription>> listSnapshots();
+  default CompletableFuture<List<SnapshotDescription>> listSnapshots() {
+    return listSnapshots(Optional.empty());
+  }
 
   /**
    * 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}
    */
-  CompletableFuture<List<SnapshotDescription>> listSnapshots(Pattern pattern);
+  CompletableFuture<List<SnapshotDescription>> listSnapshots(Optional<Pattern> pattern);
 
   /**
    * List all the completed snapshots matching the given table name regular expression and snapshot
@@ -725,7 +730,9 @@ public interface AsyncAdmin {
    * Delete existing snapshots whose names match the pattern passed.
    * @param pattern pattern for names of the snapshot to match
    */
-  CompletableFuture<Void> deleteSnapshots(Pattern pattern);
+  default CompletableFuture<Void> deleteSnapshots(Pattern pattern) {
+    return deleteTableSnapshots(null, pattern);
+  }
 
   /**
    * Delete all existing snapshots matching the given table name regular expression and snapshot

http://git-wip-us.apache.org/repos/asf/hbase/blob/14f0423b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
index 65005fa..22ed064 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
@@ -96,11 +96,17 @@ public interface AsyncConnection extends Closeable {
   AsyncTableBuilder<AsyncTable> getTableBuilder(TableName tableName, ExecutorService pool);
 
   /**
-   * Retrieve an AsyncAdmin implementation to administer an HBase cluster. The returned AsyncAdmin
-   * is not guaranteed to be thread-safe. A new instance should be created for each using thread.
-   * This is a lightweight operation. Pooling or caching of the returned AsyncAdmin is not
-   * recommended.
+   * Retrieve an AsyncAdmin implementation to administer an HBase cluster. The returned
+   * {@code CompletableFuture} will be finished directly in the rpc framework's callback thread, so
+   * typically you should not do any time consuming work inside these methods.
    * @return an AsyncAdmin instance for cluster administration
    */
   AsyncAdmin getAdmin();
+
+  /**
+   * Retrieve an AsyncAdmin implementation to administer an HBase cluster.
+   * @param pool the thread pool to use for executing callback
+   * @return an AsyncAdmin instance for cluster administration
+   */
+  AsyncAdmin getAdmin(ExecutorService pool);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/14f0423b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 776498a..c170bce 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -279,6 +279,11 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   @Override
   public AsyncAdmin getAdmin() {
-    return new AsyncHBaseAdmin(this);
+    return new RawAsyncHBaseAdmin(this);
+  }
+
+  @Override
+  public AsyncAdmin getAdmin(ExecutorService pool) {
+    return new AsyncHBaseAdmin(new RawAsyncHBaseAdmin(this), pool);
   }
 }
\ No newline at end of file