kfaraz commented on code in PR #16528:
URL: https://github.com/apache/druid/pull/16528#discussion_r1623237401


##########
server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidLeaderSelector.java:
##########
@@ -80,66 +77,62 @@ private LeaderLatch createNewLeaderLatch()
   private LeaderLatch createNewLeaderLatchWithListener()
   {
     final LeaderLatch newLeaderLatch = createNewLeaderLatch();
+    newLeaderLatch.addListener(new LeaderLatchListener()
+    {
+      @Override
+      public void isLeader()
+      {
+        try {
+          if (leader) {
+            log.warn("I'm being asked to become leader. But I am already the 
leader. Ignored event.");
+            return;
+          }
 
-    newLeaderLatch.addListener(
-        new LeaderLatchListener()
-        {
-          @Override
-          public void isLeader()
-          {
-            try {
-              if (leader) {
-                log.warn("I'm being asked to become leader. But I am already 
the leader. Ignored event.");
-                return;
-              }
-
-              leader = true;
-              term++;
-              listener.becomeLeader();
-            }
-            catch (Exception ex) {
-              log.makeAlert(ex, "listener becomeLeader() failed. Unable to 
become leader").emit();
-
-              // give others a chance to become leader.
-              CloseableUtils.closeAndSuppressExceptions(
-                  createNewLeaderLatchWithListener(),
-                  e -> log.warn("Could not close old leader latch; continuing 
with new one anyway.")
-              );
-
-              leader = false;
-              try {
-                //Small delay before starting the latch so that others waiting 
are chosen to become leader.
-                Thread.sleep(ThreadLocalRandom.current().nextInt(1000, 5000));
-                leaderLatch.get().start();
-              }
-              catch (Exception e) {
-                // If an exception gets thrown out here, then the node will 
zombie out 'cause it won't be looking for
-                // the latch anymore.  I don't believe it's actually possible 
for an Exception to throw out here, but
-                // Curator likes to have "throws Exception" on methods so it 
might happen...
-                log.makeAlert(e, "I am a zombie").emit();
-              }
-            }
+          leader = true;
+          term++;
+          listener.becomeLeader();
+        }
+        catch (Exception ex) {
+          log.makeAlert(ex, "listener becomeLeader() failed. Unable to become 
leader").emit();
+
+          // give others a chance to become leader.
+          CloseableUtils.closeAndSuppressExceptions(
+              createNewLeaderLatchWithListener(),
+              e -> log.warn("Could not close old leader latch; continuing with 
new one anyway.")
+          );
+
+          leader = false;
+          try {
+            // Small delay before starting the latch so that others waiting 
are chosen to become leader.
+            Thread.sleep(ThreadLocalRandom.current().nextInt(1000, 5000));
+            leaderLatch.get().start();
+          }
+          catch (Exception e) {
+            // If an exception gets thrown out here, then the node will zombie 
out 'cause it won't be looking for
+            // the latch anymore.  I don't believe it's actually possible for 
an Exception to throw out here, but
+            // Curator likes to have "throws Exception" on methods so it might 
happen...
+            log.makeAlert(e, "I am a zombie").emit();
           }
+        }
+      }
 
-          @Override
-          public void notLeader()
-          {
-            try {
-              if (!leader) {
-                log.warn("I'm being asked to stop being leader. But I am not 
the leader. Ignored event.");
-                return;
-              }
-
-              leader = false;
-              listener.stopBeingLeader();
-            }
-            catch (Exception ex) {
-              log.makeAlert(ex, "listener.stopBeingLeader() failed. Unable to 
stopBeingLeader").emit();
-            }
+      @Override
+      public void notLeader()
+      {
+        try {
+          if (!leader) {

Review Comment:
   I double checked the `LeaderLatch` code. As expected, it already has a 
`ConnectionStateListener` which calls `notLeader` if connection is lost or gets 
suspended.
   
   
https://github.com/apache/curator/blob/5af54047d3a1d85a8c08fae097b7bc1329f2a55c/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java#L638-L648



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to