mirror of https://github.com/apache/druid.git
Separate stop/start logic for LeaderLatch (#17546)
This commit is contained in:
parent
f61ec0af85
commit
7736228f37
|
@ -105,8 +105,8 @@ public class CuratorDruidLeaderSelector implements DruidLeaderSelector
|
|||
}
|
||||
catch (Exception ex) {
|
||||
log.makeAlert(ex, "listener becomeLeader() failed. Unable to become leader").emit();
|
||||
|
||||
recreateLeaderLatch();
|
||||
stopAndCreateNewLeaderLatch();
|
||||
startLeaderLatch();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -120,8 +120,10 @@ public class CuratorDruidLeaderSelector implements DruidLeaderSelector
|
|||
}
|
||||
|
||||
leader = false;
|
||||
// give others a chance to become leader.
|
||||
stopAndCreateNewLeaderLatch();
|
||||
listener.stopBeingLeader();
|
||||
recreateLeaderLatch();
|
||||
startLeaderLatch();
|
||||
}
|
||||
catch (Exception ex) {
|
||||
log.makeAlert(ex, "listener.stopBeingLeader() failed. Unable to stopBeingLeader").emit();
|
||||
|
@ -206,15 +208,18 @@ public class CuratorDruidLeaderSelector implements DruidLeaderSelector
|
|||
listenerExecutor.shutdownNow();
|
||||
}
|
||||
|
||||
private void recreateLeaderLatch()
|
||||
private void stopAndCreateNewLeaderLatch()
|
||||
{
|
||||
// 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;
|
||||
}
|
||||
|
||||
private void startLeaderLatch()
|
||||
{
|
||||
try {
|
||||
//Small delay before starting the latch so that others waiting are chosen to become leader.
|
||||
Thread.sleep(ThreadLocalRandom.current().nextInt(1000, 5000));
|
||||
|
|
Loading…
Reference in New Issue