sorock 0.12.0

A Multi-Raft implementation in Rust
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
use super::*;

use anyhow::Result;
use std::collections::BTreeMap;
use std::collections::HashMap;
use std::collections::HashSet;
use tracing::{debug, error, info, warn};

use std::sync::atomic::AtomicU64;
use std::sync::atomic::Ordering;

mod storage;
pub use storage::RaftStorage;
mod api;
pub(super) use api::*;
mod peers;
use app::state_machine::StateMachine;
use peers::Peers;
mod voter;
use voter::Voter;
mod app;
use app::query_processing;
use app::state_machine;
use app::App;
use node::RaftHandle;
use state_machine::Command;

use app::completion;
mod kernel_message;
use completion::*;
mod thread;

/// Election term.
/// In Raft, only one leader can be elected per a term.
pub(super) type Term = u64;

/// Log index.
pub type LogIndex = u64;

/// Clock of log entry.
/// If two entries have the same clock, they should be the same entry.
/// It is like the hash of the git commit.
#[derive(Clone, Copy, Eq, Debug)]
pub(super) struct Clock {
    pub term: Term,
    pub index: LogIndex,
}
impl PartialEq for Clock {
    fn eq(&self, that: &Self) -> bool {
        self.term == that.term && self.index == that.index
    }
}

/// Log entry.
#[derive(Clone, Debug)]
struct Entry {
    prev_clock: Clock,
    this_clock: Clock,
    command: Bytes,
}

/// Ballot in election.
#[derive(Clone, Debug, PartialEq)]
struct Ballot {
    cur_term: Term,
    voted_for: Option<NodeAddress>,
}
impl Ballot {
    pub fn new() -> Self {
        Self {
            cur_term: 0,
            voted_for: None,
        }
    }
}

/// Snapshot is transferred as stream of bytes.
/// `SnapshotStream` is converted to gRPC streaming outside of the `RaftProcess`.
pub type SnapshotStream =
    std::pin::Pin<Box<dyn futures::stream::Stream<Item = anyhow::Result<Bytes>> + Send>>;

// This is only a marker that indicates the owner doesn't mutate the object.
// This is only to improve the readability.
// Compile-time or even runtime checking is more preferable.
#[derive(Deref, Clone)]
struct Read<T>(T);

/// `RaftApp` is an abstraction of state machine and snapshot store used by `RaftProcess`.
#[async_trait::async_trait]
pub trait RaftApp: Sync + Send + 'static {
    /// Apply read request to the application.
    /// Calling of this function should not change the state of the application.
    async fn process_read(&self, request: &[u8]) -> Result<Bytes>;

    /// Apply write request to the application.
    /// Calling of this function may change the state of the application.
    async fn process_write(&self, request: &[u8], entry_index: LogIndex) -> Result<Bytes>;

    /// Replace the state of the application with the snapshot.
    /// The snapshot is guaranteed to exist in the snapshot store.
    async fn install_snapshot(&self, snapshot_index: LogIndex) -> Result<()>;

    /// Save snapshot with index `snapshot_index` to the snapshot store.
    /// This function is called when the snapshot is fetched from the leader.
    async fn save_snapshot(&self, st: SnapshotStream, snapshot_index: LogIndex) -> Result<()>;

    /// Read existing snapshot with index `snapshot_index` from the snapshot store.
    /// This function is called when a follower requests a snapshot from the leader.
    async fn open_snapshot(&self, snapshot_index: LogIndex) -> Result<SnapshotStream>;

    /// Delete all the snapshots in `[,  i)` from the snapshot store.
    async fn delete_snapshots_before(&self, i: LogIndex) -> Result<()>;

    /// Get the index of the latest snapshot in the snapshot store.
    /// If the index is greater than the current snapshot entry index,
    /// it will replace the snapshot entry with the new one.
    async fn get_latest_snapshot(&self) -> Result<LogIndex>;
}

#[allow(dead_code)]
struct ThreadHandles {
    advance_kernel_handle: thread::ThreadHandle,
    advance_application_handle: thread::ThreadHandle,
    advance_snapshot_handle: thread::ThreadHandle,
    advance_commit_handle: thread::ThreadHandle,
    election_handle: thread::ThreadHandle,
    log_compaction_handle: thread::ThreadHandle,
    query_execution_handle: thread::ThreadHandle,
    snapshot_deleter_handle: thread::ThreadHandle,
    stepdown_handle: thread::ThreadHandle,
}

/// `RaftProcess` is a implementation of Raft process in `RaftNode`.
/// `RaftProcess` is agnostic to the I/O implementation and focuses on pure Raft algorithm.
pub struct RaftProcess {
    state_machine: StateMachine,
    voter: Voter,
    peers: Peers,
    query_queue: query_processing::QueryQueue,
    app: App,
    driver: node::RaftHandle,
    _thread_handles: ThreadHandles,

    queue_tx: thread::EventProducer<thread::QueueEvent>,
    replication_tx: thread::EventProducer<thread::ReplicationEvent>,
}

impl RaftProcess {
    pub async fn new(
        app: impl RaftApp,
        storage: &storage::RaftStorage,
        driver: node::RaftHandle,
    ) -> Result<Self> {
        let app = App::new(app);
        let (log_store, ballot_store) = storage.get(driver.shard_index)?;

        let (query_tx, query_rx) = query_processing::new(Read(app.clone()));

        let state_machine = StateMachine::new(log_store, app.clone());
        state_machine::effect::restore_state::Effect {
            state_machine: state_machine.clone(),
        }
        .exec()
        .await?;

        let (queue_tx, queue_rx) = thread::notify();
        let (replication_tx, replication_rx) = thread::notify();
        let (commit_tx, commit_rx) = thread::notify();
        let (kern_tx, kern_rx) = thread::notify();
        let (app_tx, app_rx) = thread::notify();

        let peers = Peers::new(
            Read(state_machine.clone()),
            queue_rx.clone(),
            replication_tx.clone(),
            driver.clone(),
        );

        let voter = Voter::new(
            ballot_store,
            Read(state_machine.clone()),
            Read(peers.clone()),
            driver.clone(),
        );

        peers::effect::restore_state::Effect {
            peers: peers.clone(),
            state_machine: state_machine.clone(),
            voter: Read(voter.clone()),
            driver: driver.clone(),
        }
        .exec()
        .await?;

        let _thread_handles = ThreadHandles {
            advance_kernel_handle: thread::advance_kernel::new(
                state_machine.clone(),
                voter.clone(),
                commit_rx.clone(),
                kern_tx.clone(),
            ),
            advance_application_handle: thread::advance_application::new(
                state_machine.clone(),
                kern_rx.clone(),
                app_tx.clone(),
            ),
            advance_snapshot_handle: thread::advance_snapshot::new(state_machine.clone()),
            advance_commit_handle: thread::advance_commit::new(
                state_machine.clone(),
                Read(peers.clone()),
                Read(voter.clone()),
                replication_rx.clone(),
                commit_tx.clone(),
            ),
            election_handle: thread::election::new(
                voter.clone(),
                state_machine.clone(),
                peers.clone(),
            ),
            log_compaction_handle: thread::gc_log::new(state_machine.clone()),
            query_execution_handle: thread::query_execution::new(
                query_rx.clone(),
                Read(state_machine.clone()),
                app_rx.clone(),
            ),
            snapshot_deleter_handle: thread::gc_snapshot::new(
                app.clone(),
                Read(state_machine.clone()),
            ),
            stepdown_handle: thread::stepdown::new(
                voter.clone(),
                Read(state_machine.clone()),
                Read(peers.clone()),
            ),
        };

        Ok(Self {
            state_machine,
            voter,
            peers,
            query_queue: query_tx,
            driver,
            app,
            _thread_handles,

            queue_tx,
            replication_tx,
        })
    }

    /// Process configuration change if the command contains configuration.
    /// Configuration should be applied as soon as it is inserted into the log because doing so
    /// guarantees that majority of the servers move to the configuration when the entry is committed.
    /// Without this property, servers may still be in some old configuration which may cause split-brain
    /// by electing two leaders in a single term which is not allowed in Raft.
    async fn process_configuration_command(&self, command: &[u8], index: LogIndex) -> Result<()> {
        let config0 = match Command::deserialize(command) {
            Command::Snapshot { membership } => Some(membership),
            Command::ClusterConfiguration { membership } => Some(membership),
            _ => None,
        };
        if let Some(config) = config0 {
            peers::effect::set_membership::Effect {
                peers: self.peers.clone(),
                state_machine: self.state_machine.clone(),
                voter: Read(self.voter.clone()),
                driver: self.driver.clone(),
            }
            .exec(config, index)
            .await?;
        }
        Ok(())
    }

    async fn queue_new_entry(&self, command: Bytes, completion: Completion) -> Result<LogIndex> {
        ensure!(self.voter.allow_queue_new_entry().await?);

        let append_index = state_machine::effect::append_entry::Effect {
            state_machine: self.state_machine.clone(),
        }
        .exec(command.clone(), None)
        .await?;

        self.state_machine
            .register_completion(append_index, completion);

        self.process_configuration_command(&command, append_index)
            .await?;

        self.queue_tx.push_event(thread::QueueEvent);
        self.replication_tx.push_event(thread::ReplicationEvent);

        Ok(append_index)
    }

    async fn queue_received_entries(&self, mut req: request::ReplicationStream) -> Result<u64> {
        let mut prev_clock = req.prev_clock;
        let mut n_inserted = 0;
        while let Some(Some(cur)) = req.entries.next().await {
            let entry = Entry {
                prev_clock,
                this_clock: cur.this_clock,
                command: cur.command,
            };
            let insert_index = entry.this_clock.index;
            let command = entry.command.clone();

            use state_machine::effect::try_insert::TryInsertResult;

            let insert_result = state_machine::effect::try_insert::Effect {
                state_machine: self.state_machine.clone(),
                driver: self.driver.clone(),
            }
            .exec(entry, req.sender_id.clone())
            .await?;

            match insert_result {
                TryInsertResult::Inserted => {
                    self.process_configuration_command(&command, insert_index)
                        .await?;
                }
                TryInsertResult::SkippedInsertion => {}
                TryInsertResult::InconsistentInsertion { want, found } => {
                    warn!("rejected append entry (clock={:?}) for inconsisntency (want:{want:?} != found:{found:?}", cur.this_clock);
                    break;
                }
                TryInsertResult::LeapInsertion { want } => {
                    debug!(
                        "rejected append entry (clock={:?}) for leap insertion (want={want:?})",
                        cur.this_clock
                    );
                    break;
                }
            }
            prev_clock = cur.this_clock;
            n_inserted += 1;
        }

        Ok(n_inserted)
    }

    /// Forming a new cluster with a single node is called "cluster bootstrapping".
    /// Raft algorith doesn't define adding node when the cluster is empty.
    /// We need to handle this special case.
    async fn bootstrap_cluster(&self) -> Result<()> {
        let mut membership = HashSet::new();
        membership.insert(self.driver.self_node_id());

        let command = Command::serialize(Command::ClusterConfiguration { membership });
        state_machine::effect::append_entry::Effect {
            state_machine: self.state_machine.clone(),
        }
        .exec(command.clone(), None)
        .await?;

        self.process_configuration_command(&command, 2).await?;

        // After this function is called
        // this server should immediately become the leader by self-vote and advance commit index.
        // Consequently, when initial install_snapshot is called this server is already the leader.
        let conn = self.driver.connect(self.driver.self_node_id());
        conn.send_timeout_now().await?;

        Ok(())
    }

    pub(super) async fn add_server(&self, req: request::AddServer) -> Result<()> {
        if self.peers.read_membership().is_empty() && req.server_id == self.driver.self_node_id() {
            self.bootstrap_cluster().await?;
        } else {
            let msg = kernel_message::KernelMessage::AddServer(req.server_id);
            let req = request::KernelRequest {
                message: msg.serialize(),
            };
            let conn = self.driver.connect(self.driver.self_node_id());
            conn.process_kernel_request(req).await?;
        }
        Ok(())
    }

    pub(super) async fn remove_server(&self, req: request::RemoveServer) -> Result<()> {
        let msg = kernel_message::KernelMessage::RemoveServer(req.server_id);
        let req = request::KernelRequest {
            message: msg.serialize(),
        };
        let conn = self.driver.connect(self.driver.self_node_id());
        conn.process_kernel_request(req).await?;
        Ok(())
    }

    pub(super) async fn send_replication_stream(
        &self,
        req: request::ReplicationStream,
    ) -> Result<response::ReplicationStream> {
        let n_inserted = self.queue_received_entries(req).await?;

        let resp = response::ReplicationStream {
            n_inserted,
            log_last_index: self.state_machine.get_log_last_index().await?,
        };
        Ok(resp)
    }

    pub(super) async fn process_kernel_request(&self, req: request::KernelRequest) -> Result<()> {
        let ballot = self.voter.read_ballot().await?;

        let Some(leader_id) = ballot.voted_for else {
            bail!(Error::LeaderUnknown)
        };

        if std::matches!(
            self.voter.read_election_state(),
            voter::ElectionState::Leader
        ) {
            let (kern_completion, rx) = completion::prepare_kernel_completion();
            let command = match kernel_message::KernelMessage::deserialize(&req.message).unwrap() {
                kernel_message::KernelMessage::AddServer(id) => {
                    let mut membership = self.peers.read_membership();
                    membership.insert(id);
                    Command::ClusterConfiguration { membership }
                }
                kernel_message::KernelMessage::RemoveServer(id) => {
                    let mut membership = self.peers.read_membership();
                    membership.remove(&id);
                    Command::ClusterConfiguration { membership }
                }
            };
            ensure!(self.state_machine.allow_queue_new_membership());
            self.queue_new_entry(
                Command::serialize(command),
                Completion::Kernel(kern_completion),
            )
            .await?;

            rx.await?;
        } else {
            // Avoid looping.
            ensure!(self.driver.self_node_id() != leader_id);
            let conn = self.driver.connect(leader_id);
            conn.process_kernel_request(req).await?;
        }
        Ok(())
    }

    pub(super) async fn process_application_read_request(
        &self,
        req: request::ApplicationReadRequest,
    ) -> Result<Bytes> {
        let ballot = self.voter.read_ballot().await?;

        let Some(leader_id) = ballot.voted_for else {
            anyhow::bail!(Error::LeaderUnknown)
        };

        let will_process = req.read_locally
            || std::matches!(
                self.voter.read_election_state(),
                voter::ElectionState::Leader
            );

        let resp = if will_process {
            let (app_completion, rx) = completion::prepare_application_completion();

            let read_index = self.state_machine.commit_pointer.load(Ordering::SeqCst);
            let query = query_processing::Query {
                message: req.message,
                app_completion,
            };
            self.query_queue.register(read_index, query)?;

            rx.await?
        } else {
            // Avoid looping.
            ensure!(self.driver.self_node_id() != leader_id);
            let conn = self.driver.connect(leader_id);
            conn.process_application_read_request(req).await?
        };
        Ok(resp)
    }

    pub(super) async fn process_application_write_request(
        &self,
        req: request::ApplicationWriteRequest,
    ) -> Result<Bytes> {
        let ballot = self.voter.read_ballot().await?;

        let Some(leader_id) = ballot.voted_for else {
            bail!(Error::LeaderUnknown)
        };

        let resp = if std::matches!(
            self.voter.read_election_state(),
            voter::ElectionState::Leader
        ) {
            let (app_completion, rx) = completion::prepare_application_completion();

            let command = Command::ExecuteRequest {
                message: &req.message,
                request_id: req.request_id,
            };

            self.queue_new_entry(
                Command::serialize(command),
                Completion::Application(app_completion),
            )
            .await?;

            rx.await?
        } else {
            // Avoid looping.
            ensure!(self.driver.self_node_id() != leader_id);
            let conn = self.driver.connect(leader_id);
            conn.process_application_write_request(req).await?
        };
        Ok(resp)
    }

    pub(super) async fn receive_heartbeat(
        &self,
        leader_id: NodeAddress,
        req: request::Heartbeat,
    ) -> Result<()> {
        let term = req.leader_term;
        let leader_commit = req.leader_commit_index;

        voter::effect::receive_heartbeat::Effect {
            voter: self.voter.clone(),
            state_machine: self.state_machine.clone(),
        }
        .exec(leader_id, term, leader_commit)
        .await?;

        Ok(())
    }

    pub(super) async fn get_snapshot(&self, index: LogIndex) -> Result<SnapshotStream> {
        let cur_snapshot_index = self.state_machine.snapshot_pointer.load(Ordering::SeqCst);
        ensure!(index == cur_snapshot_index);
        let st = self.app.open_snapshot(index).await?;
        Ok(st)
    }

    pub(super) async fn send_timeout_now(&self) -> Result<()> {
        info!("received TimeoutNow. try to become a leader.");
        voter::effect::try_promote::Effect {
            voter: self.voter.clone(),
            state_machine: self.state_machine.clone(),
            peers: self.peers.clone(),
        }
        .exec(true)
        .await?;
        Ok(())
    }

    pub(super) async fn request_vote(&self, req: request::RequestVote) -> Result<bool> {
        let candidate_term = req.vote_term;
        let candidate_id = req.candidate_id;
        let candidate_clock = req.candidate_clock;
        let force_vote = req.force_vote;
        let pre_vote = req.pre_vote;

        let vote_granted = voter::effect::receive_vote_request::Effect {
            voter: self.voter.clone(),
            // state_machine: Read(self.state_machine.clone()),
        }
        .exec(
            candidate_term,
            candidate_id,
            candidate_clock,
            force_vote,
            pre_vote,
        )
        .await?;

        Ok(vote_granted)
    }

    pub(super) async fn get_log_state(&self) -> Result<response::LogState> {
        let out = response::LogState {
            head_index: self.state_machine.get_log_head_index().await?,
            last_index: self.state_machine.get_log_last_index().await?,
            snapshot_index: self.state_machine.snapshot_pointer.load(Ordering::SeqCst),
            application_index: self
                .state_machine
                .application_pointer
                .load(Ordering::SeqCst),
            commit_index: self.state_machine.commit_pointer.load(Ordering::SeqCst),
        };
        Ok(out)
    }

    pub(super) async fn get_membership(&self) -> Result<response::Membership> {
        let out = response::Membership {
            members: self.peers.read_membership(),
        };
        Ok(out)
    }
}