From 844b2177dee27a7d96d45a92ef344da009c43f52 Mon Sep 17 00:00:00 2001 From: razinbouzar Date: Fri, 7 Jun 2024 05:37:30 -0400 Subject: [PATCH] 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 --- .../discovery/CuratorDruidLeaderSelector.java | 47 ++++++++++++------- 1 file changed, 29 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidLeaderSelector.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidLeaderSelector.java index 1aa77cec5fd1..b2179f5d69d4 100644 --- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidLeaderSelector.java +++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidLeaderSelector.java @@ -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; @@ -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(); } } @@ -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(); @@ -215,4 +204,26 @@ public void unregisterListener() CloseableUtils.closeAndSuppressExceptions(leaderLatch.get(), e -> log.warn(e, "Failed to close LeaderLatch.")); 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(); + } + } }