gianm commented on code in PR #17927:
URL: https://github.com/apache/druid/pull/17927#discussion_r2047812862
##########
server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidLeaderSelector.java:
##########
@@ -208,28 +209,26 @@ public void unregisterListener()
listenerExecutor.shutdownNow();
}
- private void stopAndCreateNewLeaderLatch()
+ private void closeLeaderLatchQuietly()
Review Comment:
This still does close and re-create the latch, how about naming it
`closeAndRecreateLeaderLatchQuietly`? The name `closeLeaderLatchQuietly` loses
the sense that it's also creating a new latch.
##########
server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidLeaderSelector.java:
##########
@@ -105,29 +105,30 @@ public void isLeader()
}
catch (Exception ex) {
log.makeAlert(ex, "listener becomeLeader() failed. Unable to
become leader").emit();
- stopAndCreateNewLeaderLatch();
- startLeaderLatch();
+ notLeader();
}
}
@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;
- }
+ if (!leader) {
+ log.warn("I'm being asked to stop being leader. But I am not the
leader. Ignored event.");
+ return;
+ }
+
+ // Stop the current latch and create a new one
+ leader = false;
+ closeLeaderLatchQuietly();
- leader = false;
- // give others a chance to become leader.
- stopAndCreateNewLeaderLatch();
+ try {
listener.stopBeingLeader();
- startLeaderLatch();
}
catch (Exception ex) {
Review Comment:
catch `Throwable`
##########
server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidLeaderSelector.java:
##########
@@ -105,29 +105,30 @@ public void isLeader()
}
catch (Exception ex) {
log.makeAlert(ex, "listener becomeLeader() failed. Unable to
become leader").emit();
- stopAndCreateNewLeaderLatch();
- startLeaderLatch();
+ notLeader();
}
}
@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;
- }
+ if (!leader) {
+ log.warn("I'm being asked to stop being leader. But I am not the
leader. Ignored event.");
+ return;
+ }
+
+ // Stop the current latch and create a new one
+ leader = false;
+ closeLeaderLatchQuietly();
- leader = false;
- // give others a chance to become leader.
- stopAndCreateNewLeaderLatch();
+ try {
listener.stopBeingLeader();
- startLeaderLatch();
}
catch (Exception ex) {
log.makeAlert(ex, "listener.stopBeingLeader() failed. Unable to
stopBeingLeader").emit();
Review Comment:
When this happens, the OL is left in an inconsistent, partially-leading
state and nothing will again ever fix it. It seems dangerous to leave the OL in
this state. Perhaps we should `System.exit(1)` in this case so we can start
back up and try to get back into a normal state. I notice the
`K8sDruidLeaderSelector` already does this.
--
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]