d-engine-core 0.2.3

Pure Raft consensus algorithm - for building custom Raft-based systems
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
use std::fmt::Debug;
use std::marker::PhantomData;
use std::sync::Arc;

use d_engine_proto::common::LogId;
use d_engine_proto::common::NodeRole::Candidate;
use d_engine_proto::server::cluster::ClusterConfUpdateResponse;
use d_engine_proto::server::election::VoteResponse;
use d_engine_proto::server::election::VotedFor;
use tokio::sync::mpsc;
use tokio::time::Instant;
use tonic::Status;
use tonic::async_trait;
use tracing::debug;
use tracing::error;
use tracing::info;
use tracing::trace;
use tracing::warn;

use super::RaftRole;
use super::Result;
use super::SharedState;
use super::StateSnapshot;
use super::follower_state::FollowerState;
use super::role_state::RaftRoleState;
use crate::ConsensusError;
use crate::ElectionCore;
use crate::ElectionError;
use crate::ElectionTimer;
use crate::Error;
use crate::Membership;
use crate::NetworkError;
use crate::RaftContext;
use crate::RaftEvent;
use crate::RaftLog;
use crate::RaftNodeConfig;
use crate::ReplicationCore;
use crate::RoleEvent;
use crate::StateTransitionError;
use crate::TypeConfig;

/// Candidate node's volatile state during leader election.
///
/// This transient state manages election timers and vote solicitation process.
///
/// # Type Parameters
/// - `T`: Application-specific Raft type configuration
#[derive(Clone)]
pub struct CandidateState<T: TypeConfig> {
    // -- Core State --
    /// Shared cluster state with atomic access
    pub shared_state: SharedState,

    // -- Log Compaction --
    /// === Persistent State ===
    /// Last physically purged log index (inclusive)
    ///
    /// Note:
    /// Even though candidates don’t purge logs, they must keep last_purged_index from the
    /// follower. Otherwise, the system may lose purge info and get into an inconsistent state.
    pub last_purged_index: Option<LogId>,

    // -- Election Timing --
    /// Election timeout manager
    ///
    /// Tracks:
    /// - Randomized election timeout duration
    /// - Remaining time before transitioning to new election
    pub(super) timer: ElectionTimer,

    // -- Cluster Configuration --
    /// Immutable node configuration (shared reference)
    pub(super) node_config: Arc<RaftNodeConfig>,

    // -- Type System Marker --
    /// Phantom type for compile-time safety
    _marker: PhantomData<T>,
}

#[async_trait]
impl<T: TypeConfig> RaftRoleState for CandidateState<T> {
    type T = T;

    fn shared_state(&self) -> &SharedState {
        &self.shared_state
    }

    fn shared_state_mut(&mut self) -> &mut SharedState {
        &mut self.shared_state
    }

    fn is_candidate(&self) -> bool {
        true
    }

    fn become_leader(&self) -> Result<RaftRole<T>> {
        info!(
            "Node {} term {} transitioning to Leader",
            self.node_id(),
            self.current_term(),
        );
        println!(
            "[Node {}] Candidate → Leader (term {})",
            self.node_id(),
            self.current_term()
        );
        Ok(RaftRole::Leader(Box::new(self.into())))
    }

    fn become_candidate(&self) -> Result<RaftRole<T>> {
        warn!("I am candidate already");
        Err(StateTransitionError::InvalidTransition.into())
    }

    fn become_follower(&self) -> Result<RaftRole<T>> {
        info!(
            "Node {} term {} transitioning to Follower",
            self.node_id(),
            self.current_term(),
        );
        println!(
            "[Node {}] Candidate → Follower (term {})",
            self.node_id(),
            self.current_term()
        );
        Ok(RaftRole::Follower(Box::new(self.into())))
    }

    fn become_learner(&self) -> Result<RaftRole<T>> {
        info!(
            "Node {} term {} transitioning to Learner",
            self.node_id(),
            self.current_term(),
        );
        println!(
            "[Node {}] Candidate → Learner (term {})",
            self.node_id(),
            self.current_term()
        );
        Ok(RaftRole::Learner(Box::new(self.into())))
    }

    fn reset_timer(&mut self) {
        self.timer.reset()
    }

    fn is_timer_expired(&self) -> bool {
        self.timer.is_expired()
    }
    fn next_deadline(&self) -> Instant {
        self.timer.next_deadline()
    }

    /// Election Timeout: as candidate, it should send vote requests now
    async fn tick(
        &mut self,
        role_tx: &mpsc::UnboundedSender<RoleEvent>,
        _raft_event_tx: &mpsc::Sender<RaftEvent>,
        ctx: &RaftContext<T>,
    ) -> Result<()> {
        if Instant::now() < self.timer.next_deadline() {
            return Ok(());
        }
        debug!("reset timer");
        self.timer.reset();

        debug!("candidate::start_election...");

        self.increase_current_term();
        self.reset_voted_for()?;

        debug!("candidate new term: {}", self.current_term());

        self.vote_myself()?;

        match ctx
            .election_handler()
            .broadcast_vote_requests(
                self.current_term(),
                ctx.membership(),
                ctx.raft_log(),
                ctx.transport(),
                &ctx.node_config(),
            )
            .await
        {
            Ok(_) => {
                debug!("BecomeLeader");
                if let Err(e) = role_tx.send(RoleEvent::BecomeLeader) {
                    error!(
                        "self.my_role_change_event_sender.send(RaftRole::Leader) failed: {:?}",
                        e
                    );
                }
            }
            Err(Error::Consensus(ConsensusError::Election(ElectionError::HigherTerm(
                higher_term,
            )))) => {
                // Immediately update the Term and become a Follower.
                self.update_current_term(higher_term);
                self.send_become_follower_event(role_tx)?;
            }
            Err(e) => {
                warn!("candidate broadcast_vote_requests with error: {:?}", e);
            }
        }
        Ok(())
    }

    async fn handle_raft_event(
        &mut self,
        raft_event: RaftEvent,
        ctx: &RaftContext<T>,
        role_tx: mpsc::UnboundedSender<RoleEvent>,
    ) -> Result<()> {
        let my_term = self.current_term();
        match raft_event {
            RaftEvent::ReceiveVoteRequest(vote_request, sender) => {
                debug!(
                    "handle_raft_event::RaftEvent::ReceiveVoteRequest: {:?}",
                    &vote_request
                );

                let LogId {
                    index: last_log_index,
                    term: last_log_term,
                } = ctx.raft_log().last_log_id().unwrap_or(LogId { index: 0, term: 0 });

                if ctx.election_handler().check_vote_request_is_legal(
                    &vote_request,
                    my_term,
                    last_log_index,
                    last_log_term,
                    self.voted_for().unwrap(),
                ) {
                    self.update_current_term(vote_request.term);
                    // Step down as Follower
                    self.send_become_follower_event(&role_tx)?;

                    info!(
                        "Candidate will not process ReceiveVoteRequest, it should let Follower do it."
                    );
                    self.send_replay_raft_event(
                        &role_tx,
                        RaftEvent::ReceiveVoteRequest(vote_request, sender),
                    )?;
                } else {
                    let response = VoteResponse {
                        term: my_term,
                        vote_granted: false,
                        last_log_index,
                        last_log_term,
                    };
                    if let Err(e) = sender.send(Ok(response)) {
                        // Receiver timed out and dropped — this is normal, do not crash the node
                        error!("Failed to send VoteResponse (receiver dropped): {:?}", e);
                    }
                }
            }

            RaftEvent::ClusterConf(_metadata_request, sender) => {
                let cluster_conf = ctx
                    .membership()
                    .retrieve_cluster_membership_config(self.shared_state().current_leader())
                    .await;
                debug!("Candidate receive ClusterConf: {:?}", &cluster_conf);

                if let Err(e) = sender.send(Ok(cluster_conf)) {
                    // Receiver timed out and dropped — this is normal, do not crash the node
                    error!(
                        "Failed to send ClusterConf response (receiver dropped): {:?}",
                        e
                    );
                }
            }

            RaftEvent::ClusterConfUpdate(cluste_conf_change_request, sender) => {
                let current_conf_version = ctx.membership().get_cluster_conf_version().await;

                let current_leader_id = self.shared_state().current_leader();

                debug!(?current_leader_id, %current_conf_version, ?cluste_conf_change_request,
                    "Candidate receive ClusterConfUpdate"
                );

                let my_id = self.node_id();
                let response = match ctx
                    .membership()
                    .update_cluster_conf_from_leader(
                        my_id,
                        my_term,
                        current_conf_version,
                        current_leader_id,
                        &cluste_conf_change_request,
                    )
                    .await
                {
                    Ok(res) => res,
                    Err(e) => {
                        error!(?e, "update_cluster_conf_from_leader");
                        ClusterConfUpdateResponse::internal_error(
                            my_id,
                            my_term,
                            current_conf_version,
                        )
                    }
                };

                debug!(
                    "[peer-{}] update_cluster_conf_from_leader response: {:?}",
                    my_id, &response
                );
                if let Err(e) = sender.send(Ok(response)) {
                    // Receiver timed out and dropped — this is normal, do not crash the node
                    error!(
                        "Failed to send ClusterConfUpdate response (receiver dropped): {:?}",
                        e
                    );
                }
            }

            RaftEvent::AppendEntries(append_entries_request, sender) => {
                debug!(
                    "handle_raft_event::RaftEvent::AppendEntries: {:?}",
                    &append_entries_request
                );

                // Important to reset timer immediatelly
                self.reset_timer();

                let my_term = self.current_term();

                let response = ctx.replication_handler().check_append_entries_request_is_legal(
                    my_term,
                    &append_entries_request,
                    ctx.raft_log(),
                );

                // Handle illegal requests (return conflict or higher Term)
                if response.is_conflict() || response.is_higher_term() {
                    debug!("Rejecting AppendEntries: {:?}", &response);

                    if let Err(e) = sender.send(Ok(response)) {
                        // Receiver timed out and dropped — this is normal, do not crash the node
                        error!(
                            "Failed to send AppendEntries rejection (receiver dropped): {:?}",
                            e
                        );
                    }
                    return Ok(());
                } else {
                    // Keep syncing leader_id (hot-path: ~5ns atomic store)
                    self.shared_state().set_current_leader(append_entries_request.leader_id);

                    if append_entries_request.term > my_term {
                        self.update_current_term(append_entries_request.term);
                    }
                    // Step down as Follower
                    self.send_become_follower_event(&role_tx)?;

                    info!(
                        "Candidate will not process AppendEntries request, it should let Follower do it."
                    );
                    self.send_replay_raft_event(
                        &role_tx,
                        RaftEvent::AppendEntries(append_entries_request, sender),
                    )?;
                }
            }

            RaftEvent::InstallSnapshotChunk(_streaming, sender) => {
                sender
                    .send(Err(Status::permission_denied("Not Follower or Learner.")))
                    .map_err(|e| {
                        let error_str = format!("{e:?}");
                        error!("Failed to send: {}", error_str);
                        NetworkError::SingalSendFailed(error_str)
                    })?;

                return Err(ConsensusError::RoleViolation {
                    current_role: "Candidate",
                    required_role: "Follower or Learner",
                    context: format!(
                        "Candidate node {} receives RaftEvent::InstallSnapshotChunk",
                        ctx.node_id
                    ),
                }
                .into());
            }

            RaftEvent::CreateSnapshotEvent => {
                return Err(ConsensusError::RoleViolation {
                    current_role: "Candidate",
                    required_role: "Leader",
                    context: format!(
                        "Candidate node {} attempted to create snapshot.",
                        ctx.node_id
                    ),
                }
                .into());
            }

            RaftEvent::SnapshotCreated(_result) => {
                return Err(ConsensusError::RoleViolation {
                    current_role: "Candidate",
                    required_role: "Leader",
                    context: format!(
                        "Candidate node {} attempted to handle created snapshot.",
                        ctx.node_id
                    ),
                }
                .into());
            }

            RaftEvent::LogPurgeCompleted(_purged_id) => {
                return Err(ConsensusError::RoleViolation {
                    current_role: "Learner",
                    required_role: "Leader",
                    context: format!(
                        "Learner node {} should not receive LogPurgeCompleted event.",
                        ctx.node_id
                    ),
                }
                .into());
            }

            RaftEvent::JoinCluster(_join_request, sender) => {
                sender
                    .send(Err(Status::permission_denied(
                        "Candidate should not receive JoinCluster event.",
                    )))
                    .map_err(|e| {
                        let error_str = format!("{e:?}");
                        error!("Failed to send: {}", error_str);
                        NetworkError::SingalSendFailed(error_str)
                    })?;

                return Err(ConsensusError::RoleViolation {
                    current_role: "Candidate",
                    required_role: "Leader",
                    context: format!(
                        "Candidate node {} receives RaftEvent::JoinCluster",
                        ctx.node_id
                    ),
                }
                .into());
            }

            RaftEvent::DiscoverLeader(request, sender) => {
                debug!(?request, "Candidate::RaftEvent::DiscoverLeader");
                sender
                    .send(Err(Status::permission_denied(
                        "Candidate should not response DiscoverLeader event.",
                    )))
                    .map_err(|e| {
                        let error_str = format!("{e:?}");
                        error!("Failed to send: {}", error_str);
                        NetworkError::SingalSendFailed(error_str)
                    })?;

                return Ok(());
            }

            RaftEvent::StreamSnapshot(request, sender) => {
                debug!(?request, "Candidate::RaftEvent::StreamSnapshot");
                sender
                    .send(Err(Status::permission_denied(
                        "Candidate should not receive StreamSnapshot event.",
                    )))
                    .map_err(|e| {
                        let error_str = format!("{e:?}");
                        error!("Failed to send: {}", error_str);
                        NetworkError::SingalSendFailed(error_str)
                    })?;

                return Ok(());
            }

            RaftEvent::TriggerSnapshotPush { peer_id: _ } => {
                return Err(ConsensusError::RoleViolation {
                    current_role: "Candidate",
                    required_role: "Leader",
                    context: format!(
                        "Candidate node {} receives RaftEvent::TriggerSnapshotPush",
                        ctx.node_id
                    ),
                }
                .into());
            }

            RaftEvent::PromoteReadyLearners => {
                return Err(ConsensusError::RoleViolation {
                    current_role: "Candidate",
                    required_role: "Leader",
                    context: format!(
                        "Candidate node {} receives RaftEvent::PromoteReadyLearners",
                        ctx.node_id
                    ),
                }
                .into());
            }

            RaftEvent::MembershipApplied => {
                // Candidates don't maintain cluster metadata cache
                // This event is only relevant for leaders
                trace!("Candidate ignoring MembershipApplied event");
            }

            RaftEvent::ApplyCompleted {
                last_index,
                results,
            } => {
                // Candidate is a transient state; snapshot will be triggered after role transition.
                let _ = (last_index, results);
            }

            RaftEvent::FatalError { source, error } => {
                error!("[Candidate] Fatal error from {}: {}", source, error);
                return Err(crate::Error::Fatal(format!(
                    "Fatal error from {source}: {error}"
                )));
            }

            RaftEvent::StepDownSelfRemoved => {
                // Unreachable: handled at Raft level before reaching RoleState
                unreachable!("StepDownSelfRemoved should be handled in Raft::run()");
            }
        }

        Ok(())
    }
}

impl<T: TypeConfig> CandidateState<T> {
    pub fn can_vote_myself(&self) -> bool {
        if let Ok(Some(vf)) = self.voted_for() {
            let current_term = self.current_term();
            debug!(
                "[candiate::can_vote_myself] vf={:?}, current_term={}",
                &vf, current_term
            );
            // voted term is smaller than mine
            vf.voted_for_term < current_term
        } else {
            // either I have not voted for anyone yet
            debug!("[candiate::can_vote_myself] true");
            true
        }
    }
    pub fn vote_myself(&mut self) -> Result<()> {
        info!("vote myself as candidate");
        trace!(
            "Vote myself: my_id: {}, my_new_term:{}",
            self.node_id(),
            self.current_term()
        );
        let _ = self.update_voted_for(VotedFor {
            voted_for_id: self.node_id(),
            voted_for_term: self.current_term(),
            committed: false,
        })?;
        Ok(())
    }

    /// The fun will retrieve current state snapshot
    pub fn state_snapshot(&self) -> StateSnapshot {
        StateSnapshot {
            current_term: self.current_term(),
            voted_for: None,
            commit_index: self.commit_index(),
            role: Candidate as i32,
        }
    }

    pub fn send_become_follower_event(
        &self,
        role_tx: &mpsc::UnboundedSender<RoleEvent>,
    ) -> Result<()> {
        role_tx.send(RoleEvent::BecomeFollower(None)).map_err(|e| {
            let error_str = format!("{e:?}");
            error!("Failed to send: {}", error_str);
            NetworkError::SingalSendFailed(error_str).into()
        })
    }

    pub fn send_replay_raft_event(
        &self,
        role_tx: &mpsc::UnboundedSender<RoleEvent>,
        raft_event: RaftEvent,
    ) -> Result<()> {
        debug!("send_replay_raft_event, raft_event:{:?}", &raft_event);
        role_tx.send(RoleEvent::ReprocessEvent(Box::new(raft_event))).map_err(|e| {
            let error_str = format!("{e:?}");
            error!("Failed to send: {}", error_str);
            NetworkError::SingalSendFailed(error_str).into()
        })
    }

    #[cfg(test)]
    pub(crate) fn new(
        node_id: u32,
        node_config: Arc<RaftNodeConfig>,
    ) -> Self {
        Self {
            shared_state: SharedState::new(node_id, None, None),
            timer: ElectionTimer::new((
                node_config.raft.election.election_timeout_min,
                node_config.raft.election.election_timeout_max,
            )),
            node_config,
            _marker: PhantomData,
            last_purged_index: None, //TODO
        }
    }
}

impl<T: TypeConfig> From<&FollowerState<T>> for CandidateState<T> {
    fn from(follower: &FollowerState<T>) -> Self {
        trace!(%follower.node_config.raft.election.election_timeout_min, "From<&FollowerState<T>> for CandidateState");
        Self {
            shared_state: follower.shared_state.clone(),
            timer: ElectionTimer::new((
                follower.node_config.raft.election.election_timeout_min,
                follower.node_config.raft.election.election_timeout_max,
            )),
            node_config: follower.node_config.clone(),
            last_purged_index: follower.last_purged_index,
            _marker: PhantomData,
        }
    }
}
impl<T: TypeConfig> Drop for CandidateState<T> {
    fn drop(&mut self) {
        // self.votes.clear();
    }
}

impl<T: TypeConfig> Debug for CandidateState<T> {
    fn fmt(
        &self,
        f: &mut std::fmt::Formatter<'_>,
    ) -> std::fmt::Result {
        f.debug_struct("CandidateState")
            .field("shared_state", &self.shared_state)
            .finish()
    }
}