Skip to content

recipes: Leader election#60

Open
rodio wants to merge 4 commits intostackabletech:mainfrom
rodio:leader-election
Open

recipes: Leader election#60
rodio wants to merge 4 commits intostackabletech:mainfrom
rodio:leader-election

Conversation

@rodio
Copy link
Copy Markdown

@rodio rodio commented Apr 23, 2026

I am implementing the leader election recipe from #10 and would appreciate early feedback to see if it makes sense.

I thought that a nice API for leader would be similar to watches: the leader election function would return a receiving end of a oneshot channel so that users could use it to check if a node is a leader. There would no way to "withdraw" yourself from leader election process once you've volunteered to be a leader because I think that no one needs it in practice.

So now that I have a somewhat working draft, I'd appreciate if someone could give feedback on this API and function signatures or really anything. Eventually I would like to resolve all these TODOs hide this recipe behind a feture flag.

To test it:

  • run a ZooKeeper instance on localhost:2181
  • create a node ./bin/zkCli.sh -server localhost:2181 create /election ""
  • run the test with cargo test election_works -- --nocapture
  • or run multiple copies of this:
use core::time;
use std::{io, time::Duration};

use tokio_zookeeper::{recipes::LeaderElection, *};

fn init_tracing_subscriber() {
    let _ = tracing_subscriber::fmt()
        .with_env_filter(tracing_subscriber::EnvFilter::from_default_env())
        .init();
}

#[tokio::main]
async fn main() {
    init_tracing_subscriber();
    let binding = "127.0.0.1:2181".parse().unwrap();
    let mut zk_builder = ZooKeeperBuilder::default();
    zk_builder.set_timeout(time::Duration::from_millis(10_000));
    let (zk, _default_watcher) = ZooKeeper::connect(&binding).await.unwrap();

    let leader_election = LeaderElection::new(zk.clone(), "/election");
    let mut leader_receiver = leader_election.volunteer().await.unwrap();

    loop {
        match leader_receiver.try_recv() {
            Ok(Some(_)) => {
                println!("I am the leader");
                println!("doing leader stuff...");
                _ = dbg!(zk.exists("/leader").await);
                _ = io::stdin().read_line(&mut String::new());
                println!("good bye!");
                break;
            }
            Ok(None) => {
                println!("I am a follower");
                println!("doing follower stuff...");
                _ = dbg!(zk.exists("/follower").await);
                tokio::time::sleep(Duration::from_secs(1)).await;
                println!("now let me check if I am the leader yet...");
            }
            Err(_) => unreachable!("closed channel"),
        }
    }
}

rodio added 4 commits April 21, 2026 18:58
Signed-off-by: Rodion Borovyk <rodion.borovyk@gmail.com>
Signed-off-by: Rodion Borovyk <rodion.borovyk@gmail.com>
Signed-off-by: Rodion Borovyk <rodion.borovyk@gmail.com>
Signed-off-by: Rodion Borovyk <rodion.borovyk@gmail.com>
@stackable-cla
Copy link
Copy Markdown

stackable-cla Bot commented Apr 23, 2026

CLA assistant check
All committers have signed the CLA.

@rodio rodio marked this pull request as ready for review April 25, 2026 10:42
@rodio
Copy link
Copy Markdown
Author

rodio commented Apr 25, 2026

I've marked it as ready for review because apparently no one receives notifications about drafts?

@maltesander
Copy link
Copy Markdown
Member

Hi @rodio ,

thank you for the contribution. I had a look, i think the general approach looks good.

I see two problems (not sure if Draft related):

  • The oneshot is wrong and cannot model all cases from https://zookeeper.apache.org/doc/current/recipes.html. Leadership is an ongoing state, not a one-shot event. Needed for session-loss semantics and the acknowledgment-ZNode pattern. I would go with some sort of watch::<LeaderState> for that.
  • Furthermore i would split the LeaderElection like:
use tokio::sync::watch;

#[derive(Debug, Clone, PartialEq, Eq)]
pub enum LeaderState {
	// volunteered, watching predecessor
    Pending,
    // at index 0
    Leader,
    // session/candidacy ended; terminal
    Resigned,
}

/// Configuration for participating in a leader election.
#[derive(Debug, Clone)]
pub struct LeaderElection {
    election_node: String,
    zk: ZooKeeper,
}

/// An active candidacy. Drop to stop observing (the ephemeral znode
/// is still only removed when the underlying session ends).
#[derive(Debug)]
pub struct Candidate {
    my_path: String,
    state: watch::<LeaderState>,
    ...
}

impl Candidate {
    pub fn path(&self) -> &str { &self.my_path }
    pub fn state(&self) -> LeaderState { self.state.borrow().clone() }

    /// Resolves once we become leader, or returns `Err` if we lose
    /// the session before that happens.
    pub async fn wait_for_leadership(&mut self) -> Result<(), ElectionError> {
        loop {
            match *self.state.borrow_and_update() {
                LeaderState::Leader => return Ok(()),
                // Should probably even be split into something like `SessionLost` and `Withdrawn`
                LeaderState::Resigned   => return Err(ElectionError::SessionLost),
                LeaderState::Pending => {}
            }
            self.state.changed().await.map_err(|_| ElectionError::SessionLost)?;
        }
    }
}

With a volunteer impl like:

pub async fn volunteer(&self) -> Result<Candidate, ElectionError> {...}

So the candidate only exists after you volunteered and my_path is never an Option.

This will help with a couple of your TODOs:

  • The TOCTOU fix is just if stat.is_none() { continue; } (would block otherwise)
  • TODO could be ActiveLeaderElection?

Some other things:

  • proper GUID handling
  • sort children by sequence number suffix, not full string (If GUID differ, largest j < i)
  • "cant find myself" -> probe own path, retry on lag and transition to terminal if ephermeral is gone.

Malte

@maltesander maltesander self-requested a review April 26, 2026 16:32
@rodio
Copy link
Copy Markdown
Author

rodio commented Apr 27, 2026

Hi, @maltesander! Thank you for the thorough review!

I'd like to argue a bit here and defend the oneshot approach.

The "Resigned" state that you mentioned in your code snippet is not a valid state at all and should not be exposed to users. You have the reason in your comment there: Drop to stop observing (the ephemeral znode is still only removed when the underlying session ends). So resigning from the leader election process without dropping the connection would leave other nodes with an "impression" that you're still participating. When your node becomes the node with the lowest ID everyone would just be sitting there assuming you're the leader, but you don't care. You've resigned but but no one knows about it. So it puts the whole system into an invalid state where it is permanently with a leader that does not perform its "duties".

In fact, if users really want to stop participating they should drop their connection like I did in the test. The connection and the participation should be always be coupled and it should not be possible to drop one without dropping the other. This way there'll be no "orphan" ephemeral nodes. And the way to do it is the one-shot approach: you either are the leader, or wait for the leadership or drop the whole connection.

@maltesander
Copy link
Copy Markdown
Member

Hi @rodio,

yeah, i agree that "Resigned" is wrong and we shouldnt expose "Withdrawal".

Id still argue about involuntary withdrawal? E.g. Session loss (which the candidate did not choose)?

Consider: A candidate becomes leader, fires Leader on the oneshot. Application starts doing leader work. A little later, the ZK session expires (network partition, GC pause, ZK server failover, whatever). The ephemeral znode is gone. Some other node is now the leader. The original application is still doing leader work, because nobody told it otherwise. The oneshot already fired and can't fire again.

Without withdrawal, if we go with the oneshot we should document explicitly that we have to watch connect() for session expiry. Then it should treat itself as no longer Leader, regardless of the oneshot?

Im fine with both approaches.

Malte

@rodio
Copy link
Copy Markdown
Author

rodio commented Apr 27, 2026

I think you're right that we need to monitor session loss after and I think it is better to hide it inside the implementation of the recipe because it probably is something that almost everyone needs. I'll think more about it later and we'll see what I can come up with. Thanks!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants