Skip to content

Commit 4c12066

Browse files
committed
[FLINK-36451][runtime] Adds Async suffix to method names
1 parent ed0697d commit 4c12066

File tree

19 files changed

+78
-74
lines changed

19 files changed

+78
-74
lines changed

flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -178,7 +178,7 @@ private void forwardConfirmLeaderSessionFuture(
178178
.getLeaderAddressFuture()
179179
.thenCompose(
180180
leaderAddress ->
181-
leaderElection.confirmLeadership(
181+
leaderElection.confirmLeadershipAsync(
182182
leaderSessionID, leaderAddress)));
183183
}
184184

flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -467,15 +467,15 @@ public void close() {
467467
}
468468

469469
@Override
470-
public CompletableFuture<Void> confirmLeadership(
470+
public CompletableFuture<Void> confirmLeadershipAsync(
471471
UUID leaderSessionID, String leaderAddress) {
472472
checkNotNull(leaderSessionID);
473473
checkNotNull(leaderAddress);
474474
return confirmLeader(this, leaderSessionID, leaderAddress);
475475
}
476476

477477
@Override
478-
public CompletableFuture<Boolean> hasLeadership(UUID leaderSessionId) {
478+
public CompletableFuture<Boolean> hasLeadershipAsync(UUID leaderSessionId) {
479479
return CompletableFuture.completedFuture(
480480
isLeader && leaderSessionId.equals(currentLeaderSessionId));
481481
}

flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -344,7 +344,7 @@ private void confirmLeadership(
344344
LOG.debug(
345345
"Confirm leadership {}.",
346346
leaderSessionId);
347-
return leaderElection.confirmLeadership(
347+
return leaderElection.confirmLeadershipAsync(
348348
leaderSessionId, address);
349349
},
350350
"confirming leadership")
@@ -487,7 +487,7 @@ private CompletableFuture<Void> runIfValidLeader(
487487
synchronized (lock) {
488488
if (isRunning() && leaderElection != null) {
489489
return leaderElection
490-
.hasLeadership(expectedLeaderId)
490+
.hasLeadershipAsync(expectedLeaderId)
491491
.thenAccept(
492492
hasLeadership -> {
493493
synchronized (lock) {

flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElection.java

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -44,13 +44,14 @@ public void startLeaderElection(LeaderContender contender) throws Exception {
4444
}
4545

4646
@Override
47-
public CompletableFuture<Void> confirmLeadership(UUID leaderSessionID, String leaderAddress) {
48-
return parentService.confirmLeadership(componentId, leaderSessionID, leaderAddress);
47+
public CompletableFuture<Void> confirmLeadershipAsync(
48+
UUID leaderSessionID, String leaderAddress) {
49+
return parentService.confirmLeadershipAsync(componentId, leaderSessionID, leaderAddress);
4950
}
5051

5152
@Override
52-
public CompletableFuture<Boolean> hasLeadership(UUID leaderSessionId) {
53-
return parentService.hasLeadership(componentId, leaderSessionId);
53+
public CompletableFuture<Boolean> hasLeadershipAsync(UUID leaderSessionId) {
54+
return parentService.hasLeadershipAsync(componentId, leaderSessionId);
5455
}
5556

5657
@Override
@@ -82,7 +83,7 @@ abstract static class ParentService {
8283
* the {@link LeaderContender} that is associated with the {@code componentId}. The
8384
* information is only propagated to the HA backend if the leadership is still acquired.
8485
*/
85-
abstract CompletableFuture<Void> confirmLeadership(
86+
abstract CompletableFuture<Void> confirmLeadershipAsync(
8687
String componentId, UUID leaderSessionID, String leaderAddress);
8788

8889
/**
@@ -92,6 +93,7 @@ abstract CompletableFuture<Void> confirmLeadership(
9293
* @return {@code true} if the service has leadership with the passed {@code
9394
* leaderSessionID} acquired; {@code false} otherwise.
9495
*/
95-
abstract CompletableFuture<Boolean> hasLeadership(String componentId, UUID leaderSessionID);
96+
abstract CompletableFuture<Boolean> hasLeadershipAsync(
97+
String componentId, UUID leaderSessionID);
9698
}
9799
}

flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -313,7 +313,7 @@ public void close() throws Exception {
313313
}
314314

315315
@Override
316-
protected CompletableFuture<Void> confirmLeadership(
316+
protected CompletableFuture<Void> confirmLeadershipAsync(
317317
String componentId, UUID leaderSessionID, String leaderAddress) {
318318
Preconditions.checkArgument(leaderContenderRegistry.containsKey(componentId));
319319
LOG.debug(
@@ -365,7 +365,8 @@ protected CompletableFuture<Void> confirmLeadership(
365365
}
366366

367367
@Override
368-
protected CompletableFuture<Boolean> hasLeadership(String componentId, UUID leaderSessionId) {
368+
protected CompletableFuture<Boolean> hasLeadershipAsync(
369+
String componentId, UUID leaderSessionId) {
369370
return CompletableFuture.supplyAsync(
370371
() -> {
371372
synchronized (lock) {

flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElection.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ public interface LeaderElection extends AutoCloseable {
4343
* @param leaderSessionID The new leader session ID
4444
* @param leaderAddress The address of the new leader
4545
*/
46-
CompletableFuture<Void> confirmLeadership(UUID leaderSessionID, String leaderAddress);
46+
CompletableFuture<Void> confirmLeadershipAsync(UUID leaderSessionID, String leaderAddress);
4747

4848
/**
4949
* Returns {@code true} if the service's {@link LeaderContender} has the leadership under the
@@ -52,7 +52,7 @@ public interface LeaderElection extends AutoCloseable {
5252
* @param leaderSessionId identifying the current leader
5353
* @return true if the associated {@link LeaderContender} is the leader, otherwise false
5454
*/
55-
CompletableFuture<Boolean> hasLeadership(UUID leaderSessionId);
55+
CompletableFuture<Boolean> hasLeadershipAsync(UUID leaderSessionId);
5656

5757
/**
5858
* Closes the {@code LeaderElection} by deregistering the {@link LeaderContender} from the

flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -28,10 +28,10 @@
2828
* instantiate its own leader election service.
2929
*
3030
* <p>Once a contender has been granted leadership he has to confirm the received leader session ID
31-
* by calling the method {@link LeaderElection#confirmLeadership(UUID, String)}. This will notify
32-
* the leader election service, that the contender has accepted the leadership specified and that
33-
* the leader session id as well as the leader address can now be published for leader retrieval
34-
* services.
31+
* by calling the method {@link LeaderElection#confirmLeadershipAsync(UUID, String)}. This will
32+
* notify the leader election service, that the contender has accepted the leadership specified and
33+
* that the leader session id as well as the leader address can now be published for leader
34+
* retrieval services.
3535
*/
3636
public interface LeaderElectionService {
3737

flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElection.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,12 +59,13 @@ public void startLeaderElection(LeaderContender contender) throws Exception {
5959
}
6060

6161
@Override
62-
public CompletableFuture<Void> confirmLeadership(UUID leaderSessionID, String leaderAddress) {
62+
public CompletableFuture<Void> confirmLeadershipAsync(
63+
UUID leaderSessionID, String leaderAddress) {
6364
return FutureUtils.completedVoidFuture();
6465
}
6566

6667
@Override
67-
public CompletableFuture<Boolean> hasLeadership(UUID leaderSessionId) {
68+
public CompletableFuture<Boolean> hasLeadershipAsync(UUID leaderSessionId) {
6869
synchronized (lock) {
6970
return CompletableFuture.completedFuture(
7071
this.leaderContender != null && this.sessionID.equals(leaderSessionId));

flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServiceImpl.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -266,7 +266,7 @@ private void startNewLeaderResourceManager(UUID newLeaderSessionID) throws Excep
266266
.thenAcceptAsync(
267267
(isStillLeader) -> {
268268
if (isStillLeader) {
269-
leaderElection.confirmLeadership(
269+
leaderElection.confirmLeadershipAsync(
270270
newLeaderSessionID, newLeaderResourceManager.getAddress());
271271
}
272272
},

flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1213,7 +1213,7 @@ public void grantLeadership(final UUID leaderSessionID) {
12131213
"{} was granted leadership with leaderSessionID={}",
12141214
getRestBaseUrl(),
12151215
leaderSessionID);
1216-
leaderElection.confirmLeadership(leaderSessionID, getRestBaseUrl());
1216+
leaderElection.confirmLeadershipAsync(leaderSessionID, getRestBaseUrl());
12171217
}
12181218

12191219
@Override

0 commit comments

Comments
 (0)