HBASE-17498 Implement listTables and listTableNames methods

Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
Guanghao Zhang 2017-01-19 15:13:45 +08:00 committed by Michael Stack
parent 2285c57a35
commit 79018056f5
3 changed files with 171 additions and 10 deletions

View File

@ -17,8 +17,11 @@
*/ */
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.concurrent.CompletableFuture; 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.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -30,6 +33,54 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Unstable @InterfaceStability.Unstable
public interface AsyncAdmin { 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. * @param tableName Table to check.
@ -43,7 +94,7 @@ public interface AsyncAdmin {
* @param on * @param on
* @return Previous balancer value wrapped by a {@link CompletableFuture}. * @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 * 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 * @return True if balancer ran, false otherwise. The return value will be wrapped by a
* {@link CompletableFuture}. * {@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 * 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 * @return True if balancer ran, false otherwise. The return value will be wrapped by a
* {@link CompletableFuture}. * {@link CompletableFuture}.
*/ */
CompletableFuture<Boolean> balancer(boolean force) throws IOException; CompletableFuture<Boolean> balancer(boolean force);
/** /**
* Query the current state of the balancer. * Query the current state of the balancer.
* @return true if the balancer is enabled, false otherwise. * @return true if the balancer is enabled, false otherwise.
* The return value will be wrapped by a {@link CompletableFuture}. * The return value will be wrapped by a {@link CompletableFuture}.
*/ */
CompletableFuture<Boolean> isBalancerEnabled() throws IOException; CompletableFuture<Boolean> isBalancerEnabled();
} }

View File

@ -20,7 +20,10 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException; import java.io.IOException;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit; 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.AsyncMetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback; 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.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; 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.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.IsBalancerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
@ -107,7 +115,51 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
} }
@Override @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 return this
.<Boolean> newCaller() .<Boolean> newCaller()
.action( .action(
@ -119,12 +171,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
} }
@Override @Override
public CompletableFuture<Boolean> balancer() throws IOException { public CompletableFuture<Boolean> balancer() {
return balancer(false); return balancer(false);
} }
@Override @Override
public CompletableFuture<Boolean> balancer(boolean force) throws IOException { public CompletableFuture<Boolean> balancer(boolean force) {
return this return this
.<Boolean> newCaller() .<Boolean> newCaller()
.action( .action(
@ -134,7 +186,7 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
} }
@Override @Override
public CompletableFuture<Boolean> isBalancerEnabled() throws IOException { public CompletableFuture<Boolean> isBalancerEnabled() {
return this return this
.<Boolean> newCaller() .<Boolean> newCaller()
.action( .action(
@ -148,4 +200,4 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
public CompletableFuture<Boolean> tableExists(TableName tableName) { public CompletableFuture<Boolean> tableExists(TableName tableName) {
return AsyncMetaTableAccessor.tableExists(connection, tableName); return AsyncMetaTableAccessor.tableExists(connection, tableName);
} }
} }

View File

@ -18,15 +18,21 @@
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals; 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.io.IOUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility; 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.TableName;
import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -66,6 +72,58 @@ public class TestAsyncAdmin {
this.admin = ASYNC_CONN.getAdmin(); this.admin = ASYNC_CONN.getAdmin();
} }
@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 @Test
public void testTableExist() throws Exception { public void testTableExist() throws Exception {
final TableName table = TableName.valueOf("testTableExist"); final TableName table = TableName.valueOf("testTableExist");