Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,11 @@ private LeaderLatch createNewLeaderLatchWithListener()
public void isLeader()
{
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) {
log.warn("I'm being asked to become leader. But I am already the leader. Ignored event.");
return;
Expand All @@ -100,24 +105,7 @@ public void isLeader()
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();
}
recreateLeaderLatch();
}
}

Expand All @@ -132,6 +120,7 @@ public void notLeader()

leader = false;
listener.stopBeingLeader();
recreateLeaderLatch();
}
catch (Exception ex) {
log.makeAlert(ex, "listener.stopBeingLeader() failed. Unable to stopBeingLeader").emit();
Expand Down Expand Up @@ -215,4 +204,26 @@ public void unregisterListener()
CloseableUtils.closeAndSuppressExceptions(leaderLatch.get(), e -> log.warn(e, "Failed to close LeaderLatch."));
listenerExecutor.shutdownNow();
}

private void recreateLeaderLatch()
Comment thread
kfaraz marked this conversation as resolved.
{
// 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();
}
}
}