mirror of https://github.com/apache/druid.git
Fix 2 coordinators elected as leader (#16528)
Changes: - Recreate the leader latch when connection to zookeeper is lost - Do not become leader if leader latch is already closed
This commit is contained in:
parent
03a38be446
commit
844b2177de
|
@ -88,6 +88,11 @@ public class CuratorDruidLeaderSelector implements DruidLeaderSelector
|
||||||
public void isLeader()
|
public void isLeader()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
if (newLeaderLatch.getState().equals(LeaderLatch.State.CLOSED)) {
|
||||||
|
log.warn("I'm being asked to become leader, but the latch is CLOSED. Ignored event.");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
if (leader) {
|
if (leader) {
|
||||||
log.warn("I'm being asked to become leader. But I am already the leader. Ignored event.");
|
log.warn("I'm being asked to become leader. But I am already the leader. Ignored event.");
|
||||||
return;
|
return;
|
||||||
|
@ -100,24 +105,7 @@ public class CuratorDruidLeaderSelector implements DruidLeaderSelector
|
||||||
catch (Exception ex) {
|
catch (Exception ex) {
|
||||||
log.makeAlert(ex, "listener becomeLeader() failed. Unable to become leader").emit();
|
log.makeAlert(ex, "listener becomeLeader() failed. Unable to become leader").emit();
|
||||||
|
|
||||||
// give others a chance to become leader.
|
recreateLeaderLatch();
|
||||||
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();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -132,6 +120,7 @@ public class CuratorDruidLeaderSelector implements DruidLeaderSelector
|
||||||
|
|
||||||
leader = false;
|
leader = false;
|
||||||
listener.stopBeingLeader();
|
listener.stopBeingLeader();
|
||||||
|
recreateLeaderLatch();
|
||||||
}
|
}
|
||||||
catch (Exception ex) {
|
catch (Exception ex) {
|
||||||
log.makeAlert(ex, "listener.stopBeingLeader() failed. Unable to stopBeingLeader").emit();
|
log.makeAlert(ex, "listener.stopBeingLeader() failed. Unable to stopBeingLeader").emit();
|
||||||
|
@ -215,4 +204,26 @@ public class CuratorDruidLeaderSelector implements DruidLeaderSelector
|
||||||
CloseableUtils.closeAndSuppressExceptions(leaderLatch.get(), e -> log.warn(e, "Failed to close LeaderLatch."));
|
CloseableUtils.closeAndSuppressExceptions(leaderLatch.get(), e -> log.warn(e, "Failed to close LeaderLatch."));
|
||||||
listenerExecutor.shutdownNow();
|
listenerExecutor.shutdownNow();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void recreateLeaderLatch()
|
||||||
|
{
|
||||||
|
// 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();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue