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 2017/01/20 22:12:26 UTC

hbase git commit: HBASE-17498 Implement listTables and listTableNames methods

Repository: hbase
Updated Branches:
  refs/heads/master 2285c57a3 -> 79018056f


HBASE-17498 Implement listTables and listTableNames methods

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/master
Commit: 79018056f542cde5850b1d1fc2fe248f0007fd66
Parents: 2285c57
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Jan 19 15:13:45 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Jan 20 14:12:13 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/AsyncAdmin.java  | 61 +++++++++++++++++--
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    | 62 ++++++++++++++++++--
 .../hadoop/hbase/client/TestAsyncAdmin.java     | 58 ++++++++++++++++++
 3 files changed, 171 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/79018056/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 1ce23b9..20a6070 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
@@ -17,8 +17,11 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
+import java.util.regex.Pattern;
+ 
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -30,6 +33,54 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public interface AsyncAdmin {
+  /**
+   * List all the userspace tables.
+   * @return - returns an array of HTableDescriptors wrapped by a {@link CompletableFuture}.
+   * @see #listTables(Pattern, boolean)
+   */
+  CompletableFuture<HTableDescriptor[]> 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 HTableDescriptors wrapped by a {@link CompletableFuture}.
+   * @see #listTables(Pattern, boolean)
+   */
+  CompletableFuture<HTableDescriptor[]> listTables(String regex, boolean includeSysTables);
+
+  /**
+   * 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 HTableDescriptors wrapped by a {@link CompletableFuture}.
+   */
+  CompletableFuture<HTableDescriptor[]> 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)
+   */
+  CompletableFuture<TableName[]> listTableNames();
+
+  /**
+   * 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)
+   */
+  CompletableFuture<TableName[]> listTableNames(final String regex, final boolean includeSysTables);
+
+  /**
+   * 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}.
+   */
+  CompletableFuture<TableName[]> listTableNames(final Pattern pattern,
+      final boolean includeSysTables);
 
   /**
    * @param tableName Table to check.
@@ -43,7 +94,7 @@ public interface AsyncAdmin {
    * @param on
    * @return Previous balancer value wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> setBalancerRunning(final boolean on) throws IOException;
+  CompletableFuture<Boolean> setBalancerRunning(final boolean on);
 
   /**
    * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
@@ -51,7 +102,7 @@ public interface AsyncAdmin {
    * @return True if balancer ran, false otherwise. The return value will be wrapped by a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> balancer() throws IOException;
+  CompletableFuture<Boolean> balancer();
 
   /**
    * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
@@ -61,12 +112,12 @@ public interface AsyncAdmin {
    * @return True if balancer ran, false otherwise. The return value will be wrapped by a
    *         {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> balancer(boolean force) throws IOException;
+  CompletableFuture<Boolean> balancer(boolean force);
 
   /**
    * 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() throws IOException;
+  CompletableFuture<Boolean> isBalancerEnabled();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/79018056/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 9398972..8682116 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
@@ -20,7 +20,10 @@ package org.apache.hadoop.hbase.client;
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
 
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -28,9 +31,14 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
@@ -107,7 +115,51 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> setBalancerRunning(final boolean on) throws IOException {
+  public CompletableFuture<HTableDescriptor[]> listTables() {
+    return listTables((Pattern)null, false);
+  }
+
+  @Override
+  public CompletableFuture<HTableDescriptor[]> listTables(String regex, boolean includeSysTables) {
+    return listTables(Pattern.compile(regex), false);
+  }
+
+  @Override
+  public CompletableFuture<HTableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables) {
+    return this
+        .<HTableDescriptor[]> newCaller()
+        .action(
+          (controller, stub) -> this
+              .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, HTableDescriptor[]> call(
+                controller, stub, RequestConverter.buildGetTableDescriptorsRequest(pattern,
+                  includeSysTables), (s, c, req, done) -> s.getTableDescriptors(c, req, done), (
+                    resp) -> ProtobufUtil.getHTableDescriptorArray(resp))).call();
+  }
+
+  @Override
+  public CompletableFuture<TableName[]> listTableNames() {
+    return listTableNames((Pattern)null, false);
+  }
+
+  @Override
+  public CompletableFuture<TableName[]> listTableNames(String regex, boolean includeSysTables) {
+    return listTableNames(Pattern.compile(regex), false);
+  }
+
+  @Override
+  public CompletableFuture<TableName[]> listTableNames(Pattern pattern, boolean includeSysTables) {
+    return this
+        .<TableName[]> newCaller()
+        .action(
+          (controller, stub) -> this
+              .<GetTableNamesRequest, GetTableNamesResponse, TableName[]> call(controller, stub,
+                RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables), (s, c, req,
+                    done) -> s.getTableNames(c, req, done), (resp) -> ProtobufUtil
+                    .getTableNameArray(resp.getTableNamesList()))).call();
+  }
+
+  @Override
+  public CompletableFuture<Boolean> setBalancerRunning(final boolean on) {
     return this
         .<Boolean> newCaller()
         .action(
@@ -119,12 +171,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> balancer() throws IOException {
+  public CompletableFuture<Boolean> balancer() {
     return balancer(false);
   }
 
   @Override
-  public CompletableFuture<Boolean> balancer(boolean force) throws IOException {
+  public CompletableFuture<Boolean> balancer(boolean force) {
     return this
         .<Boolean> newCaller()
         .action(
@@ -134,7 +186,7 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> isBalancerEnabled() throws IOException {
+  public CompletableFuture<Boolean> isBalancerEnabled() {
     return this
         .<Boolean> newCaller()
         .action(
@@ -148,4 +200,4 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Boolean> tableExists(TableName tableName) {
     return AsyncMetaTableAccessor.tableExists(connection, tableName);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/79018056/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdmin.java
index 70b4cf2..ccd6873 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdmin.java
@@ -18,15 +18,21 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.regex.Pattern;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -67,6 +73,58 @@ public class TestAsyncAdmin {
   }
 
   @Test
+  public void testListTables() throws Exception {
+    TableName t1 = TableName.valueOf("testListTables1");
+    TableName t2 = TableName.valueOf("testListTables2");
+    TableName t3 = TableName.valueOf("testListTables3");
+    TableName[] tables = new TableName[] { t1, t2, t3 };
+    for (int i = 0; i < tables.length; i++) {
+      TEST_UTIL.createTable(tables[i], FAMILY);
+    }
+
+    HTableDescriptor[] tableDescs = admin.listTables().get();
+    int size = tableDescs.length;
+    assertTrue(size >= tables.length);
+    for (int i = 0; i < tables.length && i < size; i++) {
+      boolean found = false;
+      for (int j = 0; j < tableDescs.length; j++) {
+        if (tableDescs[j].getTableName().equals(tables[i])) {
+          found = true;
+          break;
+        }
+      }
+      assertTrue("Not found: " + tables[i], found);
+    }
+
+    TableName[] tableNames = admin.listTableNames().get();
+    size = tableNames.length;
+    assertTrue(size >= tables.length);
+    for (int i = 0; i < tables.length && i < size; i++) {
+      boolean found = false;
+      for (int j = 0; j < tableNames.length; j++) {
+        if (tableNames[j].equals(tables[i])) {
+          found = true;
+          break;
+        }
+      }
+      assertTrue("Not found: " + tables[i], found);
+    }
+
+    for (int i = 0; i < tables.length; i++) {
+      TEST_UTIL.deleteTable(tables[i]);
+    }
+    tableDescs = admin.listTables().get();
+    assertEquals(0, tableDescs.length);
+    tableNames = admin.listTableNames().get();
+    assertEquals(0, tableNames.length);
+
+    tableDescs = admin.listTables((Pattern) null, true).get();
+    assertTrue("Not found system tables", tableDescs.length > 0);
+    tableNames = admin.listTableNames((Pattern) null, true).get();
+    assertTrue("Not found system tables", tableNames.length > 0);
+  }
+
+  @Test
   public void testTableExist() throws Exception {
     final TableName table = TableName.valueOf("testTableExist");
     boolean exist;