Trait openraft::storage::RaftLogStorage

source ·
pub trait RaftLogStorage<C>: Sealed + RaftLogReader<C> + OptionalSend + OptionalSync + 'static + Send
where C: RaftTypeConfig,
{ type LogReader: RaftLogReader<C>; // Required methods fn get_log_state( &mut self ) -> impl Future<Output = Result<LogState<C>, StorageError<C::NodeId>>> + Send; fn get_log_reader(&mut self) -> impl Future<Output = Self::LogReader> + Send; fn save_vote( &mut self, vote: &Vote<C::NodeId> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send; fn read_vote( &mut self ) -> impl Future<Output = Result<Option<Vote<C::NodeId>>, StorageError<C::NodeId>>> + Send; fn append<I>( &mut self, entries: I, callback: LogFlushed<C> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send where I: IntoIterator<Item = C::Entry> + OptionalSend, I::IntoIter: OptionalSend; fn truncate( &mut self, log_id: LogId<C::NodeId> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send; fn purge( &mut self, log_id: LogId<C::NodeId> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send; // Provided methods fn save_committed( &mut self, _committed: Option<LogId<C::NodeId>> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send { ... } fn read_committed( &mut self ) -> impl Future<Output = Result<Option<LogId<C::NodeId>>, StorageError<C::NodeId>>> + Send { ... } }
Expand description

API for log store.

vote API are also included because in raft, vote is part to the log: vote is about when, while log is about what. A distributed consensus is about at what a time, happened what a event.

§To ensure correctness:

  • Logs must be consecutive, i.e., there must NOT leave a hole in logs.
  • All write-IO must be serialized, i.e., the internal implementation must NOT apply a latter write request before a former write request is completed. This rule applies to both vote and log IO. E.g., Saving a vote and appending a log entry must be serialized too.

Required Associated Types§

source

type LogReader: RaftLogReader<C>

Log reader type.

Log reader is used by multiple replication tasks, which read logs and send them to remote nodes.

Required Methods§

source

fn get_log_state( &mut self ) -> impl Future<Output = Result<LogState<C>, StorageError<C::NodeId>>> + Send

Returns the last deleted log id and the last log id.

The impl should not consider the applied log id in state machine. The returned last_log_id could be the log id of the last present log entry, or the last_purged_log_id if there is no entry at all.

source

fn get_log_reader(&mut self) -> impl Future<Output = Self::LogReader> + Send

Get the log reader.

The method is intentionally async to give the implementation a chance to use asynchronous primitives to serialize access to the common internal object, if needed.

source

fn save_vote( &mut self, vote: &Vote<C::NodeId> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send

Save vote to storage.

§To ensure correctness:

The vote must be persisted on disk before returning.

source

fn read_vote( &mut self ) -> impl Future<Output = Result<Option<Vote<C::NodeId>>, StorageError<C::NodeId>>> + Send

Return the last saved vote by Self::save_vote.

source

fn append<I>( &mut self, entries: I, callback: LogFlushed<C> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send

Append log entries and call the callback once logs are persisted on disk.

It should returns immediately after saving the input log entries in memory, and calls the callback when the entries are persisted on disk, i.e., avoid blocking.

This method is still async because preparing the IO is usually async.

§To ensure correctness:
  • When this method returns, the entries must be readable, i.e., a LogReader can read these entries.

  • When the callback is called, the entries must be persisted on disk.

    NOTE that: the callback can be called either before or after this method returns.

  • There must not be a hole in logs. Because Raft only examine the last log id to ensure correctness.

source

fn truncate( &mut self, log_id: LogId<C::NodeId> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send

Truncate logs since log_id, inclusive

§To ensure correctness:
  • It must not leave a hole in logs.
source

fn purge( &mut self, log_id: LogId<C::NodeId> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send

Purge logs upto log_id, inclusive

§To ensure correctness:
  • It must not leave a hole in logs.

Provided Methods§

source

fn save_committed( &mut self, _committed: Option<LogId<C::NodeId>> ) -> impl Future<Output = Result<(), StorageError<C::NodeId>>> + Send

Saves the last committed log id to storage.

§Optional feature

If the state machine flushes state to disk before returning from apply(), then the application does not need to implement this method. Otherwise, this method is also optional(but not recommended), but your application has to deal with state reversion of state machine carefully upon restart. E.g., do not serve read operation a new commit message is received.

See: docs::data::log_pointers.

source

fn read_committed( &mut self ) -> impl Future<Output = Result<Option<LogId<C::NodeId>>, StorageError<C::NodeId>>> + Send

Return the last saved committed log id by Self::save_committed.

Object Safety§

This trait is not object safe.

Implementors§