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 ofStateMachine
implementations are not persisted with this interface.A
RaftStore
implementation can implementRaftNodeLifecycleAware
to perform initialization and clean up work duringRaftNode
startup and termination.RaftNode
callsRaftNodeLifecycleAware.onRaftNodeStart()
before calling any other method onRaftStore
, and finally callsRaftNodeLifecycleAware.onRaftNodeTerminate()
on termination.- See Also:
RaftModel
,RaftModelFactory
,RaftNode
-
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description void
flush()
Forces all buffered (in any layer) Raft log changes to be written to the storage and returns after those changes are written.void
persistAndFlushInitialGroupMembers(RaftGroupMembersView initialGroupMembers)
Persists and flushes the given initial Raft group members.void
persistAndFlushLocalEndpoint(RaftEndpoint localEndpoint, boolean localEndpointVoting)
Persists and flushes the given local Raft endpoint and its voting flag.void
persistAndFlushTerm(int term, RaftEndpoint votedFor)
Persists the term and the Raft endpoint that the local Raft node voted for in the given term.void
persistLogEntry(LogEntry logEntry)
Persists the given log entry.void
persistSnapshotChunk(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
truncateSnapshotChunksUntil(long logIndexInclusive)
Rolls back the persisted snapshot chunks only when all of the expected snapshot chunks are not already persisted.
-
-
-
Method Detail
-
persistAndFlushLocalEndpoint
void persistAndFlushLocalEndpoint(RaftEndpoint localEndpoint, boolean localEndpointVoting) 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:
localEndpoint
- the Raft endpoint of the local Raft node to persistlocalEndpointVoting
- the flag that denotes whether if the local Raft node is a voting or non-voting member- Throws:
IOException
- if any failure occurs during persisting the given values
-
persistAndFlushInitialGroupMembers
void persistAndFlushInitialGroupMembers(@Nonnull 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(int term, @Nullable RaftEndpoint votedFor) 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:
term
- the term value to persistvotedFor
- the voted Raft endpoint to persist- Throws:
IOException
- if any failure occurs during persisting the given values
-
persistLogEntry
void persistLogEntry(@Nonnull LogEntry logEntry) throws IOException
Persists the given log entry.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, Raft must delete a range of the highest entries, rolling back the index of the next persisted entry. Consider the following case where Raft persists 3 log entries and then deletes entries from index=2:
- persistLogEntry(1)
- persistLogEntry(2)
- persistLogEntry(3)
- truncateLogEntriesFrom(2)
- Parameters:
logEntry
- the log entry object to persist- Throws:
IOException
- if any failure occurs during persisting the given log entry- See Also:
flush()
,persistSnapshotChunk(SnapshotChunk)
,truncateLogEntriesFrom(long)
,RaftConfig
-
persistSnapshotChunk
void persistSnapshotChunk(@Nonnull 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, andflush()
could be called afterwards.After a snapshot is persisted at index=i and
flush()
is called, the log entry at index=i, all the preceding log entries, and all the preceding snapshots are no longer needed and can be evicted from storage. Failing to evict stale entries and snapshots do not cause a consistency problem, but can increase the time to recover after a crash or restart. Therefore eviction can be done in a background task.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 that 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 apersistSnapshot()
call is made with snapshotIndex=100, the index of the precedingpersistLogEntry()
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:
flush()
,persistLogEntry(LogEntry)
,RaftConfig
-
truncateLogEntriesFrom
void truncateLogEntriesFrom(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:
flush()
,persistLogEntry(LogEntry)
,RaftConfig
-
truncateSnapshotChunksUntil
void truncateSnapshotChunksUntil(long logIndexInclusive) throws IOException
Rolls back the persisted snapshot chunks only when all of the expected snapshot chunks are not already persisted. A truncated snapshot chunk is no longer valid and must not be restored (or at least must be ignored during the restore process).- Parameters:
logIndexInclusive
- the log index value until which the log entries must be truncated- Throws:
IOException
- if any failure occurs during truncating the log entries- See Also:
persistSnapshotChunk(SnapshotChunk)
-
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 done via the other methods have become durable.
- Throws:
IOException
- if any failure occurs during the flush operation- See Also:
persistLogEntry(LogEntry)
,persistSnapshotChunk(SnapshotChunk)
,truncateSnapshotChunksUntil(long)
,truncateLogEntriesFrom(long)
-
-