Skip to content

Commit e4cbe57

Browse files
committed
HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (apache#994)
* HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin No need to create and close a new registry on demand. Audited other usages of getRegistry() and the code looks fine. * Fix checkstyle issues in RawAsyncHBaseAdmin
1 parent 99afd13 commit e4cbe57

File tree

1 file changed

+47
-63
lines changed

1 file changed

+47
-63
lines changed

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

Lines changed: 47 additions & 63 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
/**
1+
/*
22
* Licensed to the Apache Software Foundation (ASF) under one
33
* or more contributor license agreements. See the NOTICE file
44
* distributed with this work for additional information
@@ -21,7 +21,6 @@
2121
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
2222
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
2323
import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException;
24-
2524
import com.google.protobuf.Message;
2625
import com.google.protobuf.RpcChannel;
2726
import java.io.IOException;
@@ -46,7 +45,6 @@
4645
import java.util.regex.Pattern;
4746
import java.util.stream.Collectors;
4847
import java.util.stream.Stream;
49-
import org.apache.commons.io.IOUtils;
5048
import org.apache.hadoop.conf.Configuration;
5149
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
5250
import org.apache.hadoop.hbase.CacheEvictionStats;
@@ -99,14 +97,12 @@
9997
import org.apache.yetus.audience.InterfaceAudience;
10098
import org.slf4j.Logger;
10199
import org.slf4j.LoggerFactory;
102-
103100
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
104101
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
105102
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
106103
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
107104
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
108105
import org.apache.hbase.thirdparty.io.netty.util.TimerTask;
109-
110106
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
111107
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
112108
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
@@ -755,7 +751,8 @@ public CompletableFuture<Boolean> isTableAvailable(TableName tableName) {
755751
}
756752

757753
@Override
758-
public CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) {
754+
public CompletableFuture<Void> addColumnFamily(
755+
TableName tableName, ColumnFamilyDescriptor columnFamily) {
759756
return this.<AddColumnRequest, AddColumnResponse> procedureCall(tableName,
760757
RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
761758
ng.newNonce()), (s, c, req, done) -> s.addColumn(c, req, done), (resp) -> resp.getProcId(),
@@ -809,10 +806,10 @@ public CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name
809806
.<NamespaceDescriptor> newMasterCaller()
810807
.action(
811808
(controller, stub) -> this
812-
.<GetNamespaceDescriptorRequest, GetNamespaceDescriptorResponse, NamespaceDescriptor> call(
813-
controller, stub, RequestConverter.buildGetNamespaceDescriptorRequest(name), (s, c,
814-
req, done) -> s.getNamespaceDescriptor(c, req, done), (resp) -> ProtobufUtil
815-
.toNamespaceDescriptor(resp.getNamespaceDescriptor()))).call();
809+
.<GetNamespaceDescriptorRequest, GetNamespaceDescriptorResponse, NamespaceDescriptor>
810+
call(controller, stub, RequestConverter.buildGetNamespaceDescriptorRequest(name),
811+
(s, c, req, done) -> s.getNamespaceDescriptor(c, req, done), (resp)
812+
-> ProtobufUtil.toNamespaceDescriptor(resp.getNamespaceDescriptor()))).call();
816813
}
817814

818815
@Override
@@ -830,13 +827,12 @@ public CompletableFuture<List<String>> listNamespaces() {
830827
@Override
831828
public CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors() {
832829
return this
833-
.<List<NamespaceDescriptor>> newMasterCaller()
834-
.action(
835-
(controller, stub) -> this
836-
.<ListNamespaceDescriptorsRequest, ListNamespaceDescriptorsResponse, List<NamespaceDescriptor>> call(
837-
controller, stub, ListNamespaceDescriptorsRequest.newBuilder().build(), (s, c, req,
838-
done) -> s.listNamespaceDescriptors(c, req, done), (resp) -> ProtobufUtil
839-
.toNamespaceDescriptorList(resp))).call();
830+
.<List<NamespaceDescriptor>> newMasterCaller().action((controller, stub) -> this
831+
.<ListNamespaceDescriptorsRequest, ListNamespaceDescriptorsResponse,
832+
List<NamespaceDescriptor>> call(controller, stub,
833+
ListNamespaceDescriptorsRequest.newBuilder().build(), (s, c, req, done) ->
834+
s.listNamespaceDescriptors(c, req, done),
835+
(resp) -> ProtobufUtil.toNamespaceDescriptorList(resp))).call();
840836
}
841837

842838
@Override
@@ -1080,10 +1076,7 @@ private CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFa
10801076
private CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(TableName tableName) {
10811077
if (TableName.META_TABLE_NAME.equals(tableName)) {
10821078
CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
1083-
// For meta table, we use zk to fetch all locations.
1084-
ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(
1085-
connection.getConfiguration());
1086-
addListener(registry.getMetaRegionLocations(), (metaRegions, err) -> {
1079+
addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> {
10871080
if (err != null) {
10881081
future.completeExceptionally(err);
10891082
} else if (metaRegions == null || metaRegions.isEmpty() ||
@@ -1092,8 +1085,6 @@ private CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(TableN
10921085
} else {
10931086
future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
10941087
}
1095-
// close the registry.
1096-
IOUtils.closeQuietly(registry);
10971088
});
10981089
return future;
10991090
} else {
@@ -1689,11 +1680,10 @@ public CompletableFuture<Void> disableReplicationPeer(String peerId) {
16891680
@Override
16901681
public CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId) {
16911682
return this.<ReplicationPeerConfig> newMasterCaller().action((controller, stub) -> this
1692-
.<GetReplicationPeerConfigRequest, GetReplicationPeerConfigResponse, ReplicationPeerConfig> call(
1693-
controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId),
1694-
(s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
1695-
(resp) -> ReplicationPeerConfigUtil.convert(resp.getPeerConfig())))
1696-
.call();
1683+
.<GetReplicationPeerConfigRequest, GetReplicationPeerConfigResponse, ReplicationPeerConfig>
1684+
call(controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId),
1685+
(s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
1686+
(resp) -> ReplicationPeerConfigUtil.convert(resp.getPeerConfig()))).call();
16971687
}
16981688

16991689
@Override
@@ -1710,13 +1700,13 @@ public CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
17101700
@Override
17111701
public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
17121702
SyncReplicationState clusterState) {
1713-
return this
1714-
.<TransitReplicationPeerSyncReplicationStateRequest, TransitReplicationPeerSyncReplicationStateResponse> procedureCall(
1703+
return this.<TransitReplicationPeerSyncReplicationStateRequest,
1704+
TransitReplicationPeerSyncReplicationStateResponse> procedureCall(
17151705
RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId,
17161706
clusterState),
1717-
(s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done),
1718-
(resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId,
1719-
() -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"));
1707+
(s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done),
1708+
(resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId,
1709+
() -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"));
17201710
}
17211711

17221712
@Override
@@ -1786,11 +1776,8 @@ private CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers
17861776
return this
17871777
.<List<ReplicationPeerDescription>> newMasterCaller()
17881778
.action(
1789-
(controller, stub) -> this
1790-
.<ListReplicationPeersRequest, ListReplicationPeersResponse, List<ReplicationPeerDescription>> call(
1791-
controller,
1792-
stub,
1793-
request,
1779+
(controller, stub) -> this.<ListReplicationPeersRequest, ListReplicationPeersResponse,
1780+
List<ReplicationPeerDescription>> call(controller, stub, request,
17941781
(s, c, req, done) -> s.listReplicationPeers(c, req, done),
17951782
(resp) -> resp.getPeerDescList().stream()
17961783
.map(ReplicationPeerConfigUtil::toReplicationPeerDescription)
@@ -2299,11 +2286,13 @@ public CompletableFuture<String> getLocks() {
22992286
}
23002287

23012288
@Override
2302-
public CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers, boolean offload) {
2289+
public CompletableFuture<Void> decommissionRegionServers(
2290+
List<ServerName> servers, boolean offload) {
23032291
return this.<Void> newMasterCaller()
23042292
.action((controller, stub) -> this
23052293
.<DecommissionRegionServersRequest, DecommissionRegionServersResponse, Void> call(
2306-
controller, stub, RequestConverter.buildDecommissionRegionServersRequest(servers, offload),
2294+
controller, stub,
2295+
RequestConverter.buildDecommissionRegionServersRequest(servers, offload),
23072296
(s, c, req, done) -> s.decommissionRegionServers(c, req, done), resp -> null))
23082297
.call();
23092298
}
@@ -2325,11 +2314,11 @@ List<ServerName>> call(
23252314
public CompletableFuture<Void> recommissionRegionServer(ServerName server,
23262315
List<byte[]> encodedRegionNames) {
23272316
return this.<Void> newMasterCaller()
2328-
.action((controller, stub) -> this
2329-
.<RecommissionRegionServerRequest, RecommissionRegionServerResponse, Void> call(controller,
2330-
stub, RequestConverter.buildRecommissionRegionServerRequest(server, encodedRegionNames),
2331-
(s, c, req, done) -> s.recommissionRegionServer(c, req, done), resp -> null))
2332-
.call();
2317+
.action((controller, stub) ->
2318+
this.<RecommissionRegionServerRequest, RecommissionRegionServerResponse, Void> call(
2319+
controller, stub, RequestConverter.buildRecommissionRegionServerRequest(
2320+
server, encodedRegionNames), (s, c, req, done) -> s.recommissionRegionServer(
2321+
c, req, done), resp -> null)).call();
23332322
}
23342323

23352324
/**
@@ -2395,7 +2384,6 @@ CompletableFuture<HRegionLocation> getRegionLocation(byte[] regionNameOrEncodedR
23952384
* Get the region info for the passed region name. The region name may be a full region name or
23962385
* encoded region name. If the region does not found, then it'll throw an UnknownRegionException
23972386
* wrapped by a {@link CompletableFuture}
2398-
* @param regionNameOrEncodedRegionName
23992387
* @return region info, wrapped by a {@link CompletableFuture}
24002388
*/
24012389
private CompletableFuture<RegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
@@ -2886,10 +2874,11 @@ public CompletableFuture<List<SecurityCapability>> getSecurityCapabilities() {
28862874
.<List<SecurityCapability>> newMasterCaller()
28872875
.action(
28882876
(controller, stub) -> this
2889-
.<SecurityCapabilitiesRequest, SecurityCapabilitiesResponse, List<SecurityCapability>> call(
2890-
controller, stub, SecurityCapabilitiesRequest.newBuilder().build(), (s, c, req,
2891-
done) -> s.getSecurityCapabilities(c, req, done), (resp) -> ProtobufUtil
2892-
.toSecurityCapabilityList(resp.getCapabilitiesList()))).call();
2877+
.<SecurityCapabilitiesRequest, SecurityCapabilitiesResponse, List<SecurityCapability>>
2878+
call(controller, stub, SecurityCapabilitiesRequest.newBuilder().build(),
2879+
(s, c, req, done) -> s.getSecurityCapabilities(c, req, done),
2880+
(resp) -> ProtobufUtil.toSecurityCapabilityList(resp.getCapabilitiesList())))
2881+
.call();
28932882
}
28942883

28952884
@Override
@@ -3066,14 +3055,10 @@ public CompletableFuture<Optional<Long>> getLastMajorCompactionTimestamp(TableNa
30663055
MajorCompactionTimestampRequest request =
30673056
MajorCompactionTimestampRequest.newBuilder()
30683057
.setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
3069-
return this
3070-
.<Optional<Long>> newMasterCaller()
3071-
.action(
3072-
(controller, stub) -> this
3073-
.<MajorCompactionTimestampRequest, MajorCompactionTimestampResponse, Optional<Long>> call(
3074-
controller, stub, request,
3075-
(s, c, req, done) -> s.getLastMajorCompactionTimestamp(c, req, done),
3076-
ProtobufUtil::toOptionalTimestamp)).call();
3058+
return this.<Optional<Long>> newMasterCaller().action((controller, stub) ->
3059+
this.<MajorCompactionTimestampRequest, MajorCompactionTimestampResponse, Optional<Long>>
3060+
call(controller, stub, request, (s, c, req, done) -> s.getLastMajorCompactionTimestamp(
3061+
c, req, done), ProtobufUtil::toOptionalTimestamp)).call();
30773062
}
30783063

30793064
@Override
@@ -3213,11 +3198,10 @@ public CompletableFuture<Boolean> balance(boolean forcible) {
32133198
public CompletableFuture<Boolean> isBalancerEnabled() {
32143199
return this
32153200
.<Boolean> newMasterCaller()
3216-
.action(
3217-
(controller, stub) -> this.<IsBalancerEnabledRequest, IsBalancerEnabledResponse, Boolean> call(
3218-
controller, stub, RequestConverter.buildIsBalancerEnabledRequest(),
3219-
(s, c, req, done) -> s.isBalancerEnabled(c, req, done), (resp) -> resp.getEnabled()))
3220-
.call();
3201+
.action((controller, stub) ->
3202+
this.<IsBalancerEnabledRequest, IsBalancerEnabledResponse, Boolean> call(controller,
3203+
stub, RequestConverter.buildIsBalancerEnabledRequest(), (s, c, req, done)
3204+
-> s.isBalancerEnabled(c, req, done), (resp) -> resp.getEnabled())).call();
32213205
}
32223206

32233207
@Override

0 commit comments

Comments
 (0)