Interface RaftStore

All Known Implementing Classes:
NopRaftStore

public interface RaftStore
This interface is used for persisting only the internal state of the Raft consensus algorithm. Internal state of StateMachine implementations are not persisted with this interface.

A RaftStore implementation can implement RaftNodeLifecycleAware to perform initialization and clean up work during RaftNode startup and termination. RaftNode calls RaftNodeLifecycleAware.onRaftNodeStart() before calling any other method on RaftStore, and finally calls RaftNodeLifecycleAware.onRaftNodeTerminate() on termination.

See Also:
  • Method Summary

    Modifier and Type
    Method
    Description
    void
    deleteSnapshotChunks(long logIndex, int snapshotChunkCount)
    Deletes persisted snapshot chunks at the given log index.
    void
    Forces all buffered (in any layer) Raft log changes to be written to the storage and returns after those changes are written.
    void
    persistAndFlushInitialGroupMembers(io.microraft.model.log.RaftGroupMembersView initialGroupMembers)
    Persists and flushes the given initial Raft group members.
    void
    persistAndFlushLocalEndpoint(io.microraft.model.persistence.RaftEndpointPersistentState localEndpointPersistentState)
    Persists and flushes the given local Raft endpoint and its voting flag.
    void
    persistAndFlushTerm(io.microraft.model.persistence.RaftTermPersistentState termPersistentState)
    Persists the term and the Raft endpoint that the local Raft node voted for in the given term.
    void
    persistLogEntries(List<io.microraft.model.log.LogEntry> logEntries)
    Persists the given log entries.
    void
    persistSnapshotChunk(io.microraft.model.log.SnapshotChunk snapshotChunk)
    Persists the given snapshot chunk.
    void
    truncateLogEntriesFrom(long logIndexInclusive)
    Rolls back the log by truncating all entries starting with the given index.
    void
    truncateLogEntriesUntil(long logIndexInclusive)
    MicroRaft calls this method after it successfully persists and flushes a snapshot.
  • Method Details

    • persistAndFlushLocalEndpoint

      void persistAndFlushLocalEndpoint(@Nonnull io.microraft.model.persistence.RaftEndpointPersistentState localEndpointPersistentState) throws IOException
      Persists and flushes the given local Raft endpoint and its voting flag.

      When this method returns, all the provided data has become durable.

      Parameters:
      localEndpointPersistentState - the local endpoint state to be persisted
      Throws:
      IOException - if any failure occurs during persisting the given values
    • persistAndFlushInitialGroupMembers

      void persistAndFlushInitialGroupMembers(@Nonnull io.microraft.model.log.RaftGroupMembersView initialGroupMembers) throws IOException
      Persists and flushes the given initial Raft group members.

      When this method returns, all the provided data has become durable.

      Parameters:
      initialGroupMembers - the initial Raft group member list to persist
      Throws:
      IOException - if any failure occurs during persisting the given values
    • persistAndFlushTerm

      void persistAndFlushTerm(@Nonnull io.microraft.model.persistence.RaftTermPersistentState termPersistentState) throws IOException
      Persists the term and the Raft endpoint that the local Raft node voted for in the given term.

      When this method returns, all the provided data has become durable.

      Parameters:
      termPersistentState - the term state to be persisted
      Throws:
      IOException - if any failure occurs during persisting the given values
    • persistLogEntries

      void persistLogEntries(@Nonnull List<io.microraft.model.log.LogEntry> logEntries) throws IOException
      Persists the given log entries.

      Log entries are appended to the Raft log with sequential log indices. The first log index is 1.

      A block of consecutive log entries has no gaps in the indices, but a gap can appear between a snapshot entry and its preceding regular log entry. This happens in an edge case where a follower has fallen so far behind that the missing entries are no longer available from the leader. In that case the leader will send its snapshot entry instead.

      In another rare failure scenario, MicroRaft can delete a range of the latest entries which are uncommitted and roll back to a previous log index which is known to be committed. Consider the following case where Raft persists 3 log entries and then deletes entries from index=2:

      • persistLogEntries(1, 2, 3)
      • truncateLogEntriesFrom(2)
      After this call sequence log indices will remain sequential and the next persistLogEntries() call will be for index=2.
      Parameters:
      logEntries - the list of log entries to be persisted
      Throws:
      IOException - if any failure occurs during persisting the given log entry
      See Also:
    • persistSnapshotChunk

      void persistSnapshotChunk(@Nonnull io.microraft.model.log.SnapshotChunk snapshotChunk) throws IOException
      Persists the given snapshot chunk.

      A snapshot is persisted with at least 1 chunk. The number of chunks in a snapshot is provided via SnapshotChunk.getSnapshotChunkCount(). A snapshot is considered to be complete when all of its chunks are provided to this method in any order, and flush() will be called afterwards.

      MicroRaft takes snapshots at a predetermined interval, controlled by RaftConfig.getCommitCountToTakeSnapshot(). For instance, if it is 100, snapshots will occur at indices 100, 200, 300, and so on.

      The snapshot index can lag behind the index of the highest log entry which was already persisted and flushed, but there is an upper bound to this difference, controlled by RaftConfig.getMaxPendingLogEntryCount(). For instance, if it is 10, and a persistSnapshot() call is made with snapshotIndex=100, the index of the preceding persistLogEntries() call can be at most 110.

      On the other hand, the snapshot index can also be ahead of the highest log entry. This can happen when a Raft follower has fallen so far behind the leader and the leader no longer holds the missing entries. In that case, the follower receives a snapshot from the leader. There is no upper-bound on the gap between the highest log entry and the index of the received snapshot.

      Parameters:
      snapshotChunk - the snapshot chunk object to persist
      Throws:
      IOException - if any failure occurs during persisting the given snapshot chunk
      See Also:
    • truncateLogEntriesFrom

      void truncateLogEntriesFrom(@Nonnegative long logIndexInclusive) throws IOException
      Rolls back the log by truncating all entries starting with the given index. A truncated log entry is no longer valid and must not be restored (or at least must be ignored during the restore process).

      There is an upper-bound on the number of persisted log entries that can be truncated afterwards, which is specified by RaftConfig.getMaxPendingLogEntryCount() + 1. Say that it is 5 and the highest persisted log entry index is 20. Then, at most 5 highest entries can be truncated, hence truncation can start at index=16 or higher.

      Parameters:
      logIndexInclusive - the log index value from which the log entries must be truncated
      Throws:
      IOException - if any failure occurs during truncating the log entries
      See Also:
    • truncateLogEntriesUntil

      void truncateLogEntriesUntil(@Nonnegative long logIndexInclusive) throws IOException
      MicroRaft calls this method after it successfully persists and flushes a snapshot. This method is used for deleting log entries and snapshot chunks that are before the snapshot index and are not needed to be restored. This is merely an optimization method and its side-effects can be sync'ed to the storage when flush() is called,
      Parameters:
      logIndexInclusive - the log index value until which the log entries and snapshot chunks must be truncated
      Throws:
      IOException - if any failure occurs during truncating the log entries
    • deleteSnapshotChunks

      void deleteSnapshotChunks(@Nonnegative long logIndex, @Nonnegative int snapshotChunkCount) throws IOException
      Deletes persisted snapshot chunks at the given log index. MicroRaft calls this method when it detects that it needs to start installing a newer snapshot while there is a snapshot persisted partially. Those snapshot chunks are no longer valid and must not be restored (or at least must be ignored during the restore process). This is merely an optimization method and its side-effects can be sync'ed to the storage when flush() is called.
      Parameters:
      logIndex - the log index value at which some snapshot chunks are persisted.
      snapshotChunkCount - the number of snapshot chunks that could have been persisted.
      Throws:
      IOException - if any failure occurs during deletion.
    • flush

      void flush() throws IOException
      Forces all buffered (in any layer) Raft log changes to be written to the storage and returns after those changes are written.

      When this method returns, all the changes previously done via the other methods have become durable.

      Throws:
      IOException - if any failure occurs during the flush operation
      See Also: