zentol commented on code in PR #25679:
URL: https://github.com/apache/flink/pull/25679#discussion_r1865559170


##########
flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElection.java:
##########
@@ -39,17 +44,37 @@ public interface LeaderElection extends AutoCloseable {
      *
      * @param leaderSessionID The new leader session ID
      * @param leaderAddress The address of the new leader
+     * @return A future that completes successfully if the confirmation 
succeeded or fails
+     *     exceptionally with a {@link LeadershipLostException} if the 
leadership was revoked in the
+     *     meantime.
+     */
+    default CompletableFuture<Void> confirmLeadershipAsLeader(
+            UUID leaderSessionID, String leaderAddress) {
+        checkNotNull(leaderSessionID);
+        checkNotNull(leaderAddress);
+
+        return runAsLeader(
+                leaderSessionID, () -> confirmLeadership(leaderSessionID, 
leaderAddress));
+    }
+
+    /**
+     * Runs the actual leadership confirmation. This method should only be 
called if the leadership
+     * is acquired. Otherwise, a {@link IllegalStateException} should be 
thrown.
      */
     void confirmLeadership(UUID leaderSessionID, String leaderAddress);
 
     /**
-     * Returns {@code true} if the service's {@link LeaderContender} has the 
leadership under the
-     * given leader session ID acquired.
+     * Runs the given {@code callback} on the leader election's main thread 
only if the associated
+     * {@code leaderSessionId} is still valid.
      *
-     * @param leaderSessionId identifying the current leader
-     * @return true if the associated {@link LeaderContender} is the leader, 
otherwise false
+     * @param leaderSessionId The session ID that's associated with the given 
{@code callback}.
+     * @param callback The callback that shall be executed as a leader.
+     * @return The future referring to the result of the callback operation. 
This future would
+     *     complete exceptionally with a {@link LeadershipLostException} if 
the leadership wasn't
+     *     active anymore.
      */
-    boolean hasLeadership(UUID leaderSessionId);
+    CompletableFuture<Void> runAsLeader(

Review Comment:
   > But the chained callback that is called after the async call completes can 
run concurrently to a leadership revocation again
   
   Why not? It is a matter of fact that we will occasionally do stuff while we 
no longer have leadership. Some S3 delete call can be on the wire while we have 
lost leadership. All the components like DIspatchers/JobManagers/etc can do 
stuff stuff while the leadership revocation message sits in the akka message 
queue waiting to be scheduled into the main thread.
   
   The _only_ place where leadership must be truly maintained during the 
operation is access to HA itself, which uses different primitives. Everything 
else is best-effort.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to