Skip to content

Commit 0f29e3b

Browse files
committed
HBASE-27444 Add tool commands list_enabled_tables and list_disabled_tables
1 parent 71826de commit 0f29e3b

File tree

19 files changed

+502
-1
lines changed

19 files changed

+502
-1
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -141,6 +141,13 @@ public interface Admin extends Abortable, Closeable {
141141
*/
142142
List<TableDescriptor> listTableDescriptors() throws IOException;
143143

144+
/**
145+
* List all enabled or disabled tables
146+
* @param isEnabled is true means return enabled tables, false means return disabled tables
147+
* @return a list of enabled or disabled tables
148+
*/
149+
List<TableDescriptor> listTableDescriptorsByState(boolean isEnabled) throws IOException;
150+
144151
/**
145152
* List all the userspace tables that match the given pattern.
146153
* @param pattern The compiled regular expression to match against
@@ -264,6 +271,14 @@ default TableName[] listTableNames(Pattern pattern) throws IOException {
264271
@Deprecated
265272
TableName[] listTableNames(String regex, boolean includeSysTables) throws IOException;
266273

274+
/**
275+
* List all enabled or disabled table names
276+
* @param isEnabled is true means return enabled table names, false means return disabled table
277+
* names
278+
* @return a list of enabled or disabled table names
279+
*/
280+
List<TableName> listTableNamesByState(boolean isEnabled) throws IOException;
281+
267282
/**
268283
* Get a table descriptor.
269284
* @param tableName as a {@link TableName}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -106,6 +106,14 @@ CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern,
106106
*/
107107
CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name);
108108

109+
/**
110+
* List all enabled or disabled table descriptors
111+
* @param isEnabled is true means return enabled table descriptors, false means return disabled
112+
* table descriptors
113+
* @return a list of table names wrapped by a {@link CompletableFuture}.
114+
*/
115+
CompletableFuture<List<TableDescriptor>> listTableDescriptorsByState(boolean isEnabled);
116+
109117
/**
110118
* List all of the names of userspace tables.
111119
* @return a list of table names wrapped by a {@link CompletableFuture}.
@@ -137,6 +145,14 @@ default CompletableFuture<List<TableName>> listTableNames() {
137145
*/
138146
CompletableFuture<List<TableName>> listTableNamesByNamespace(String name);
139147

148+
/**
149+
* List all enabled or disabled table names
150+
* @param isEnabled is true means return enabled table names, false means return disabled table
151+
* names
152+
* @return a list of table names wrapped by a {@link CompletableFuture}.
153+
*/
154+
CompletableFuture<List<TableName>> listTableNamesByState(boolean isEnabled);
155+
140156
/**
141157
* Method for getting the tableDescriptor
142158
* @param tableName as a {@link TableName}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -99,6 +99,11 @@ public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(
9999
return wrap(rawAdmin.listTableDescriptorsByNamespace(name));
100100
}
101101

102+
@Override
103+
public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByState(boolean isEnabled) {
104+
return wrap(rawAdmin.listTableDescriptorsByState(isEnabled));
105+
}
106+
102107
@Override
103108
public CompletableFuture<List<TableName>> listTableNames(boolean includeSysTables) {
104109
return wrap(rawAdmin.listTableNames(includeSysTables));
@@ -115,6 +120,11 @@ public CompletableFuture<List<TableName>> listTableNamesByNamespace(String name)
115120
return wrap(rawAdmin.listTableNamesByNamespace(name));
116121
}
117122

123+
@Override
124+
public CompletableFuture<List<TableName>> listTableNamesByState(boolean isEnabled) {
125+
return wrap(rawAdmin.listTableNamesByState(isEnabled));
126+
}
127+
118128
@Override
119129
public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName) {
120130
return wrap(rawAdmin.getDescriptor(tableName));

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1727,12 +1727,26 @@ public MasterProtos.GetTableDescriptorsResponse getTableDescriptors(RpcControlle
17271727
return stub.getTableDescriptors(controller, request);
17281728
}
17291729

1730+
@Override
1731+
public MasterProtos.ListTableDescriptorsByStateResponse listTableDescriptorsByState(
1732+
RpcController controller, MasterProtos.ListTableDescriptorsByStateRequest request)
1733+
throws ServiceException {
1734+
return stub.listTableDescriptorsByState(controller, request);
1735+
}
1736+
17301737
@Override
17311738
public MasterProtos.GetTableNamesResponse getTableNames(RpcController controller,
17321739
MasterProtos.GetTableNamesRequest request) throws ServiceException {
17331740
return stub.getTableNames(controller, request);
17341741
}
17351742

1743+
@Override
1744+
public MasterProtos.ListTableNamesByStateResponse listTableNamesByState(
1745+
RpcController controller, MasterProtos.ListTableNamesByStateRequest request)
1746+
throws ServiceException {
1747+
return stub.listTableNamesByState(controller, request);
1748+
}
1749+
17361750
@Override
17371751
public MasterProtos.GetClusterStatusResponse getClusterStatus(RpcController controller,
17381752
MasterProtos.GetClusterStatusRequest request) throws ServiceException {

hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -190,7 +190,11 @@
190190
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
191191
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesRequest;
192192
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
193+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateRequest;
194+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateResponse;
193195
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
196+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByStateRequest;
197+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByStateResponse;
194198
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
195199
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
196200
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
@@ -343,6 +347,20 @@ public List<TableDescriptor> listTableDescriptors() throws IOException {
343347
return listTableDescriptors((Pattern) null, false);
344348
}
345349

350+
@Override
351+
public List<TableDescriptor> listTableDescriptorsByState(boolean isEnabled) throws IOException {
352+
return executeCallable(
353+
new MasterCallable<List<TableDescriptor>>(getConnection(), getRpcControllerFactory()) {
354+
@Override
355+
protected List<TableDescriptor> rpcCall() throws Exception {
356+
ListTableDescriptorsByStateResponse response =
357+
master.listTableDescriptorsByState(getRpcController(),
358+
ListTableDescriptorsByStateRequest.newBuilder().setIsEnabled(isEnabled).build());
359+
return ProtobufUtil.toTableDescriptorList(response);
360+
}
361+
});
362+
}
363+
346364
@Override
347365
public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
348366
throws IOException {
@@ -566,6 +584,19 @@ public TableName[] listTableNames(final String regex, final boolean includeSysTa
566584
return listTableNames(Pattern.compile(regex), includeSysTables);
567585
}
568586

587+
@Override
588+
public List<TableName> listTableNamesByState(boolean isEnabled) throws IOException {
589+
return executeCallable(
590+
new MasterCallable<List<TableName>>(getConnection(), getRpcControllerFactory()) {
591+
@Override
592+
protected List<TableName> rpcCall() throws Exception {
593+
ListTableNamesByStateResponse response = master.listTableNamesByState(getRpcController(),
594+
ListTableNamesByStateRequest.newBuilder().setIsEnabled(isEnabled).build());
595+
return ProtobufUtil.toTableNameList(response.getTableNamesList());
596+
}
597+
});
598+
}
599+
569600
@Override
570601
public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
571602
return getHTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,

hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java

Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -219,8 +219,12 @@
219219
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesResponse;
220220
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
221221
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
222+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateRequest;
223+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateResponse;
222224
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
223225
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
226+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByStateRequest;
227+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByStateResponse;
224228
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
225229
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
226230
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
@@ -550,6 +554,17 @@ ListTableDescriptorsByNamespaceResponse, List<TableDescriptor>> call(controller,
550554
.call();
551555
}
552556

557+
@Override
558+
public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByState(boolean isEnabled) {
559+
return this.<List<TableDescriptor>> newMasterCaller()
560+
.action((controller, stub) -> this.<ListTableDescriptorsByStateRequest,
561+
ListTableDescriptorsByStateResponse, List<TableDescriptor>> call(controller, stub,
562+
ListTableDescriptorsByStateRequest.newBuilder().setIsEnabled(isEnabled).build(),
563+
(s, c, req, done) -> s.listTableDescriptorsByState(c, req, done),
564+
(resp) -> ProtobufUtil.toTableDescriptorList(resp)))
565+
.call();
566+
}
567+
553568
@Override
554569
public CompletableFuture<List<TableName>> listTableNamesByNamespace(String name) {
555570
return this.<List<TableName>> newMasterCaller()
@@ -561,6 +576,17 @@ ListTableNamesByNamespaceResponse, List<TableName>> call(controller, stub,
561576
.call();
562577
}
563578

579+
@Override
580+
public CompletableFuture<List<TableName>> listTableNamesByState(boolean isEnabled) {
581+
return this.<List<TableName>> newMasterCaller()
582+
.action((controller, stub) -> this.<ListTableNamesByStateRequest,
583+
ListTableNamesByStateResponse, List<TableName>> call(controller, stub,
584+
ListTableNamesByStateRequest.newBuilder().setIsEnabled(isEnabled).build(),
585+
(s, c, req, done) -> s.listTableNamesByState(c, req, done),
586+
(resp) -> ProtobufUtil.toTableNameList(resp.getTableNamesList())))
587+
.call();
588+
}
589+
564590
@Override
565591
public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName) {
566592
CompletableFuture<TableDescriptor> future = new CompletableFuture<>();

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -117,8 +117,12 @@
117117
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesResponse;
118118
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
119119
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
120+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateRequest;
121+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateResponse;
120122
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
121123
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
124+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByStateRequest;
125+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByStateResponse;
122126
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
123127
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
124128
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
@@ -443,12 +447,24 @@ public GetTableNamesResponse getTableNames(RpcController controller, GetTableNam
443447
return stub.getTableNames(controller, request);
444448
}
445449

450+
@Override
451+
public ListTableNamesByStateResponse listTableNamesByState(RpcController controller,
452+
ListTableNamesByStateRequest request) throws ServiceException {
453+
return stub.listTableNamesByState(controller, request);
454+
}
455+
446456
@Override
447457
public GetTableDescriptorsResponse getTableDescriptors(RpcController controller,
448458
GetTableDescriptorsRequest request) throws ServiceException {
449459
return stub.getTableDescriptors(controller, request);
450460
}
451461

462+
@Override
463+
public ListTableDescriptorsByStateResponse listTableDescriptorsByState(RpcController controller,
464+
ListTableDescriptorsByStateRequest request) throws ServiceException {
465+
return stub.listTableDescriptorsByState(controller, request);
466+
}
467+
452468
@Override
453469
public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller,
454470
SecurityCapabilitiesRequest request) throws ServiceException {

hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -195,6 +195,7 @@
195195
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
196196
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
197197
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
198+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByStateResponse;
198199
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
199200
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
200201
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
@@ -457,6 +458,18 @@ public static List<TableDescriptor> toTableDescriptorList(GetTableDescriptorsRes
457458
.collect(Collectors.toList());
458459
}
459460

461+
/**
462+
* Get a list of TableDescriptor from ListTableDescriptorsByNamespaceResponse protobuf
463+
* @param proto the ListTableDescriptorsByNamespaceResponse
464+
* @return a list of TableDescriptor
465+
*/
466+
public static List<TableDescriptor>
467+
toTableDescriptorList(ListTableDescriptorsByStateResponse proto) {
468+
if (proto == null) return new ArrayList<>();
469+
return proto.getTableSchemaList().stream().map(ProtobufUtil::toTableDescriptor)
470+
.collect(Collectors.toList());
471+
}
472+
460473
/**
461474
* get the split keys in form "byte [][]" from a CreateTableRequest proto
462475
* @param proto the CreateTableRequest

hbase-protocol-shaded/src/main/protobuf/Master.proto

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -511,6 +511,14 @@ message GetTableDescriptorsResponse {
511511
repeated TableSchema table_schema = 1;
512512
}
513513

514+
message ListTableDescriptorsByStateRequest {
515+
required bool is_enabled = 1;
516+
}
517+
518+
message ListTableDescriptorsByStateResponse {
519+
repeated TableSchema table_schema = 1;
520+
}
521+
514522
message GetTableNamesRequest {
515523
optional string regex = 1;
516524
optional bool include_sys_tables = 2 [default=false];
@@ -521,6 +529,14 @@ message GetTableNamesResponse {
521529
repeated TableName table_names = 1;
522530
}
523531

532+
message ListTableNamesByStateRequest {
533+
required bool is_enabled = 1;
534+
}
535+
536+
message ListTableNamesByStateResponse {
537+
repeated TableName table_names = 1;
538+
}
539+
524540
message GetTableStateRequest {
525541
required TableName table_name = 1;
526542
}
@@ -769,10 +785,18 @@ service MasterService {
769785
rpc GetTableDescriptors(GetTableDescriptorsRequest)
770786
returns(GetTableDescriptorsResponse);
771787

788+
/** List all enabled or disabled table descriptors. */
789+
rpc ListTableDescriptorsByState(ListTableDescriptorsByStateRequest)
790+
returns(ListTableDescriptorsByStateResponse);
791+
772792
/** Get the list of table names. */
773793
rpc GetTableNames(GetTableNamesRequest)
774794
returns(GetTableNamesResponse);
775795

796+
/** List all enabled or disabled table names. */
797+
rpc ListTableNamesByState(ListTableNamesByStateRequest)
798+
returns(ListTableNamesByStateResponse);
799+
776800
/** Return cluster status. */
777801
rpc GetClusterStatus(GetClusterStatusRequest)
778802
returns(GetClusterStatusResponse);

0 commit comments

Comments
 (0)