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


##########
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElector.java:
##########
@@ -86,12 +117,38 @@ public KubernetesLeaderElector(
                                                         newLeader,
                                                         
leaderConfig.getConfigMapName())))
                         .build();
+        this.executorService = executorService;
+
+        LOG.info(
+                "Create KubernetesLeaderElector on lock {}.",
+                leaderElectionConfig.getLock().describe());
+    }
+
+    @GuardedBy("lock")
+    private void resetInternalLeaderElector() {
+        stopLeaderElectionCycle();
+
         internalLeaderElector =
                 new LeaderElector(kubernetesClient, leaderElectionConfig, 
executorService);
+        currentLeaderElectionSession = internalLeaderElector.start();
+
         LOG.info(
-                "Create KubernetesLeaderElector {} with lock identity {}.",
-                leaderConfig.getConfigMapName(),
-                leaderConfig.getLockIdentity());
+                "Triggered leader election on lock {}.", 
leaderElectionConfig.getLock().describe());
+    }
+
+    @GuardedBy("lock")
+    private void stopLeaderElectionCycle() {
+        if (internalLeaderElector != null) {
+            Preconditions.checkNotNull(currentLeaderElectionSession);
+
+            // the current leader election cycle needs to be cancelled before 
releasing the lock to
+            // avoid retrying
+            currentLeaderElectionSession.cancel(true);
+            currentLeaderElectionSession = null;
+
+            internalLeaderElector.release();

Review Comment:
   I verified with the changed test that your finding is correct (the ITCase 
succeeds with the change below and my previous version of the PR but fails with 
the version where the explicit release call is removed):
   ```diff
   diff --git 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
   index 51631dc54f6..0bca6c4d641 100644
   --- 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
   +++ 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java
   @@ -182,6 +182,9 @@ class KubernetesLeaderElectorITCase {
                // revoking the leadership initiates another leadership 
lifecycle
                testInstance.run();
    
   +            // leadership should be lost eventually due to the renewal loop 
being stopped
   +            leadershipCallbackHandler.waitForRevokeLeader();
   +
                // triggers acquiring the leadership again
                executorService.trigger(waitForNextTaskForever);
   ```
   
   I didn't address this in a test because it's hard to come up with a 
condition that checks the non-existence of an event (either we would have to 
add a timeout which might make the test unstable or the test would take longer 
than necessary). WDYT?



-- 
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