[2/3] hbase git commit: HBASE-18234 Revisit the async admin api

2017-06-26 Thread zghao
http://git-wip-us.apache.org/repos/asf/hbase/blob/28993833/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index c972b4c..8505241 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -40,10 +40,13 @@ import com.google.common.annotations.VisibleForTesting;
 
 import io.netty.util.Timeout;
 import io.netty.util.TimerTask;
+
 import java.util.stream.Stream;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.directory.api.util.OptionalComponentsMonitor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -190,7 +193,6 @@ import org.apache.hadoop.hbase.util.Pair;
  * The implementation of AsyncAdmin.
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public class AsyncHBaseAdmin implements AsyncAdmin {
   public static final String FLUSH_TABLE_PROCEDURE_SIGNATURE = 
"flush-table-proc";
 
@@ -278,7 +280,6 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
 return future;
   }
 
-  //TODO abstract call and adminCall into a single method.
   private  CompletableFuture 
adminCall(HBaseRpcController controller,
   AdminService.Interface stub, PREQ preq, AdminRpcCall 
rpcCall,
   Converter respConverter) {
@@ -318,25 +319,26 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
 CompletableFuture operate(TableName table);
   }
 
-  private CompletableFuture batchTableOperations(Pattern 
pattern,
+  private CompletableFuture 
batchTableOperations(Pattern pattern,
   TableOperator operator, String operationType) {
-CompletableFuture future = new CompletableFuture<>();
+CompletableFuture future = new 
CompletableFuture<>();
 List failed = new LinkedList<>();
-listTables(pattern, false).whenComplete(
+listTables(Optional.ofNullable(pattern), false).whenComplete(
   (tables, error) -> {
 if (error != null) {
   future.completeExceptionally(error);
   return;
 }
-CompletableFuture[] futures = Arrays.stream(tables)
-.map((table) -> 
operator.operate(table.getTableName()).whenComplete((v, ex) -> {
-  if (ex != null) {
-LOG.info("Failed to " + operationType + " table " + 
table.getTableName(), ex);
-failed.add(table);
-  }
-})). toArray(size -> new 
CompletableFuture[size]);
+CompletableFuture[] futures =
+tables.stream()
+.map((table) -> 
operator.operate(table.getTableName()).whenComplete((v, ex) -> {
+  if (ex != null) {
+LOG.info("Failed to " + operationType + " table " + 
table.getTableName(), ex);
+failed.add(table);
+  }
+})). toArray(size -> new 
CompletableFuture[size]);
 CompletableFuture.allOf(futures).thenAccept((v) -> {
-  future.complete(failed.toArray(new TableDescriptor[failed.size()]));
+  future.complete(failed);
 });
   });
 return future;
@@ -353,47 +355,28 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture listTables() {
-return listTables((Pattern) null, false);
-  }
-
-  @Override
-  public CompletableFuture listTables(String regex, boolean 
includeSysTables) {
-return listTables(Pattern.compile(regex), false);
-  }
-
-  @Override
-  public CompletableFuture listTables(Pattern pattern, 
boolean includeSysTables) {
-return this
-.newMasterCaller()
-.action(
-  (controller, stub) -> this
-  . call(
-controller, stub, 
RequestConverter.buildGetTableDescriptorsRequest(pattern,
-  includeSysTables), (s, c, req, done) -> 
s.getTableDescriptors(c, req, done), (
-resp) -> 
ProtobufUtil.getTableDescriptorArray(resp))).call();
-  }
-
-  @Override
-  public CompletableFuture listTableNames() {
-return listTableNames((Pattern) null, false);
-  }
-
-  @Override
-  public CompletableFuture listTableNames(String regex, boolean 
includeSysTables) {
-return listTableNames(Pattern.compile(regex), false);
+  public CompletableFuture listTables(Optional 

[2/3] hbase git commit: HBASE-18234 Revisit the async admin api

2017-06-26 Thread zghao
http://git-wip-us.apache.org/repos/asf/hbase/blob/2d781aa1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index c972b4c..8505241 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -40,10 +40,13 @@ import com.google.common.annotations.VisibleForTesting;
 
 import io.netty.util.Timeout;
 import io.netty.util.TimerTask;
+
 import java.util.stream.Stream;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.directory.api.util.OptionalComponentsMonitor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -190,7 +193,6 @@ import org.apache.hadoop.hbase.util.Pair;
  * The implementation of AsyncAdmin.
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public class AsyncHBaseAdmin implements AsyncAdmin {
   public static final String FLUSH_TABLE_PROCEDURE_SIGNATURE = 
"flush-table-proc";
 
@@ -278,7 +280,6 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
 return future;
   }
 
-  //TODO abstract call and adminCall into a single method.
   private  CompletableFuture 
adminCall(HBaseRpcController controller,
   AdminService.Interface stub, PREQ preq, AdminRpcCall 
rpcCall,
   Converter respConverter) {
@@ -318,25 +319,26 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
 CompletableFuture operate(TableName table);
   }
 
-  private CompletableFuture batchTableOperations(Pattern 
pattern,
+  private CompletableFuture 
batchTableOperations(Pattern pattern,
   TableOperator operator, String operationType) {
-CompletableFuture future = new CompletableFuture<>();
+CompletableFuture future = new 
CompletableFuture<>();
 List failed = new LinkedList<>();
-listTables(pattern, false).whenComplete(
+listTables(Optional.ofNullable(pattern), false).whenComplete(
   (tables, error) -> {
 if (error != null) {
   future.completeExceptionally(error);
   return;
 }
-CompletableFuture[] futures = Arrays.stream(tables)
-.map((table) -> 
operator.operate(table.getTableName()).whenComplete((v, ex) -> {
-  if (ex != null) {
-LOG.info("Failed to " + operationType + " table " + 
table.getTableName(), ex);
-failed.add(table);
-  }
-})). toArray(size -> new 
CompletableFuture[size]);
+CompletableFuture[] futures =
+tables.stream()
+.map((table) -> 
operator.operate(table.getTableName()).whenComplete((v, ex) -> {
+  if (ex != null) {
+LOG.info("Failed to " + operationType + " table " + 
table.getTableName(), ex);
+failed.add(table);
+  }
+})). toArray(size -> new 
CompletableFuture[size]);
 CompletableFuture.allOf(futures).thenAccept((v) -> {
-  future.complete(failed.toArray(new TableDescriptor[failed.size()]));
+  future.complete(failed);
 });
   });
 return future;
@@ -353,47 +355,28 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture listTables() {
-return listTables((Pattern) null, false);
-  }
-
-  @Override
-  public CompletableFuture listTables(String regex, boolean 
includeSysTables) {
-return listTables(Pattern.compile(regex), false);
-  }
-
-  @Override
-  public CompletableFuture listTables(Pattern pattern, 
boolean includeSysTables) {
-return this
-.newMasterCaller()
-.action(
-  (controller, stub) -> this
-  . call(
-controller, stub, 
RequestConverter.buildGetTableDescriptorsRequest(pattern,
-  includeSysTables), (s, c, req, done) -> 
s.getTableDescriptors(c, req, done), (
-resp) -> 
ProtobufUtil.getTableDescriptorArray(resp))).call();
-  }
-
-  @Override
-  public CompletableFuture listTableNames() {
-return listTableNames((Pattern) null, false);
-  }
-
-  @Override
-  public CompletableFuture listTableNames(String regex, boolean 
includeSysTables) {
-return listTableNames(Pattern.compile(regex), false);
+  public CompletableFuture listTables(Optional