@@ -313,7 +313,7 @@ public void close() throws Exception {
313
313
}
314
314
315
315
@ Override
316
- protected void confirmLeadership (
316
+ protected CompletableFuture < Void > confirmLeadershipAsync (
317
317
String componentId , UUID leaderSessionID , String leaderAddress ) {
318
318
Preconditions .checkArgument (leaderContenderRegistry .containsKey (componentId ));
319
319
LOG .debug (
@@ -324,59 +324,73 @@ protected void confirmLeadership(
324
324
325
325
checkNotNull (leaderSessionID );
326
326
327
- synchronized (lock ) {
328
- if (hasLeadership (componentId , leaderSessionID )) {
329
- Preconditions .checkState (
330
- leaderElectionDriver != null ,
331
- "The leadership check should only return true if a driver is instantiated." );
332
- Preconditions .checkState (
333
- !confirmedLeaderInformation .hasLeaderInformation (componentId ),
334
- "No confirmation should have happened, yet." );
335
-
336
- final LeaderInformation newConfirmedLeaderInformation =
337
- LeaderInformation .known (leaderSessionID , leaderAddress );
338
- confirmedLeaderInformation =
339
- LeaderInformationRegister .merge (
340
- confirmedLeaderInformation ,
341
- componentId ,
342
- newConfirmedLeaderInformation );
343
- leaderElectionDriver .publishLeaderInformation (
344
- componentId , newConfirmedLeaderInformation );
345
- } else {
346
- if (!leaderSessionID .equals (this .issuedLeaderSessionID )) {
347
- LOG .debug (
348
- "Received an old confirmation call of leader session ID {} for component '{}' (current issued session ID is {})." ,
349
- leaderSessionID ,
350
- componentId ,
351
- issuedLeaderSessionID );
352
- } else {
353
- LOG .warn (
354
- "The leader session ID {} for component '{}' was confirmed even though the corresponding "
355
- + "service was not elected as the leader or has been stopped already." ,
356
- componentId ,
357
- leaderSessionID );
358
- }
359
- }
360
- }
327
+ return CompletableFuture .runAsync (
328
+ () -> {
329
+ synchronized (lock ) {
330
+ if (hasLeadershipInternal (componentId , leaderSessionID )) {
331
+ Preconditions .checkState (
332
+ leaderElectionDriver != null ,
333
+ "The leadership check should only return true if a driver is instantiated." );
334
+ Preconditions .checkState (
335
+ !confirmedLeaderInformation .hasLeaderInformation (componentId ),
336
+ "No confirmation should have happened, yet." );
337
+
338
+ final LeaderInformation newConfirmedLeaderInformation =
339
+ LeaderInformation .known (leaderSessionID , leaderAddress );
340
+ confirmedLeaderInformation =
341
+ LeaderInformationRegister .merge (
342
+ confirmedLeaderInformation ,
343
+ componentId ,
344
+ newConfirmedLeaderInformation );
345
+ leaderElectionDriver .publishLeaderInformation (
346
+ componentId , newConfirmedLeaderInformation );
347
+ } else {
348
+ if (!leaderSessionID .equals (this .issuedLeaderSessionID )) {
349
+ LOG .debug (
350
+ "Received an old confirmation call of leader session ID {} for component '{}' (current issued session ID is {})." ,
351
+ leaderSessionID ,
352
+ componentId ,
353
+ issuedLeaderSessionID );
354
+ } else {
355
+ LOG .warn (
356
+ "The leader session ID {} for component '{}' was confirmed even though the corresponding "
357
+ + "service was not elected as the leader or has been stopped already." ,
358
+ componentId ,
359
+ leaderSessionID );
360
+ }
361
+ }
362
+ }
363
+ },
364
+ leadershipOperationExecutor );
361
365
}
362
366
363
367
@ Override
364
- protected boolean hasLeadership (String componentId , UUID leaderSessionId ) {
365
- synchronized (lock ) {
366
- if (leaderElectionDriver != null ) {
367
- if (leaderContenderRegistry .containsKey (componentId )) {
368
- return leaderElectionDriver .hasLeadership ()
369
- && leaderSessionId .equals (issuedLeaderSessionID );
370
- } else {
371
- LOG .debug (
372
- "hasLeadership is called for component '{}' while there is no contender registered under that ID in the service, returning false." ,
373
- componentId );
374
- return false ;
375
- }
368
+ protected CompletableFuture <Boolean > hasLeadershipAsync (
369
+ String componentId , UUID leaderSessionId ) {
370
+ return CompletableFuture .supplyAsync (
371
+ () -> {
372
+ synchronized (lock ) {
373
+ return hasLeadershipInternal (componentId , leaderSessionId );
374
+ }
375
+ },
376
+ leadershipOperationExecutor );
377
+ }
378
+
379
+ @ GuardedBy ("lock" )
380
+ private boolean hasLeadershipInternal (String componentId , UUID leaderSessionId ) {
381
+ if (leaderElectionDriver != null ) {
382
+ if (leaderContenderRegistry .containsKey (componentId )) {
383
+ return leaderElectionDriver .hasLeadership ()
384
+ && leaderSessionId .equals (issuedLeaderSessionID );
376
385
} else {
377
- LOG .debug ("hasLeadership is called after the service is closed, returning false." );
386
+ LOG .debug (
387
+ "hasLeadership is called for component '{}' while there is no contender registered under that ID in the service, returning false." ,
388
+ componentId );
378
389
return false ;
379
390
}
391
+ } else {
392
+ LOG .debug ("hasLeadership is called after the service is closed, returning false." );
393
+ return false ;
380
394
}
381
395
}
382
396
0 commit comments