Documentation
¶
Index ¶
- type DiskStorage
- func (w *DiskStorage) Checkpoint() (uint64, error)
- func (w *DiskStorage) Close() error
- func (w *DiskStorage) CreateSnapshot(i uint64, cs *raftpb.ConfState, data []byte) error
- func (w *DiskStorage) Entries(lo, hi, maxSize uint64) (es []raftpb.Entry, rerr error)
- func (w *DiskStorage) FirstIndex() (uint64, error)
- func (w *DiskStorage) HardState() (raftpb.HardState, error)
- func (w *DiskStorage) InitialState() (hs raftpb.HardState, cs raftpb.ConfState, err error)
- func (w *DiskStorage) LastIndex() (uint64, error)
- func (w *DiskStorage) NumEntries() int
- func (w *DiskStorage) NumLogFiles() int
- func (w *DiskStorage) Save(h *raftpb.HardState, es []raftpb.Entry, snap *raftpb.Snapshot) error
- func (w *DiskStorage) SetUint(info MetaInfo, id uint64)
- func (w *DiskStorage) Snapshot() (raftpb.Snapshot, error)
- func (w *DiskStorage) Sync() error
- func (w *DiskStorage) Term(idx uint64) (uint64, error)
- func (w *DiskStorage) TruncateEntriesUntil(lastIdx uint64)
- func (w *DiskStorage) Uint(info MetaInfo) uint64
- type MetaInfo
Constants ¶
This section is empty.
Variables ¶
This section is empty.
Functions ¶
This section is empty.
Types ¶
type DiskStorage ¶
type DiskStorage struct {
// contains filtered or unexported fields
}
DiskStorage handles disk access and writing for the RAFT write-ahead log. Dir contains wal.meta file and <start idx zero padded>.wal files.
=== wal.meta file === This file is generally around 4KB, so it can fit nicely in one Linux page.
Layout:
00-08 Bytes: Raft ID 08-16 Bytes: Group ID 16-24 Bytes: Checkpoint Index 512 Bytes: Hard State (Marshalled) 1024-1032 Bytes: Snapshot Index 1032-1040 Bytes: Snapshot Term 1040 Bytes: Snapshot (Marshalled)
--- <0000i>.wal files --- These files contain raftpb.Entry protos. Each entry is composed of term, index, type and data.
Term takes 8 bytes. Index takes 8 bytes. Type takes 8 bytes. And for data, we store an offset to the actual slice, which is 8 bytes. Size of entry = 32 bytes. First 30K entries would consume 960KB, hence fitting on the first MB of the file (logFileOffset).
Pre-allocate 1MB in each file just for these entries, and zero them out explicitly. Zeroing them out ensures that we know when these entries end, in case of a restart.
And the data for these entries are laid out starting logFileOffset. Those are the offsets you store in the Entry for Data field. After 30K entries, we rotate the file.
--- clean up --- If snapshot idx = Idx_s. We find the first log file whose first entry is less than Idx_s. This file and anything above MUST be kept. All the log files lower than this file can be deleted.
--- sync --- mmap fares well with process crashes without doing anything. In case HardSync is set, msync is called after every write, which flushes those writes to disk.
func Init ¶
func Init(dir string) *DiskStorage
Init initializes an instance of DiskStorage without encryption.
func InitEncrypted ¶
func InitEncrypted(dir string, encKey x.Sensitive) (*DiskStorage, error)
InitEncrypted initializes returns a properly initialized instance of DiskStorage. To gracefully shutdown DiskStorage, store.Closer.SignalAndWait() should be called.
func (*DiskStorage) Checkpoint ¶
func (w *DiskStorage) Checkpoint() (uint64, error)
Checkpoint returns the Raft index corresponding to the checkpoint.
func (*DiskStorage) CreateSnapshot ¶
CreateSnapshot generates a snapshot with the given ConfState and data and writes it to disk.
func (*DiskStorage) Entries ¶
func (w *DiskStorage) Entries(lo, hi, maxSize uint64) (es []raftpb.Entry, rerr error)
Entries returns a slice of log entries in the range [lo,hi). MaxSize limits the total size of the log entries returned, but Entries returns at least one entry if any.
func (*DiskStorage) FirstIndex ¶
func (w *DiskStorage) FirstIndex() (uint64, error)
FirstIndex returns the first index. It is typically SnapshotIndex+1.
func (*DiskStorage) InitialState ¶
InitialState returns the saved HardState and ConfState information.
func (*DiskStorage) LastIndex ¶
func (w *DiskStorage) LastIndex() (uint64, error)
func (*DiskStorage) NumEntries ¶
func (w *DiskStorage) NumEntries() int
func (*DiskStorage) NumLogFiles ¶
func (w *DiskStorage) NumLogFiles() int
func (*DiskStorage) Save ¶
TODO(Aman): In the raft example here, we store the snapshot first, followed by entries and then, hard state. https://github.com/etcd-io/etcd/blob/main/contrib/raftexample/raft.go We should do the same here. Save would write Entries, HardState and Snapshot to persistent storage in order, i.e. Entries first, then HardState and Snapshot if they are not empty. If persistent storage supports atomic writes then all of them can be written together. Note that when writing an Entry with Index i, any previously-persisted entries with Index >= i must be discarded.
func (*DiskStorage) SetUint ¶
func (w *DiskStorage) SetUint(info MetaInfo, id uint64)
func (*DiskStorage) Snapshot ¶
func (w *DiskStorage) Snapshot() (raftpb.Snapshot, error)
Snapshot returns the most recent snapshot. If snapshot is temporarily unavailable, it should return ErrSnapshotTemporarilyUnavailable, so raft state machine could know that Storage needs some time to prepare snapshot and call Snapshot later.
func (*DiskStorage) Sync ¶
func (w *DiskStorage) Sync() error
Sync calls the Sync method in the underlying badger instance to write all the contents to disk.
func (*DiskStorage) TruncateEntriesUntil ¶
func (w *DiskStorage) TruncateEntriesUntil(lastIdx uint64)
truncateEntriesUntil deletes the data field of every raft entry of type EntryNormal and index ∈ [0, lastIdx).
func (*DiskStorage) Uint ¶
func (w *DiskStorage) Uint(info MetaInfo) uint64