rivven-cluster 0.0.17

Distributed clustering for Rivven - SWIM membership, Raft consensus, ISR 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
//! redb-based Raft log storage
//!
//! Pure Rust implementation using redb - zero C dependencies, compiles
//! cleanly for all targets including musl.

// Suppress clippy warning for large error types from openraft crate
#![allow(clippy::result_large_err)]

use crate::error::{ClusterError, Result};
use openraft::storage::{LogState, RaftLogReader, RaftLogStorage};
use openraft::{StorageError, StorageIOError};
use redb::{Database, ReadableTable, TableDefinition};
use serde::{Deserialize, Serialize};
use std::fmt::Debug;
use std::ops::RangeBounds;
use std::path::Path;
use std::sync::Arc;
use tokio::sync::RwLock;
use tracing::{debug, info};

use crate::raft::{NodeId, RaftEntry, RaftLogId, RaftVote, TypeConfig};

/// Table for Raft log entries (key: u64 index, value: serialized entry)
const LOGS_TABLE: TableDefinition<u64, &[u8]> = TableDefinition::new("raft_logs");

/// Table for Raft state (key: string, value: serialized data)
const STATE_TABLE: TableDefinition<&str, &[u8]> = TableDefinition::new("raft_state");

/// State keys
const KEY_VOTE: &str = "vote";
const KEY_LAST_PURGED: &str = "last_purged";
const KEY_COMMITTED: &str = "committed";

/// redb-backed Raft log storage
///
/// This is a pure Rust implementation that:
/// - Has zero C/C++ dependencies
/// - Compiles for any target (including musl)
/// - Provides ACID transactions
/// - Uses B-tree for efficient ordered access
///
/// # Performance Characteristics
///
/// - **Write**: O(log n) per entry, batched for throughput
/// - **Read**: O(log n) point lookup, O(k) range scan for k entries
/// - **Space**: ~1.5x raw data size (B-tree overhead)
///
/// For Raft workloads (append-heavy with sequential reads), this provides
/// excellent performance with the benefit of pure Rust simplicity.
pub struct RedbLogStore {
    /// redb database instance
    db: Arc<Database>,
    /// Cached vote (also persisted)
    vote: RwLock<Option<RaftVote>>,
    /// Last purged log ID
    last_purged: RwLock<Option<RaftLogId>>,
    /// Committed log ID
    committed: RwLock<Option<RaftLogId>>,
}

impl RedbLogStore {
    /// Create new redb log storage at the given path
    ///
    /// Creates the directory if it doesn't exist.
    pub fn new(path: impl AsRef<Path>) -> Result<Self> {
        let path = path.as_ref();

        // Create directory if needed
        if let Some(parent) = path.parent() {
            std::fs::create_dir_all(parent)
                .map_err(|e| ClusterError::RaftStorage(format!("Failed to create dir: {}", e)))?;
        }

        // Open redb database
        let db = Database::create(path)
            .map_err(|e| ClusterError::RaftStorage(format!("Failed to open redb: {}", e)))?;

        let db = Arc::new(db);

        // Initialize tables
        {
            let write_txn = db
                .begin_write()
                .map_err(|e| ClusterError::RaftStorage(e.to_string()))?;
            {
                // Create tables if they don't exist
                let _ = write_txn.open_table(LOGS_TABLE);
                let _ = write_txn.open_table(STATE_TABLE);
            }
            write_txn
                .commit()
                .map_err(|e| ClusterError::RaftStorage(e.to_string()))?;
        }

        // Load persisted state
        let vote = Self::load_state_static(&db, KEY_VOTE);
        let last_purged = Self::load_state_static(&db, KEY_LAST_PURGED);
        let committed = Self::load_state_static(&db, KEY_COMMITTED);

        info!(
            ?vote,
            ?last_purged,
            ?committed,
            "Opened redb Raft log storage"
        );

        Ok(Self {
            db,
            vote: RwLock::new(vote),
            last_purged: RwLock::new(last_purged),
            committed: RwLock::new(committed),
        })
    }

    /// Load state from database
    fn load_state_static<T: for<'de> Deserialize<'de>>(db: &Database, key: &str) -> Option<T> {
        let read_txn = db.begin_read().ok()?;
        let table = read_txn.open_table(STATE_TABLE).ok()?;
        let value = table.get(key).ok()??;
        postcard::from_bytes(value.value()).ok()
    }

    /// Save state to database (single key convenience wrapper)
    fn save_state<T: Serialize>(
        &self,
        key: &str,
        value: &T,
    ) -> std::result::Result<(), StorageError<NodeId>> {
        let bytes = postcard::to_allocvec(value).map_err(|e| StorageError::IO {
            source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
        })?;
        self.save_states(&[(key, &bytes)])
    }

    /// Save multiple state entries in a single write transaction (F-040 fix).
    ///
    /// Consolidates multiple key-value writes into one ACID transaction,
    /// avoiding the overhead and inconsistency risk of separate transactions.
    fn save_states(
        &self,
        entries: &[(&str, &[u8])],
    ) -> std::result::Result<(), StorageError<NodeId>> {
        let write_txn = self.db.begin_write().map_err(|e| StorageError::IO {
            source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
        })?;
        {
            let mut table = write_txn
                .open_table(STATE_TABLE)
                .map_err(|e| StorageError::IO {
                    source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
                })?;
            for (key, bytes) in entries {
                table.insert(*key, *bytes).map_err(|e| StorageError::IO {
                    source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
                })?;
            }
        }
        write_txn.commit().map_err(|e| StorageError::IO {
            source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
        })?;
        Ok(())
    }

    /// Get the last log entry
    fn last_log(&self) -> std::result::Result<Option<RaftEntry>, StorageError<NodeId>> {
        let read_txn = self.db.begin_read().map_err(|e| StorageError::IO {
            source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
        })?;
        let table = read_txn
            .open_table(LOGS_TABLE)
            .map_err(|e| StorageError::IO {
                source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
            })?;

        // Get the last entry using reverse iteration
        let mut iter = table.iter().map_err(|e| StorageError::IO {
            source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
        })?;

        if let Some(result) = iter.next_back() {
            let (_, value) = result.map_err(|e| StorageError::IO {
                source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
            })?;
            let entry: RaftEntry =
                postcard::from_bytes(value.value()).map_err(|e| StorageError::IO {
                    source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
                })?;
            return Ok(Some(entry));
        }
        Ok(None)
    }

    /// Get log entry by index
    #[allow(dead_code)]
    fn get_log(&self, index: u64) -> std::result::Result<Option<RaftEntry>, StorageError<NodeId>> {
        let read_txn = self.db.begin_read().map_err(|e| StorageError::IO {
            source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
        })?;
        let table = read_txn
            .open_table(LOGS_TABLE)
            .map_err(|e| StorageError::IO {
                source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
            })?;

        match table.get(index) {
            Ok(Some(value)) => {
                let entry: RaftEntry =
                    postcard::from_bytes(value.value()).map_err(|e| StorageError::IO {
                        source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
                    })?;
                Ok(Some(entry))
            }
            Ok(None) => Ok(None),
            Err(e) => Err(StorageError::IO {
                source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
            }),
        }
    }

    /// Append a batch of log entries (single transaction for efficiency)
    fn append_logs(&self, entries: &[RaftEntry]) -> std::result::Result<(), StorageError<NodeId>> {
        if entries.is_empty() {
            return Ok(());
        }

        let write_txn = self.db.begin_write().map_err(|e| StorageError::IO {
            source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
        })?;
        {
            let mut table = write_txn
                .open_table(LOGS_TABLE)
                .map_err(|e| StorageError::IO {
                    source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
                })?;

            for entry in entries {
                let value = postcard::to_allocvec(entry).map_err(|e| StorageError::IO {
                    source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
                })?;
                table
                    .insert(entry.log_id.index, value.as_slice())
                    .map_err(|e| StorageError::IO {
                        source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
                    })?;
            }
        }
        write_txn.commit().map_err(|e| StorageError::IO {
            source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
        })?;
        Ok(())
    }

    /// Delete log entries in range [start, end)
    ///
    /// F-036 fix: Propagate errors instead of silently swallowing them
    /// with `let _ = table.remove(index)`.
    fn delete_logs_range(
        &self,
        start: u64,
        end: u64,
    ) -> std::result::Result<(), StorageError<NodeId>> {
        let write_txn = self.db.begin_write().map_err(|e| StorageError::IO {
            source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
        })?;
        {
            let mut table = write_txn
                .open_table(LOGS_TABLE)
                .map_err(|e| StorageError::IO {
                    source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
                })?;

            // Use drain_filter/retain pattern with error propagation
            for index in start..end {
                table.remove(index).map_err(|e| StorageError::IO {
                    source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
                })?;
            }
        }
        write_txn.commit().map_err(|e| StorageError::IO {
            source: StorageIOError::write_logs(openraft::AnyError::new(&e)),
        })?;
        Ok(())
    }
}

// Implement RaftLogReader for RedbLogStore
impl RaftLogReader<TypeConfig> for RedbLogStore {
    /// F-035 fix: Use a single read transaction with range iteration instead
    /// of opening a separate transaction per entry via `get_log()`.
    async fn try_get_log_entries<RB: RangeBounds<u64> + Clone + Debug + Send>(
        &mut self,
        range: RB,
    ) -> std::result::Result<Vec<RaftEntry>, StorageError<NodeId>> {
        let start = match range.start_bound() {
            std::ops::Bound::Included(&n) => n,
            std::ops::Bound::Excluded(&n) => n + 1,
            std::ops::Bound::Unbounded => 0,
        };
        let end = match range.end_bound() {
            std::ops::Bound::Included(&n) => n + 1,
            std::ops::Bound::Excluded(&n) => n,
            std::ops::Bound::Unbounded => u64::MAX,
        };

        // Single read transaction for the entire range scan
        let read_txn = self.db.begin_read().map_err(|e| StorageError::IO {
            source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
        })?;
        let table = read_txn
            .open_table(LOGS_TABLE)
            .map_err(|e| StorageError::IO {
                source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
            })?;

        let mut entries = Vec::new();
        let iter = table.range(start..end).map_err(|e| StorageError::IO {
            source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
        })?;

        for item in iter {
            let (_, value) = item.map_err(|e| StorageError::IO {
                source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
            })?;
            let entry: RaftEntry =
                postcard::from_bytes(value.value()).map_err(|e| StorageError::IO {
                    source: StorageIOError::read_logs(openraft::AnyError::new(&e)),
                })?;
            entries.push(entry);
        }

        Ok(entries)
    }
}

// Implement RaftLogStorage for RedbLogStore
impl RaftLogStorage<TypeConfig> for RedbLogStore {
    type LogReader = Self;

    async fn get_log_state(
        &mut self,
    ) -> std::result::Result<LogState<TypeConfig>, StorageError<NodeId>> {
        let last_purged = *self.last_purged.read().await;
        let last_log = self.last_log()?;

        let last_log_id = last_log.map(|e| e.log_id).or(last_purged);

        Ok(LogState {
            last_purged_log_id: last_purged,
            last_log_id,
        })
    }

    async fn get_log_reader(&mut self) -> Self::LogReader {
        // Return a new instance sharing the same DB
        Self {
            db: self.db.clone(),
            vote: RwLock::new(*self.vote.read().await),
            last_purged: RwLock::new(*self.last_purged.read().await),
            committed: RwLock::new(*self.committed.read().await),
        }
    }

    async fn save_vote(
        &mut self,
        vote: &RaftVote,
    ) -> std::result::Result<(), StorageError<NodeId>> {
        self.save_state(KEY_VOTE, vote)?;
        *self.vote.write().await = Some(*vote);
        debug!(?vote, "Saved vote");
        Ok(())
    }

    async fn read_vote(&mut self) -> std::result::Result<Option<RaftVote>, StorageError<NodeId>> {
        Ok(*self.vote.read().await)
    }

    async fn save_committed(
        &mut self,
        committed: Option<RaftLogId>,
    ) -> std::result::Result<(), StorageError<NodeId>> {
        if let Some(ref c) = committed {
            self.save_state(KEY_COMMITTED, c)?;
        }
        *self.committed.write().await = committed;
        Ok(())
    }

    async fn read_committed(
        &mut self,
    ) -> std::result::Result<Option<RaftLogId>, StorageError<NodeId>> {
        Ok(*self.committed.read().await)
    }

    async fn append<I>(
        &mut self,
        entries: I,
        callback: openraft::storage::LogFlushed<TypeConfig>,
    ) -> std::result::Result<(), StorageError<NodeId>>
    where
        I: IntoIterator<Item = RaftEntry> + Send,
        I::IntoIter: Send,
    {
        // Collect entries for batch write
        let entries: Vec<_> = entries.into_iter().collect();
        self.append_logs(&entries)?;

        // Callback after successful write
        callback.log_io_completed(Ok(()));
        Ok(())
    }

    async fn truncate(
        &mut self,
        log_id: RaftLogId,
    ) -> std::result::Result<(), StorageError<NodeId>> {
        // Delete all logs after log_id.index
        let start = log_id.index + 1;
        let log_state = RaftLogStorage::get_log_state(self).await?;
        if let Some(last) = log_state.last_log_id {
            self.delete_logs_range(start, last.index + 1)?;
        }
        debug!(?log_id, "Truncated logs");
        Ok(())
    }

    async fn purge(&mut self, log_id: RaftLogId) -> std::result::Result<(), StorageError<NodeId>> {
        // Delete logs up to and including log_id.index
        let current_purged = *self.last_purged.read().await;
        let start = current_purged.map(|l| l.index + 1).unwrap_or(0);

        self.delete_logs_range(start, log_id.index + 1)?;

        // Update and persist last_purged
        self.save_state(KEY_LAST_PURGED, &log_id)?;
        *self.last_purged.write().await = Some(log_id);
        debug!(?log_id, "Purged logs");
        Ok(())
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::raft::{RaftEntry, RaftLogId, RaftVote};
    use openraft::{Entry, EntryPayload, LogId, Vote};
    use tempfile::TempDir;

    fn create_entry(index: u64, term: u64) -> RaftEntry {
        Entry {
            log_id: LogId {
                leader_id: openraft::LeaderId::new(term, 1),
                index,
            },
            payload: EntryPayload::Blank,
        }
    }

    #[tokio::test]
    async fn test_redb_store_basic() {
        let temp_dir = TempDir::new().unwrap();
        let path = temp_dir.path().join("raft.redb");

        let mut store = RedbLogStore::new(&path).unwrap();

        // Initially empty
        let state = store.get_log_state().await.unwrap();
        assert!(state.last_log_id.is_none());

        // Append entries directly using internal method
        let entries = vec![create_entry(1, 1), create_entry(2, 1), create_entry(3, 1)];
        store.append_logs(&entries).unwrap();

        // Check state
        let state = store.get_log_state().await.unwrap();
        assert_eq!(state.last_log_id.unwrap().index, 3);

        // Read entries back
        let read_entries = store.try_get_log_entries(1..=3).await.unwrap();
        assert_eq!(read_entries.len(), 3);
    }

    #[tokio::test]
    async fn test_redb_store_vote() {
        let temp_dir = TempDir::new().unwrap();
        let path = temp_dir.path().join("raft.redb");

        let mut store = RedbLogStore::new(&path).unwrap();

        // No vote initially
        let vote = store.read_vote().await.unwrap();
        assert!(vote.is_none());

        // Save vote
        let test_vote: RaftVote = Vote::new(1, 1);
        store.save_vote(&test_vote).await.unwrap();

        // Read back
        let vote = store.read_vote().await.unwrap();
        assert_eq!(vote, Some(test_vote));

        // Persistence test - reopen
        drop(store);
        let mut store = RedbLogStore::new(&path).unwrap();
        let vote = store.read_vote().await.unwrap();
        assert_eq!(vote, Some(test_vote));
    }

    #[tokio::test]
    async fn test_redb_store_truncate() {
        let temp_dir = TempDir::new().unwrap();
        let path = temp_dir.path().join("raft.redb");

        let mut store = RedbLogStore::new(&path).unwrap();

        // Append entries
        let entries = vec![
            create_entry(1, 1),
            create_entry(2, 1),
            create_entry(3, 1),
            create_entry(4, 1),
            create_entry(5, 1),
        ];
        store.append_logs(&entries).unwrap();

        // Truncate at index 3
        let log_id: RaftLogId = LogId {
            leader_id: openraft::LeaderId::new(1, 1),
            index: 3,
        };
        store.truncate(log_id).await.unwrap();

        // Should have entries 1-3
        let state = store.get_log_state().await.unwrap();
        assert_eq!(state.last_log_id.unwrap().index, 3);
    }

    #[tokio::test]
    async fn test_redb_store_purge() {
        let temp_dir = TempDir::new().unwrap();
        let path = temp_dir.path().join("raft.redb");

        let mut store = RedbLogStore::new(&path).unwrap();

        // Append entries
        let entries = vec![
            create_entry(1, 1),
            create_entry(2, 1),
            create_entry(3, 1),
            create_entry(4, 1),
            create_entry(5, 1),
        ];
        store.append_logs(&entries).unwrap();

        // Purge up to index 3
        let log_id: RaftLogId = LogId {
            leader_id: openraft::LeaderId::new(1, 1),
            index: 3,
        };
        store.purge(log_id).await.unwrap();

        // Check last_purged
        let state = store.get_log_state().await.unwrap();
        assert_eq!(state.last_purged_log_id.unwrap().index, 3);

        // Entries 1-3 should be gone
        let entries = store.try_get_log_entries(1..=3).await.unwrap();
        assert!(entries.is_empty());

        // Entries 4-5 should still exist
        let entries = store.try_get_log_entries(4..=5).await.unwrap();
        assert_eq!(entries.len(), 2);
    }

    #[tokio::test]
    async fn test_redb_store_persistence() {
        let temp_dir = TempDir::new().unwrap();
        let path = temp_dir.path().join("raft.redb");

        // Write some data
        {
            let mut store = RedbLogStore::new(&path).unwrap();
            let entries = vec![create_entry(1, 1), create_entry(2, 1)];
            store.append_logs(&entries).unwrap();

            let vote: RaftVote = Vote::new(2, 1);
            store.save_vote(&vote).await.unwrap();
        }

        // Reopen and verify
        {
            let mut store = RedbLogStore::new(&path).unwrap();

            // Check entries persisted
            let state = store.get_log_state().await.unwrap();
            assert_eq!(state.last_log_id.unwrap().index, 2);

            // Check vote persisted
            let vote = store.read_vote().await.unwrap();
            assert_eq!(vote.unwrap().leader_id().term, 2);
        }
    }
}