XComp commented on code in PR #21742:
URL: https://github.com/apache/flink/pull/21742#discussion_r1098595984


##########
flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionService.java:
##########
@@ -100,6 +100,7 @@ public final void stop() throws Exception {
             if (!running) {
                 return;
             }
+            leaderContender.revokeLeadership();

Review Comment:
   From what I've seen within the code, any implementation does essentially 
call redundant code cleaning up artifacts in the classes corresponding close 
implementation and its `revokeLeadership()`.
   
   * `DefaultDispatcherRunner`: `revokeLeadership()` 
([DefaultDispatcherRunner:189ff](https://github.com/apache/flink/blob/2d1d61c68d81eba0f721b10eaf2a7246aa0814c2/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java#L189))
 only calls `stopDispatcherLeaderProcess` which is also called as part of the 
`DefaultDispatcherRunner.closeAsync` call in 
[DefaultDispatcherRunner:96](https://github.com/apache/flink/blob/2d1d61c68d81eba0f721b10eaf2a7246aa0814c2/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java#L96).
   * `WebMonitorEndpoint` doesn't do anything except for logging the leadership 
loss 
([WebMonitorEndpoint:1101](https://github.com/apache/flink/blob/be240dea803fc82299c4711ebe656c1c4b159ca9/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java#L1101)
   * `ResourceManagerServiceImpl`: `revokeLeadership()` 
([ResourceManagerServiceImpl:214ff](https://github.com/apache/flink/blob/26aa543b3bbe2b606bbc6d332a2ef7c5b46d25eb/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServiceImpl.java#L214))
 calls `stopLeaderResourceManager()` besides some logging which is also called 
by 
[ResourceManagerServiceImpl.closeAsync](https://github.com/apache/flink/blob/26aa543b3bbe2b606bbc6d332a2ef7c5b46d25eb/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServiceImpl.java#L170)
   * `JobMasterServiceLeadershipRunner`: `revokeLeadership` 
([JobMasterServiceLeadershipRunner:381ff](https://github.com/apache/flink/blob/d745f5b3f7a64445854c735668afa9b72edb3fee/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java#L381))
 calls `stopJobMasterServiceProcessAsync` which calls `stopJobMasterService` 
(see 
[JobMasterServiceLeadershipRunner:388ff](https://github.com/apache/flink/blob/d745f5b3f7a64445854c735668afa9b72edb3fee/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java#L401))
 which completes the `jobMasterGatewayFuture` exceptionally and calls 
`closeAsync` on `jobMasterServiceProcess`. The same things are called in 
`JobMasterServiceLeadershipRunner.closeAsync` (see 
[JobMasterServiceLeadershipRunner:126ff](https://github.com/apache/flink/blob/d745f5b3f7a64445854c735668afa9b72edb3fee/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipR
 unner.java#L126)).
   
   That keeps me thinking that there's actually a clearer process possible from 
a logical standpoint: A `LeaderContender` needs to call `revokeLeadership` 
before continuing with its shutdown.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to