kfaraz commented on PR #16528:
URL: https://github.com/apache/druid/pull/16528#issuecomment-2142723498

   > One important thing that I have noticed is that the isLeader() method may 
be called on the leader latch listener even after the latch has been closed. So 
we would need to make sure that if this method is called on a closed latch, we 
just ignore that event.
   
   For more information, I encountered this in my local testing when I was 
trying to recreate the latch in `notLeader()`.
   
   Timeline:
   -[curator-thread] Leader loses connection
   - Leader gets notified with `notLeader()`
   -[curator-thread] Connection is re-established
   - Create new latch, close old one. This immediately causes some other node 
to become leader.
   - Start new latch after a delay to allow other nodes to become leader.
   - `isLeader()` is called on old latch which is already closed. (This leads 
to double leaders if not properly handled.)
   
   An example sequence of events that I observed
   ```
   2024-05-30T14:19:11,556 WARN [main-SendThread(localhost:2181)] 
org.apache.zookeeper.ClientCnxn - Session 0x1002399b0fc0000 for server 
localhost/127.0.0.1:2181, Closing socket connection. Attempting reconnect 
except it is a SessionExpiredException.
   2024-05-30T14:19:11,670 INFO [LeaderSelector[/druid/overlord/_OVERLORD]] 
org.apache.druid.curator.discovery.CuratorDruidLeaderSelector - 
[http://localhost:8081][1] Giving up leadership
   2024-05-30T14:19:11,754 INFO [main-SendThread(localhost:2183)] 
org.apache.zookeeper.ClientCnxn - Session establishment complete on server 
localhost/0:0:0:0:0:0:0:1:2183, session id = 0x1002399b0fc0000, negotiated 
timeout = 30000
   2024-05-30T14:19:11,766 INFO [LeaderSelector[/druid/overlord/_OVERLORD]] 
org.apache.druid.curator.discovery.CuratorDruidLeaderSelector - 
[http://localhost:8081][1] Recreating leader latch to allow other nodes to 
become leader.
   2024-05-30T14:19:13,379 INFO [LeaderSelector[/druid/overlord/_OVERLORD]] 
org.apache.druid.curator.discovery.CuratorDruidLeaderSelector - 
[http://localhost:8081][1] Now starting the latch
   2024-05-30T14:19:13,379 INFO [LeaderSelector[/druid/overlord/_OVERLORD]] 
org.apache.druid.curator.discovery.CuratorDruidLeaderSelector - 
[http://localhost:8081][1] I am now the leader. Latch state[CLOSED]
   ```


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