1717 */
1818package org .apache .hadoop .hbase .client ;
1919
20- import static org .apache .hadoop .hbase .HConstants .DEFAULT_ZK_SESSION_TIMEOUT ;
21- import static org .apache .hadoop .hbase .HConstants .ZK_SESSION_TIMEOUT ;
2220import static org .apache .hadoop .hbase .client .RegionInfo .DEFAULT_REPLICA_ID ;
2321import static org .apache .hadoop .hbase .client .RegionInfoBuilder .FIRST_META_REGIONINFO ;
2422import static org .apache .hadoop .hbase .client .RegionReplicaUtil .getRegionInfoForDefaultReplica ;
2826
2927import java .io .IOException ;
3028import java .util .concurrent .CompletableFuture ;
31- import java .util .concurrent .TimeUnit ;
3229
3330import org .apache .commons .lang3 .mutable .MutableInt ;
3431import org .apache .commons .logging .Log ;
3532import org .apache .commons .logging .LogFactory ;
36- import org .apache .curator .framework .CuratorFramework ;
37- import org .apache .curator .framework .CuratorFrameworkFactory ;
38- import org .apache .curator .framework .api .BackgroundPathable ;
39- import org .apache .curator .framework .api .CuratorEvent ;
40- import org .apache .curator .retry .RetryNTimes ;
4133import org .apache .hadoop .conf .Configuration ;
4234import org .apache .hadoop .hbase .ClusterId ;
4335import org .apache .hadoop .hbase .HRegionLocation ;
4436import org .apache .hadoop .hbase .RegionLocations ;
4537import org .apache .hadoop .hbase .ServerName ;
4638import org .apache .hadoop .hbase .exceptions .DeserializationException ;
4739import org .apache .hadoop .hbase .master .RegionState ;
48- import org .apache .hadoop .hbase .shaded .com .google .common .annotations .VisibleForTesting ;
49- import org .apache .hadoop .hbase .shaded .protobuf .generated .HBaseProtos ;
50- import org .apache .hadoop .hbase .shaded .protobuf .generated .ZooKeeperProtos ;
5140import org .apache .hadoop .hbase .util .Pair ;
52- import org .apache .hadoop .hbase .util .Threads ;
53- import org .apache .hadoop .hbase .zookeeper .ZKConfig ;
41+ import org .apache .hadoop .hbase .zookeeper .ReadOnlyZKClient ;
5442import org .apache .hadoop .hbase .zookeeper .ZNodePaths ;
5543import org .apache .yetus .audience .InterfaceAudience ;
56- import org .apache .zookeeper .data .Stat ;
5744
45+ import org .apache .hadoop .hbase .shaded .com .google .common .annotations .VisibleForTesting ;
46+ import org .apache .hadoop .hbase .shaded .protobuf .generated .HBaseProtos ;
47+ import org .apache .hadoop .hbase .shaded .protobuf .generated .ZooKeeperProtos ;
5848
5949/**
6050 * Fetch the registry data from zookeeper.
@@ -64,53 +54,36 @@ class ZKAsyncRegistry implements AsyncRegistry {
6454
6555 private static final Log LOG = LogFactory .getLog (ZKAsyncRegistry .class );
6656
67- private final CuratorFramework zk ;
57+ private final ReadOnlyZKClient zk ;
6858
6959 private final ZNodePaths znodePaths ;
7060
7161 ZKAsyncRegistry (Configuration conf ) {
7262 this .znodePaths = new ZNodePaths (conf );
73- int zkSessionTimeout = conf .getInt (ZK_SESSION_TIMEOUT , DEFAULT_ZK_SESSION_TIMEOUT );
74- int zkRetry = conf .getInt ("zookeeper.recovery.retry" , 30 );
75- int zkRetryIntervalMs = conf .getInt ("zookeeper.recovery.retry.intervalmill" , 1000 );
76- this .zk = CuratorFrameworkFactory .builder ()
77- .connectString (ZKConfig .getZKQuorumServersString (conf )).sessionTimeoutMs (zkSessionTimeout )
78- .retryPolicy (new RetryNTimes (zkRetry , zkRetryIntervalMs ))
79- .threadFactory (
80- Threads .newDaemonThreadFactory (String .format ("ZKClusterRegistry-0x%08x" , hashCode ())))
81- .build ();
82- this .zk .start ();
83- // TODO: temporary workaround for HBASE-19312, must be removed before 2.0.0 release!
84- try {
85- this .zk .blockUntilConnected (2 , TimeUnit .SECONDS );
86- } catch (InterruptedException e ) {
87- return ;
88- }
63+ this .zk = new ReadOnlyZKClient (conf );
8964 }
9065
91- private interface CuratorEventProcessor <T > {
92- T process ( CuratorEvent event ) throws Exception ;
66+ private interface Converter <T > {
67+ T convert ( byte [] data ) throws Exception ;
9368 }
9469
95- private static <T > CompletableFuture <T > exec (BackgroundPathable <?> opBuilder , String path ,
96- CuratorEventProcessor <T > processor ) {
70+ private <T > CompletableFuture <T > getAndConvert (String path , Converter <T > converter ) {
9771 CompletableFuture <T > future = new CompletableFuture <>();
98- try {
99- opBuilder . inBackground (( client , event ) -> {
100- try {
101- future . complete ( processor . process ( event )) ;
102- } catch ( Exception e ) {
103- future . completeExceptionally ( e );
104- }
105- }). withUnhandledErrorListener (( msg , e ) -> future . completeExceptionally ( e )). forPath ( path );
106- } catch ( Exception e ) {
107- future . completeExceptionally ( e );
108- }
72+ zk . get ( path ). whenComplete (( data , error ) -> {
73+ if ( error != null ) {
74+ future . completeExceptionally ( error );
75+ return ;
76+ }
77+ try {
78+ future . complete ( converter . convert ( data ));
79+ } catch ( Exception e ) {
80+ future . completeExceptionally ( e );
81+ }
82+ });
10983 return future ;
11084 }
11185
112- private static String getClusterId (CuratorEvent event ) throws DeserializationException {
113- byte [] data = event .getData ();
86+ private static String getClusterId (byte [] data ) throws DeserializationException {
11487 if (data == null || data .length == 0 ) {
11588 return null ;
11689 }
@@ -120,17 +93,15 @@ private static String getClusterId(CuratorEvent event) throws DeserializationExc
12093
12194 @ Override
12295 public CompletableFuture <String > getClusterId () {
123- return exec ( zk . getData (), znodePaths .clusterIdZNode , ZKAsyncRegistry ::getClusterId );
96+ return getAndConvert ( znodePaths .clusterIdZNode , ZKAsyncRegistry ::getClusterId );
12497 }
12598
12699 @ VisibleForTesting
127- CuratorFramework getCuratorFramework () {
100+ ReadOnlyZKClient getZKClient () {
128101 return zk ;
129102 }
130103
131- private static ZooKeeperProtos .MetaRegionServer getMetaProto (CuratorEvent event )
132- throws IOException {
133- byte [] data = event .getData ();
104+ private static ZooKeeperProtos .MetaRegionServer getMetaProto (byte [] data ) throws IOException {
134105 if (data == null || data .length == 0 ) {
135106 return null ;
136107 }
@@ -169,7 +140,7 @@ public CompletableFuture<RegionLocations> getMetaRegionLocation() {
169140 MutableInt remaining = new MutableInt (locs .length );
170141 znodePaths .metaReplicaZNodes .forEach ((replicaId , path ) -> {
171142 if (replicaId == DEFAULT_REPLICA_ID ) {
172- exec ( zk . getData (), path , ZKAsyncRegistry ::getMetaProto ).whenComplete ((proto , error ) -> {
143+ getAndConvert ( path , ZKAsyncRegistry ::getMetaProto ).whenComplete ((proto , error ) -> {
173144 if (error != null ) {
174145 future .completeExceptionally (error );
175146 return ;
@@ -184,13 +155,13 @@ public CompletableFuture<RegionLocations> getMetaRegionLocation() {
184155 new IOException ("Meta region is in state " + stateAndServerName .getFirst ()));
185156 return ;
186157 }
187- locs [DEFAULT_REPLICA_ID ] = new HRegionLocation (
188- getRegionInfoForDefaultReplica (FIRST_META_REGIONINFO ),
189- stateAndServerName .getSecond ());
158+ locs [DEFAULT_REPLICA_ID ] =
159+ new HRegionLocation ( getRegionInfoForDefaultReplica (FIRST_META_REGIONINFO ),
160+ stateAndServerName .getSecond ());
190161 tryComplete (remaining , locs , future );
191162 });
192163 } else {
193- exec ( zk . getData (), path , ZKAsyncRegistry ::getMetaProto ).whenComplete ((proto , error ) -> {
164+ getAndConvert ( path , ZKAsyncRegistry ::getMetaProto ).whenComplete ((proto , error ) -> {
194165 if (future .isDone ()) {
195166 return ;
196167 }
@@ -203,13 +174,13 @@ public CompletableFuture<RegionLocations> getMetaRegionLocation() {
203174 } else {
204175 Pair <RegionState .State , ServerName > stateAndServerName = getStateAndServerName (proto );
205176 if (stateAndServerName .getFirst () != RegionState .State .OPEN ) {
206- LOG .warn ("Meta region for replica " + replicaId + " is in state "
207- + stateAndServerName .getFirst ());
177+ LOG .warn ("Meta region for replica " + replicaId + " is in state " +
178+ stateAndServerName .getFirst ());
208179 locs [replicaId ] = null ;
209180 } else {
210- locs [replicaId ] = new HRegionLocation (
211- getRegionInfoForReplica (FIRST_META_REGIONINFO , replicaId ),
212- stateAndServerName .getSecond ());
181+ locs [replicaId ] =
182+ new HRegionLocation ( getRegionInfoForReplica (FIRST_META_REGIONINFO , replicaId ),
183+ stateAndServerName .getSecond ());
213184 }
214185 }
215186 tryComplete (remaining , locs , future );
@@ -219,18 +190,12 @@ public CompletableFuture<RegionLocations> getMetaRegionLocation() {
219190 return future ;
220191 }
221192
222- private static int getCurrentNrHRS (CuratorEvent event ) {
223- Stat stat = event .getStat ();
224- return stat != null ? stat .getNumChildren () : 0 ;
225- }
226-
227193 @ Override
228194 public CompletableFuture <Integer > getCurrentNrHRS () {
229- return exec ( zk .checkExists (), znodePaths .rsZNode , ZKAsyncRegistry :: getCurrentNrHRS );
195+ return zk .exists ( znodePaths .rsZNode ). thenApply ( s -> s != null ? s . getNumChildren () : 0 );
230196 }
231197
232- private static ZooKeeperProtos .Master getMasterProto (CuratorEvent event ) throws IOException {
233- byte [] data = event .getData ();
198+ private static ZooKeeperProtos .Master getMasterProto (byte [] data ) throws IOException {
234199 if (data == null || data .length == 0 ) {
235200 return null ;
236201 }
@@ -241,7 +206,7 @@ private static ZooKeeperProtos.Master getMasterProto(CuratorEvent event) throws
241206
242207 @ Override
243208 public CompletableFuture <ServerName > getMasterAddress () {
244- return exec ( zk . getData (), znodePaths .masterAddressZNode , ZKAsyncRegistry ::getMasterProto )
209+ return getAndConvert ( znodePaths .masterAddressZNode , ZKAsyncRegistry ::getMasterProto )
245210 .thenApply (proto -> {
246211 if (proto == null ) {
247212 return null ;
@@ -254,7 +219,7 @@ public CompletableFuture<ServerName> getMasterAddress() {
254219
255220 @ Override
256221 public CompletableFuture <Integer > getMasterInfoPort () {
257- return exec ( zk . getData (), znodePaths .masterAddressZNode , ZKAsyncRegistry ::getMasterProto )
222+ return getAndConvert ( znodePaths .masterAddressZNode , ZKAsyncRegistry ::getMasterProto )
258223 .thenApply (proto -> proto != null ? proto .getInfoPort () : 0 );
259224 }
260225
0 commit comments