nodedb-cluster 0.0.0-beta.5

Distributed coordination layer for NodeDB — vShards, QUIC transport, and replication
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
//! Raft event loop — drives MultiRaft ticks and dispatches messages over the transport.
//!
//! This module glues [`MultiRaft`] to [`NexarTransport`]:
//! - Periodic ticks drive elections and heartbeats
//! - `Ready` output is dispatched over QUIC (messages, vote requests)
//! - Incoming RPCs are routed to the correct Raft group
//! - Committed entries are applied via a [`CommitApplier`] callback
//! - RPC responses are fed back asynchronously (non-blocking tick loop)

use std::pin::Pin;
use std::sync::{Arc, Mutex, RwLock};
use std::time::Duration;

use tracing::{debug, warn};

use nodedb_raft::message::LogEntry;
use nodedb_raft::transport::RaftTransport;

use crate::conf_change::ConfChange;
use crate::error::{ClusterError, Result};
use crate::forward::RequestForwarder;
use crate::health;
use crate::multi_raft::MultiRaft;
use crate::rpc_codec::RaftRpc;
use crate::topology::ClusterTopology;
use crate::transport::{NexarTransport, RaftRpcHandler};

/// Default tick interval (10ms — fast enough for sub-second elections).
///
/// Matches `ClusterTransportTuning::raft_tick_interval_ms` default. Override
/// at startup by calling `.with_tick_interval()` on the `RaftLoop`, passing
/// `Duration::from_millis(config.tuning.cluster_transport.raft_tick_interval_ms)`.
const DEFAULT_TICK_INTERVAL: Duration = Duration::from_millis(10);

/// Callback for applying committed Raft log entries to the state machine.
///
/// Called synchronously during the tick loop. Implementations should be fast
/// (enqueue to SPSC, not perform I/O directly).
pub trait CommitApplier: Send + Sync + 'static {
    /// Apply committed entries for a Raft group.
    ///
    /// Returns the index of the last successfully applied entry.
    fn apply_committed(&self, group_id: u64, entries: &[LogEntry]) -> u64;
}

/// Type-erased async handler for incoming `VShardEnvelope` messages.
///
/// Receives raw envelope bytes, returns response bytes. Set by the main binary
/// to dispatch to the appropriate engine handler (Event Plane, timeseries, etc.).
pub type VShardEnvelopeHandler = Arc<
    dyn Fn(Vec<u8>) -> Pin<Box<dyn std::future::Future<Output = Result<Vec<u8>>> + Send>>
        + Send
        + Sync,
>;

/// Raft event loop coordinator.
///
/// Owns the MultiRaft state (behind `Arc<Mutex>`) and drives it via periodic
/// ticks. Implements [`RaftRpcHandler`] so it can be passed directly to
/// [`NexarTransport::serve`] for incoming RPC dispatch.
pub struct RaftLoop<A: CommitApplier, F: RequestForwarder = crate::forward::NoopForwarder> {
    node_id: u64,
    multi_raft: Arc<Mutex<MultiRaft>>,
    transport: Arc<NexarTransport>,
    topology: Arc<RwLock<ClusterTopology>>,
    applier: A,
    forwarder: Arc<F>,
    tick_interval: Duration,
    /// Optional handler for incoming VShardEnvelope messages.
    /// Set when the Event Plane or other subsystems need cross-node messaging.
    vshard_handler: Option<VShardEnvelopeHandler>,
}

impl<A: CommitApplier> RaftLoop<A> {
    pub fn new(
        multi_raft: MultiRaft,
        transport: Arc<NexarTransport>,
        topology: Arc<RwLock<ClusterTopology>>,
        applier: A,
    ) -> Self {
        let node_id = multi_raft.node_id();
        Self {
            node_id,
            multi_raft: Arc::new(Mutex::new(multi_raft)),
            transport,
            topology,
            applier,
            forwarder: Arc::new(crate::forward::NoopForwarder),
            tick_interval: DEFAULT_TICK_INTERVAL,
            vshard_handler: None,
        }
    }
}

impl<A: CommitApplier, F: RequestForwarder> RaftLoop<A, F> {
    /// Create a RaftLoop with a custom request forwarder (for cluster mode).
    pub fn with_forwarder(
        multi_raft: MultiRaft,
        transport: Arc<NexarTransport>,
        topology: Arc<RwLock<ClusterTopology>>,
        applier: A,
        forwarder: Arc<F>,
    ) -> Self {
        let node_id = multi_raft.node_id();
        Self {
            node_id,
            multi_raft: Arc::new(Mutex::new(multi_raft)),
            transport,
            topology,
            applier,
            forwarder,
            tick_interval: DEFAULT_TICK_INTERVAL,
            vshard_handler: None,
        }
    }

    /// Set a handler for incoming VShardEnvelope messages.
    pub fn with_vshard_handler(mut self, handler: VShardEnvelopeHandler) -> Self {
        self.vshard_handler = Some(handler);
        self
    }

    pub fn with_tick_interval(mut self, interval: Duration) -> Self {
        self.tick_interval = interval;
        self
    }

    /// Run the event loop until shutdown.
    ///
    /// This drives Raft elections, heartbeats, and message dispatch.
    /// Call [`NexarTransport::serve`] separately with `Arc<Self>` as the handler.
    pub async fn run(&self, mut shutdown: tokio::sync::watch::Receiver<bool>) {
        let mut interval = tokio::time::interval(self.tick_interval);
        interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay);

        loop {
            tokio::select! {
                _ = interval.tick() => {
                    self.do_tick();
                }
                _ = shutdown.changed() => {
                    if *shutdown.borrow() {
                        debug!("raft loop shutting down");
                        break;
                    }
                }
            }
        }
    }

    /// Execute a single tick: drive Raft, dispatch outbound messages, apply commits.
    fn do_tick(&self) {
        // Phase 1: tick under lock, extract Ready.
        let ready = {
            let mut mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
            mr.tick()
        };

        if ready.is_empty() {
            return;
        }

        // Phase 2: Batch messages by peer for efficient dispatch.
        // Instead of spawning one task per (group, peer) message, we batch all
        // messages to the same peer into one task, reducing QUIC stream overhead.
        use std::collections::HashMap as BatchMap;

        let mut ae_batches: BatchMap<u64, Vec<(u64, nodedb_raft::AppendEntriesRequest)>> =
            BatchMap::new();
        let mut vote_batches: BatchMap<u64, Vec<(u64, nodedb_raft::RequestVoteRequest)>> =
            BatchMap::new();

        for (group_id, group_ready) in &ready.groups {
            for (peer, req) in &group_ready.messages {
                ae_batches
                    .entry(*peer)
                    .or_default()
                    .push((*group_id, req.clone()));
            }
            for (peer, req) in &group_ready.vote_requests {
                vote_batches
                    .entry(*peer)
                    .or_default()
                    .push((*group_id, req.clone()));
            }
        }

        // Dispatch batched AppendEntries — one task per peer.
        for (peer, messages) in ae_batches {
            let transport = self.transport.clone();
            let mr = self.multi_raft.clone();
            tokio::spawn(async move {
                for (group_id, req) in messages {
                    match transport.append_entries(peer, req).await {
                        Ok(resp) => {
                            let mut mr = mr.lock().unwrap_or_else(|p| p.into_inner());
                            if let Err(e) = mr.handle_append_entries_response(group_id, peer, &resp)
                            {
                                debug!(group_id, peer, error = %e, "handle ae response");
                            }
                        }
                        Err(e) => {
                            warn!(group_id, peer, error = %e, "append_entries RPC failed");
                            break; // Peer is down — skip remaining groups.
                        }
                    }
                }
            });
        }

        // Dispatch batched RequestVote — one task per peer.
        for (peer, votes) in vote_batches {
            let transport = self.transport.clone();
            let mr = self.multi_raft.clone();
            tokio::spawn(async move {
                for (group_id, req) in votes {
                    match transport.request_vote(peer, req).await {
                        Ok(resp) => {
                            let mut mr = mr.lock().unwrap_or_else(|p| p.into_inner());
                            if let Err(e) = mr.handle_request_vote_response(group_id, peer, &resp) {
                                debug!(group_id, peer, error = %e, "handle vote response");
                            }
                        }
                        Err(e) => {
                            warn!(group_id, peer, error = %e, "request_vote RPC failed");
                            break;
                        }
                    }
                }
            });
        }

        for (group_id, group_ready) in ready.groups {
            // Apply committed entries synchronously.
            if !group_ready.committed_entries.is_empty() {
                // First, detect and apply any ConfChange entries to MultiRaft.
                for entry in &group_ready.committed_entries {
                    if let Some(cc) = ConfChange::from_entry_data(&entry.data) {
                        let mut mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
                        if let Err(e) = mr.apply_conf_change(group_id, &cc) {
                            warn!(group_id, error = %e, "failed to apply conf change");
                        }
                    }
                }

                // Then pass all entries (including ConfChanges) to the state machine.
                let last_applied = self
                    .applier
                    .apply_committed(group_id, &group_ready.committed_entries);
                if last_applied > 0 {
                    let mut mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
                    if let Err(e) = mr.advance_applied(group_id, last_applied) {
                        warn!(group_id, error = %e, "failed to advance applied index");
                    }
                }
            }

            // Send InstallSnapshot RPCs to peers that are too far behind.
            if !group_ready.snapshots_needed.is_empty() {
                // Get snapshot metadata under lock.
                let snapshot_meta = {
                    let mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
                    mr.snapshot_metadata(group_id).ok()
                };

                if let Some((term, snap_index, snap_term)) = snapshot_meta {
                    for peer in group_ready.snapshots_needed {
                        let transport = self.transport.clone();
                        let mr = self.multi_raft.clone();
                        let req = nodedb_raft::InstallSnapshotRequest {
                            term,
                            leader_id: self.node_id,
                            last_included_index: snap_index,
                            last_included_term: snap_term,
                            offset: 0,
                            data: vec![], // Metadata-only snapshot (state transfer is §5).
                            done: true,
                            group_id,
                        };
                        tokio::spawn(async move {
                            match transport.install_snapshot(peer, req).await {
                                Ok(resp) => {
                                    if resp.term > term {
                                        let mut mr = mr.lock().unwrap_or_else(|p| p.into_inner());
                                        // Higher term — let the tick loop handle step-down.
                                        let _ = mr.handle_append_entries_response(
                                            group_id,
                                            peer,
                                            &nodedb_raft::AppendEntriesResponse {
                                                term: resp.term,
                                                success: false,
                                                last_log_index: 0,
                                            },
                                        );
                                    }
                                    debug!(group_id, peer, "install_snapshot sent");
                                }
                                Err(e) => {
                                    warn!(
                                        group_id, peer, error = %e,
                                        "install_snapshot RPC failed"
                                    );
                                }
                            }
                        });
                    }
                }
            }
        }
    }

    /// Propose a command to the Raft group owning the given vShard.
    ///
    /// Returns `(group_id, log_index)` on success.
    pub fn propose(&self, vshard_id: u16, data: Vec<u8>) -> Result<(u64, u64)> {
        let mut mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
        mr.propose(vshard_id, data)
    }

    /// Snapshot all Raft group states for observability (SHOW RAFT GROUPS).
    pub fn group_statuses(&self) -> Vec<crate::multi_raft::GroupStatus> {
        let mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
        mr.group_statuses()
    }

    /// Propose a configuration change to a Raft group.
    ///
    /// Returns `(group_id, log_index)` on success.
    pub fn propose_conf_change(&self, group_id: u64, change: &ConfChange) -> Result<(u64, u64)> {
        let mut mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
        mr.propose_conf_change(group_id, change)
    }
}

// ── Incoming RPC handler ────────────────────────────────────────────

impl<A: CommitApplier, F: RequestForwarder> RaftRpcHandler for RaftLoop<A, F> {
    async fn handle_rpc(&self, rpc: RaftRpc) -> Result<RaftRpc> {
        match rpc {
            // Raft consensus RPCs — lock MultiRaft (sync, never across await).
            RaftRpc::AppendEntriesRequest(req) => {
                let mut mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
                let resp = mr.handle_append_entries(&req)?;
                Ok(RaftRpc::AppendEntriesResponse(resp))
            }
            RaftRpc::RequestVoteRequest(req) => {
                let mut mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
                let resp = mr.handle_request_vote(&req)?;
                Ok(RaftRpc::RequestVoteResponse(resp))
            }
            RaftRpc::InstallSnapshotRequest(req) => {
                let mut mr = self.multi_raft.lock().unwrap_or_else(|p| p.into_inner());
                let resp = mr.handle_install_snapshot(&req)?;
                Ok(RaftRpc::InstallSnapshotResponse(resp))
            }
            // Health check.
            RaftRpc::Ping(req) => {
                let topo_version = {
                    let topo = self.topology.read().unwrap_or_else(|p| p.into_inner());
                    topo.version()
                };
                Ok(health::handle_ping(self.node_id, topo_version, &req))
            }
            // Topology broadcast.
            RaftRpc::TopologyUpdate(update) => {
                let (_updated, ack) =
                    health::handle_topology_update(self.node_id, &self.topology, &update);
                Ok(ack)
            }
            // Query forwarding — execute locally via the RequestForwarder.
            RaftRpc::ForwardRequest(req) => {
                let resp = self.forwarder.execute_forwarded(req).await;
                Ok(RaftRpc::ForwardResponse(resp))
            }
            // VShardEnvelope — dispatch to registered handler (Event Plane, etc.).
            RaftRpc::VShardEnvelope(bytes) => {
                if let Some(ref handler) = self.vshard_handler {
                    let response_bytes = handler(bytes).await?;
                    Ok(RaftRpc::VShardEnvelope(response_bytes))
                } else {
                    Err(ClusterError::Transport {
                        detail: "VShardEnvelope handler not configured".into(),
                    })
                }
            }
            other => Err(ClusterError::Transport {
                detail: format!("unexpected request type in RPC handler: {other:?}"),
            }),
        }
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::routing::RoutingTable;
    use nodedb_types::config::tuning::ClusterTransportTuning;
    use std::sync::atomic::{AtomicU64, Ordering};
    use std::time::Instant;

    /// Test applier that counts applied entries.
    struct CountingApplier {
        applied: AtomicU64,
    }

    impl CountingApplier {
        fn new() -> Self {
            Self {
                applied: AtomicU64::new(0),
            }
        }

        fn count(&self) -> u64 {
            self.applied.load(Ordering::Relaxed)
        }
    }

    impl CommitApplier for CountingApplier {
        fn apply_committed(&self, _group_id: u64, entries: &[LogEntry]) -> u64 {
            self.applied
                .fetch_add(entries.len() as u64, Ordering::Relaxed);
            entries.last().map(|e| e.index).unwrap_or(0)
        }
    }

    /// Helper: create a transport on an ephemeral port.
    fn make_transport(node_id: u64) -> Arc<NexarTransport> {
        Arc::new(NexarTransport::new(node_id, "127.0.0.1:0".parse().unwrap()).unwrap())
    }

    #[tokio::test]
    async fn single_node_raft_loop_commits() {
        // Single-node cluster: elections and commits happen locally.
        let dir = tempfile::tempdir().unwrap();
        let transport = make_transport(1);
        let rt = RoutingTable::uniform(1, &[1], 1);
        let mut mr = MultiRaft::new(1, rt, dir.path().to_path_buf());
        mr.add_group(0, vec![]).unwrap();

        // Force election.
        for node in mr.groups_mut().values_mut() {
            node.election_deadline_override(Instant::now() - Duration::from_millis(1));
        }

        let applier = CountingApplier::new();
        let topo = Arc::new(RwLock::new(ClusterTopology::new()));
        let raft_loop = Arc::new(RaftLoop::new(mr, transport, topo, applier));

        let (shutdown_tx, shutdown_rx) = tokio::sync::watch::channel(false);

        let rl = raft_loop.clone();
        let run_handle = tokio::spawn(async move {
            rl.run(shutdown_rx).await;
        });

        // Wait for election + no-op commit.
        tokio::time::sleep(Duration::from_millis(50)).await;

        // The no-op entry should have been committed and applied.
        assert!(
            raft_loop.applier.count() >= 1,
            "expected at least 1 applied entry (no-op), got {}",
            raft_loop.applier.count()
        );

        // Propose a command.
        let (_gid, idx) = raft_loop.propose(0, b"hello".to_vec()).unwrap();
        assert!(idx >= 2); // 1 = no-op, 2+ = our command

        // Wait for commit.
        tokio::time::sleep(Duration::from_millis(50)).await;

        assert!(
            raft_loop.applier.count() >= 2,
            "expected at least 2 applied entries, got {}",
            raft_loop.applier.count()
        );

        shutdown_tx.send(true).unwrap();
        run_handle.abort();
    }

    #[tokio::test]
    async fn three_node_election_over_quic() {
        // Three-node cluster: node 1 should win election via QUIC RPCs.
        let t1 = make_transport(1);
        let t2 = make_transport(2);
        let t3 = make_transport(3);

        // Register peers.
        t1.register_peer(2, t2.local_addr());
        t1.register_peer(3, t3.local_addr());
        t2.register_peer(1, t1.local_addr());
        t2.register_peer(3, t3.local_addr());
        t3.register_peer(1, t1.local_addr());
        t3.register_peer(2, t2.local_addr());

        let rt = RoutingTable::uniform(1, &[1, 2, 3], 3);

        // Node 1: force immediate election.
        let dir1 = tempfile::tempdir().unwrap();
        let mut mr1 = MultiRaft::new(1, rt.clone(), dir1.path().to_path_buf());
        mr1.add_group(0, vec![2, 3]).unwrap();
        for node in mr1.groups_mut().values_mut() {
            node.election_deadline_override(Instant::now() - Duration::from_millis(1));
        }

        // Nodes 2 and 3: normal timeouts (won't start election).
        // Use the tuning config defaults so these values stay in sync with
        // ClusterTransportTuning::election_timeout_{min,max}_secs.
        let transport_tuning = ClusterTransportTuning::default();
        let election_timeout_min = Duration::from_secs(transport_tuning.election_timeout_min_secs);
        let election_timeout_max = Duration::from_secs(transport_tuning.election_timeout_max_secs);

        let dir2 = tempfile::tempdir().unwrap();
        let mut mr2 = MultiRaft::new(2, rt.clone(), dir2.path().to_path_buf())
            .with_election_timeout(election_timeout_min, election_timeout_max);
        mr2.add_group(0, vec![1, 3]).unwrap();

        let dir3 = tempfile::tempdir().unwrap();
        let mut mr3 = MultiRaft::new(3, rt.clone(), dir3.path().to_path_buf())
            .with_election_timeout(election_timeout_min, election_timeout_max);
        mr3.add_group(0, vec![1, 2]).unwrap();

        let a1 = CountingApplier::new();
        let a2 = CountingApplier::new();
        let a3 = CountingApplier::new();

        let topo1 = Arc::new(RwLock::new(ClusterTopology::new()));
        let topo2 = Arc::new(RwLock::new(ClusterTopology::new()));
        let topo3 = Arc::new(RwLock::new(ClusterTopology::new()));

        let rl1 = Arc::new(RaftLoop::new(mr1, t1.clone(), topo1, a1));
        let rl2 = Arc::new(RaftLoop::new(mr2, t2.clone(), topo2, a2));
        let rl3 = Arc::new(RaftLoop::new(mr3, t3.clone(), topo3, a3));

        let (shutdown_tx, _) = tokio::sync::watch::channel(false);

        // Start serve loops (incoming RPCs).
        let rl2_h = rl2.clone();
        let sr2 = shutdown_tx.subscribe();
        tokio::spawn(async move { t2.serve(rl2_h, sr2).await });

        let rl3_h = rl3.clone();
        let sr3 = shutdown_tx.subscribe();
        tokio::spawn(async move { t3.serve(rl3_h, sr3).await });

        // Start tick loops.
        let rl1_r = rl1.clone();
        let sr1 = shutdown_tx.subscribe();
        tokio::spawn(async move { rl1_r.run(sr1).await });

        let rl2_r = rl2.clone();
        let sr2r = shutdown_tx.subscribe();
        tokio::spawn(async move { rl2_r.run(sr2r).await });

        let rl3_r = rl3.clone();
        let sr3r = shutdown_tx.subscribe();
        tokio::spawn(async move { rl3_r.run(sr3r).await });

        // Also start serve for node 1 (receives responses).
        let rl1_h = rl1.clone();
        let sr1h = shutdown_tx.subscribe();
        tokio::spawn(async move { t1.serve(rl1_h, sr1h).await });

        // Wait for election + replication.
        tokio::time::sleep(Duration::from_millis(200)).await;

        // Node 1 should be leader and have committed the no-op.
        assert!(
            rl1.applier.count() >= 1,
            "node 1 should have committed at least the no-op, got {}",
            rl1.applier.count()
        );

        // Propose a command on node 1.
        let (_gid, idx) = rl1.propose(0, b"distributed-cmd".to_vec()).unwrap();
        assert!(idx >= 2);

        // Wait for replication.
        tokio::time::sleep(Duration::from_millis(200)).await;

        // All nodes should have applied the command (replicated via Raft).
        assert!(
            rl1.applier.count() >= 2,
            "node 1: expected >= 2 applied, got {}",
            rl1.applier.count()
        );

        // Followers apply committed entries when they receive AppendEntries
        // with updated leader_commit.
        assert!(
            rl2.applier.count() >= 1,
            "node 2: expected >= 1 applied, got {}",
            rl2.applier.count()
        );
        assert!(
            rl3.applier.count() >= 1,
            "node 3: expected >= 1 applied, got {}",
            rl3.applier.count()
        );

        shutdown_tx.send(true).unwrap();
    }

    #[tokio::test]
    async fn rpc_handler_routes_append_entries() {
        let dir = tempfile::tempdir().unwrap();
        let transport = make_transport(1);
        let rt = RoutingTable::uniform(1, &[1], 1);
        let mut mr = MultiRaft::new(1, rt, dir.path().to_path_buf());
        mr.add_group(0, vec![]).unwrap();

        // Make it a leader first.
        for node in mr.groups_mut().values_mut() {
            node.election_deadline_override(Instant::now() - Duration::from_millis(1));
        }

        let topo = Arc::new(RwLock::new(ClusterTopology::new()));
        let raft_loop = RaftLoop::new(mr, transport, topo, CountingApplier::new());

        // Tick to trigger election.
        raft_loop.do_tick();
        tokio::time::sleep(Duration::from_millis(20)).await;

        // Send an AppendEntries RPC via the handler.
        let req = RaftRpc::AppendEntriesRequest(nodedb_raft::AppendEntriesRequest {
            term: 99, // Higher term — will cause step-down.
            leader_id: 2,
            prev_log_index: 0,
            prev_log_term: 0,
            entries: vec![],
            leader_commit: 0,
            group_id: 0,
        });

        let resp = raft_loop.handle_rpc(req).await.unwrap();
        match resp {
            RaftRpc::AppendEntriesResponse(r) => {
                assert!(r.success);
                assert_eq!(r.term, 99);
            }
            other => panic!("expected AppendEntriesResponse, got {other:?}"),
        }
    }

    #[tokio::test]
    async fn rpc_handler_routes_request_vote() {
        let dir = tempfile::tempdir().unwrap();
        let transport = make_transport(1);
        let rt = RoutingTable::uniform(1, &[1, 2, 3], 3);
        let mut mr = MultiRaft::new(1, rt, dir.path().to_path_buf());
        mr.add_group(0, vec![2, 3]).unwrap();

        let topo = Arc::new(RwLock::new(ClusterTopology::new()));
        let raft_loop = RaftLoop::new(mr, transport, topo, CountingApplier::new());

        let req = RaftRpc::RequestVoteRequest(nodedb_raft::RequestVoteRequest {
            term: 1,
            candidate_id: 2,
            last_log_index: 0,
            last_log_term: 0,
            group_id: 0,
        });

        let resp = raft_loop.handle_rpc(req).await.unwrap();
        match resp {
            RaftRpc::RequestVoteResponse(r) => {
                assert!(r.vote_granted);
                assert_eq!(r.term, 1);
            }
            other => panic!("expected RequestVoteResponse, got {other:?}"),
        }
    }
}