Simplify raft snapshotting, entry apply.

pull/2111/head
Ben Johnson 2015-03-26 20:32:39 -06:00
parent 0bcec78527
commit 057309fc8e
5 changed files with 208 additions and 220 deletions

View File

@ -110,10 +110,10 @@ func (b *Broker) Topic(id uint64) *Topic {
// Index returns the highest index seen by the broker across all topics.
// Returns 0 if the broker is closed.
func (b *Broker) Index() (uint64, error) {
func (b *Broker) Index() uint64 {
b.mu.RLock()
defer b.mu.RUnlock()
return b.index, nil
return b.index
}
// opened returns true if the broker is in an open and running state.
@ -256,8 +256,8 @@ func (b *Broker) setMaxIndex(index uint64) error {
return nil
}
// Snapshot streams the current state of the broker and returns the index.
func (b *Broker) Snapshot(w io.Writer) (uint64, error) {
// WriteTo writes a snapshot of the broker to w.
func (b *Broker) WriteTo(w io.Writer) (int64, error) {
// TODO: Prevent truncation during snapshot.
// Calculate header under lock.
@ -291,8 +291,7 @@ func (b *Broker) Snapshot(w io.Writer) (uint64, error) {
}
}
// Return the snapshot and its last applied index.
return hdr.Index, nil
return 0, nil
}
// createSnapshotHeader creates a snapshot header.
@ -352,32 +351,32 @@ func copyFileN(w io.Writer, path string, n int64) (int64, error) {
return io.CopyN(w, f, n)
}
// Restore reads the broker state.
func (b *Broker) Restore(r io.Reader) error {
// ReadFrom reads a broker snapshot from r.
func (b *Broker) ReadFrom(r io.Reader) (int64, error) {
b.mu.Lock()
defer b.mu.Unlock()
// Remove and recreate broker path.
if err := b.reset(); err != nil && !os.IsNotExist(err) {
return fmt.Errorf("reset: %s", err)
return 0, fmt.Errorf("reset: %s", err)
} else if err = os.MkdirAll(b.path, 0777); err != nil {
return fmt.Errorf("mkdir: %s", err)
return 0, fmt.Errorf("mkdir: %s", err)
}
// Read header frame.
var sz uint32
if err := binary.Read(r, binary.BigEndian, &sz); err != nil {
return fmt.Errorf("read header size: %s", err)
return 0, fmt.Errorf("read header size: %s", err)
}
buf := make([]byte, sz)
if _, err := io.ReadFull(r, buf); err != nil {
return fmt.Errorf("read header: %s", err)
return 0, fmt.Errorf("read header: %s", err)
}
// Decode header.
sh := &snapshotHeader{}
if err := json.Unmarshal(buf, &sh); err != nil {
return fmt.Errorf("decode header: %s", err)
return 0, fmt.Errorf("decode header: %s", err)
}
// Close any topics which might be open and clear them out.
@ -389,7 +388,7 @@ func (b *Broker) Restore(r io.Reader) error {
// Create topic directory.
if err := os.MkdirAll(t.Path(), 0777); err != nil {
return fmt.Errorf("make topic dir: %s", err)
return 0, fmt.Errorf("make topic dir: %s", err)
}
// Copy data from snapshot into segment files.
@ -411,24 +410,24 @@ func (b *Broker) Restore(r io.Reader) error {
return nil
}(); err != nil {
return err
return 0, err
}
}
// Open topic.
if err := t.Open(); err != nil {
return fmt.Errorf("open topic: %s", err)
return 0, fmt.Errorf("open topic: %s", err)
}
b.topics[t.id] = t
}
// Set the highest seen index.
if err := b.setMaxIndex(sh.Index); err != nil {
return fmt.Errorf("set max index: %s", err)
return 0, fmt.Errorf("set max index: %s", err)
}
b.index = sh.Index
return nil
return 0, nil
}
// reset removes all files in the broker directory besides the raft directory.
@ -570,20 +569,21 @@ type snapshotTopicSegment struct {
// It will panic for any errors that occur during Apply.
type RaftFSM struct {
Broker interface {
io.WriterTo
io.ReaderFrom
Apply(m *Message) error
Index() (uint64, error)
Index() uint64
SetMaxIndex(uint64) error
Snapshot(w io.Writer) (uint64, error)
Restore(r io.Reader) error
}
}
func (fsm *RaftFSM) Index() (uint64, error) { return fsm.Broker.Index() }
func (fsm *RaftFSM) Snapshot(w io.Writer) (uint64, error) { return fsm.Broker.Snapshot(w) }
func (fsm *RaftFSM) Restore(r io.Reader) error { return fsm.Broker.Restore(r) }
func (fsm *RaftFSM) Index() uint64 { return fsm.Broker.Index() }
func (fsm *RaftFSM) WriteTo(w io.Writer) (n int64, err error) { return fsm.Broker.WriteTo(w) }
func (fsm *RaftFSM) ReadFrom(r io.Reader) (n int64, err error) { return fsm.Broker.ReadFrom(r) }
// MustApply applies a raft command to the broker. Panic on error.
func (fsm *RaftFSM) MustApply(e *raft.LogEntry) {
// Apply applies a raft command to the broker.
func (fsm *RaftFSM) Apply(e *raft.LogEntry) error {
switch e.Type {
case raft.LogEntryCommand:
// Decode message.
@ -595,15 +595,17 @@ func (fsm *RaftFSM) MustApply(e *raft.LogEntry) {
// Apply message.
if err := fsm.Broker.Apply(m); err != nil {
panic(err.Error())
return fmt.Errorf("broker apply: %s", err)
}
default:
// Move internal index forward if it's an internal raft comand.
if err := fsm.Broker.SetMaxIndex(e.Index); err != nil {
panic(fmt.Sprintf("set max index: idx=%d, err=%s", e.Index, err))
return fmt.Errorf("set max index: idx=%d, err=%s", e.Index, err)
}
}
return nil
}
// DefaultMaxSegmentSize is the largest a segment can get before starting a new segment.

View File

@ -103,7 +103,7 @@ func TestBroker_Apply(t *testing.T) {
}
// Verify broker high water mark.
if index, _ := b.Index(); index != 4 {
if index := b.Index(); index != 4 {
t.Fatalf("unexpected broker index: %d", index)
}
}
@ -158,7 +158,7 @@ func TestBroker_Reopen(t *testing.T) {
}
// Verify broker high water mark.
if index, _ := b.Index(); index != 4 {
if index := b.Index(); index != 4 {
t.Fatalf("unexpected broker index: %d", index)
}
@ -188,16 +188,14 @@ func TestBroker_Snapshot(t *testing.T) {
// Snapshot the first broker.
var buf bytes.Buffer
if index, err := b0.Snapshot(&buf); err != nil {
if _, err := b0.WriteTo(&buf); err != nil {
t.Fatalf("snapshot error: %s", err)
} else if index != 4 {
t.Fatalf("unexpected snapshot index: %d", index)
}
// Restore to the second broker.
b1 := OpenBroker()
defer b1.Close()
if err := b1.Restore(&buf); err != nil {
if _, err := b1.ReadFrom(&buf); err != nil {
t.Fatalf("restore error: %s", err)
}
@ -224,7 +222,7 @@ func TestBroker_Snapshot(t *testing.T) {
}
// Verify broker high water mark.
if index, _ := b1.Index(); index != 4 {
if index := b1.Index(); index != 4 {
t.Fatalf("unexpected broker index: %d", index)
}
}
@ -268,8 +266,9 @@ func TestRaftFSM_MustApply_Message(t *testing.T) {
// Encode message and apply it as a log entry.
m := messaging.Message{TopicID: 20}
data, _ := m.MarshalBinary()
fsm.MustApply(&raft.LogEntry{Index: 2, Data: data})
if !called {
if err := fsm.Apply(&raft.LogEntry{Index: 2, Data: data}); err != nil {
t.Fatal(err)
} else if !called {
t.Fatal("Apply() not called")
}
}
@ -289,8 +288,9 @@ func TestRaftFSM_MustApply_Internal(t *testing.T) {
}
// Encode message and apply it as a log entry.
fsm.MustApply(&raft.LogEntry{Type: raft.LogEntryAddPeer, Index: 2})
if !called {
if err := fsm.Apply(&raft.LogEntry{Type: raft.LogEntryAddPeer, Index: 2}); err != nil {
t.Fatal(err)
} else if !called {
t.Fatal("Apply() not called")
}
}
@ -318,9 +318,9 @@ type RaftFSMBroker struct {
func (b *RaftFSMBroker) Apply(m *messaging.Message) error { return b.ApplyFunc(m) }
func (b *RaftFSMBroker) SetMaxIndex(index uint64) error { return b.SetMaxIndexFunc(index) }
func (b *RaftFSMBroker) Index() (uint64, error) { return 0, nil }
func (b *RaftFSMBroker) Snapshot(w io.Writer) (uint64, error) { return 0, nil }
func (b *RaftFSMBroker) Restore(r io.Reader) error { return nil }
func (b *RaftFSMBroker) Index() uint64 { return 0 }
func (b *RaftFSMBroker) WriteTo(w io.Writer) (n int64, err error) { return 0, nil }
func (b *RaftFSMBroker) ReadFrom(r io.Reader) (n int64, err error) { return 0, nil }
// Ensure a list of topics can be read from a directory.
func TestReadTopics(t *testing.T) {

View File

@ -48,19 +48,22 @@ type IndexFSM struct {
}
// MustApply updates the index.
func (fsm *IndexFSM) MustApply(entry *LogEntry) { fsm.index = entry.Index }
// Index returns the highest applied index.
func (fsm *IndexFSM) Index() (uint64, error) { return fsm.index, nil }
// Snapshot writes the FSM's index as the snapshot.
func (fsm *IndexFSM) Snapshot(w io.Writer) (uint64, error) {
return fsm.index, binary.Write(w, binary.BigEndian, fsm.index)
func (fsm *IndexFSM) Apply(entry *LogEntry) error {
fsm.index = entry.Index
return nil
}
// Restore reads the snapshot from the reader.
func (fsm *IndexFSM) Restore(r io.Reader) error {
return binary.Read(r, binary.BigEndian, &fsm.index)
// Index returns the highest applied index.
func (fsm *IndexFSM) Index() uint64 { return fsm.index }
// WriteTo writes a snapshot of the FSM to w.
func (fsm *IndexFSM) WriteTo(w io.Writer) (n int64, err error) {
return 0, binary.Write(w, binary.BigEndian, fsm.index)
}
// ReadFrom reads an FSM snapshot from r.
func (fsm *IndexFSM) ReadFrom(r io.Reader) (n int64, err error) {
return 0, binary.Read(r, binary.BigEndian, &fsm.index)
}
// tempfile returns the path to a non-existent file in the temp directory.

View File

@ -25,19 +25,15 @@ import (
// FSM represents the state machine that the log is applied to.
// The FSM must maintain the highest index that it has seen.
type FSM interface {
io.WriterTo
io.ReaderFrom
// Executes a log entry against the state machine.
// Non-repeatable errors such as system and disk errors must panic.
MustApply(*LogEntry)
Apply(*LogEntry) error
// Returns the highest index saved to the state machine.
Index() (uint64, error)
// Writes a snapshot of the entire state machine to a writer.
// Returns the index at the point in time of the snapshot.
Snapshot(w io.Writer) (index uint64, err error)
// Reads a snapshot of the entire state machine.
Restore(r io.Reader) error
// Returns the applied index saved to the state machine.
Index() uint64
}
const logEntryHeaderSize = 8 + 8 + 8 // sz+index+term
@ -91,8 +87,7 @@ type Log struct {
votedFor uint64 // candidate voted for in current election term
lastContact time.Time // last contact from the leader
commitIndex uint64 // highest entry to be committed
appliedIndex uint64 // highest entry to applied to state machine
commitIndex uint64 // highest entry to be committed
reader io.ReadCloser // incoming stream from leader
writers []*logWriter // outgoing streams to followers
@ -231,13 +226,6 @@ func (l *Log) CommitIndex() uint64 {
return l.commitIndex
}
// AppliedIndex returns the highest applied index.
func (l *Log) AppliedIndex() uint64 {
l.mu.Lock()
defer l.mu.Unlock()
return l.appliedIndex
}
// Term returns the current term.
func (l *Log) Term() uint64 {
l.mu.Lock()
@ -299,13 +287,9 @@ func (l *Log) Open(path string) error {
l.config = c
// Determine last applied index from FSM.
index, err := l.FSM.Index()
if err != nil {
return err
}
index := l.FSM.Index()
l.tracef("Open: fsm: index=%d", index)
l.lastLogIndex = index
l.appliedIndex = index
l.commitIndex = index
// Start goroutine to apply logs.
@ -1095,18 +1079,18 @@ func (l *Log) internalApply(typ LogEntryType, command []byte) (index uint64, err
}
// Wait blocks until a given index is applied.
func (l *Log) Wait(index uint64) error {
func (l *Log) Wait(idx uint64) error {
// TODO(benbjohnson): Check for leadership change (?).
// TODO(benbjohnson): Add timeout.
for {
l.mu.Lock()
state, appliedIndex := l.state, l.appliedIndex
state, index := l.state, l.FSM.Index()
l.mu.Unlock()
if state == Stopped {
return ErrClosed
} else if appliedIndex >= index {
} else if index >= idx {
return nil
}
time.Sleep(WaitInterval)
@ -1192,111 +1176,102 @@ func (l *Log) applier(closing <-chan struct{}) {
//l.tracef("applier")
// Apply all entries committed since the previous apply.
err := func() error {
l.mu.Lock()
defer l.mu.Unlock()
// Verify, under lock, that we're not closing.
select {
case <-closing:
return nil
default:
// Keep applying the next entry until there are no more committed
// entries that have not been applied to the state machine.
for {
if err := l.applyNextUnappliedEntry(closing); err == errDone {
break
} else if err != nil {
panic(err.Error())
}
// Ignore if there are no pending entries.
// Ignore if all entries are applied.
if len(l.entries) == 0 {
//l.tracef("applier: no entries")
return nil
} else if l.appliedIndex == l.commitIndex {
//l.tracef("applier: up to date")
return nil
}
// Determine the available range of indices on the log.
entmin, entmax := l.entries[0].Index, l.entries[len(l.entries)-1].Index
assert(entmin <= entmax, "apply: log out of order: min=%d, max=%d", entmin, entmax)
assert(uint64(len(l.entries)) == (entmax-entmin+1), "apply: missing entries: min=%d, max=%d, len=%d", entmin, entmax, len(l.entries))
// Determine the range of indices that should be processed.
// This should be the entry after the last applied index through to
// the committed index.
nextUnappliedIndex, commitIndex := l.appliedIndex+1, l.commitIndex
l.tracef("applier: entries: available=%d-%d, [next,commit]=%d-%d", entmin, entmax, nextUnappliedIndex, commitIndex)
assert(nextUnappliedIndex <= commitIndex, "next unapplied index after commit index: next=%d, commit=%d", nextUnappliedIndex, commitIndex)
// Determine the lowest index to start from.
// This should be the next entry after the last applied entry.
// Ignore if we don't have any entries after the last applied yet.
assert(entmin <= nextUnappliedIndex, "apply: missing entries: min=%d, next=%d", entmin, nextUnappliedIndex)
if nextUnappliedIndex > entmax {
return nil
}
imin := nextUnappliedIndex
// Determine the highest index to go to.
// This should be the committed index.
// If we haven't yet received the committed index then go to the last available.
var imax uint64
if commitIndex <= entmax {
imax = commitIndex
} else {
imax = entmax
}
// Determine entries to apply.
l.tracef("applier: entries: available=%d-%d, applying=%d-%d", entmin, entmax, imin, imax)
entries := l.entries[imin-entmin : imax-entmin+1]
// Determine low water mark for entries to cut off.
for _, w := range l.writers {
if w.snapshotIndex > 0 && w.snapshotIndex < imax {
imax = w.snapshotIndex
}
}
l.entries = l.entries[imax-entmin:]
// Iterate over each entry and apply it.
for _, e := range entries {
// l.tracef("applier: entry: idx=%d", e.Index)
switch e.Type {
case LogEntryCommand, LogEntryNop:
case LogEntryInitialize:
l.mustApplyInitialize(e)
case LogEntryAddPeer:
l.mustApplyAddPeer(e)
case LogEntryRemovePeer:
l.mustApplyRemovePeer(e)
default:
panic("unsupported command type: " + strconv.Itoa(int(e.Type)))
}
// Apply to FSM.
if e.Index > 0 {
l.FSM.MustApply(e)
}
// Increment applied index.
l.appliedIndex++
}
return nil
}()
// If error occurred then log it.
// The log will retry after a given timeout.
if err != nil {
l.Logger.Printf("apply error: %s", err)
// TODO(benbjohnson): Longer timeout before retry?
}
// Trim entries.
l.mu.Lock()
l.trim()
l.mu.Unlock()
// Signal clock that apply is done.
close(confirm)
}
}
// applyNextUnappliedEntry applies the next committed entry that has not yet been applied.
func (l *Log) applyNextUnappliedEntry(closing <-chan struct{}) error {
l.mu.Lock()
defer l.mu.Unlock()
// Verify, under lock, that we're not closing.
select {
case <-closing:
return errDone
default:
}
// Ignore if there are no entries in the log.
if len(l.entries) == 0 {
return errDone
}
// Determine next index to apply.
// Ignore if next index is after the commit index.
// Ignore if the entry is not streamed to the log yet.
index := l.FSM.Index() + 1
if index > l.commitIndex {
return errDone
} else if index > l.entries[len(l.entries)-1].Index {
return errDone
}
// Retrieve next entry.
e := l.entries[index-l.entries[0].Index]
assert(e.Index == index, "apply: index mismatch: %d != %d", e.Index, index)
// Special handling for internal log entries.
switch e.Type {
case LogEntryCommand, LogEntryNop:
case LogEntryInitialize:
l.mustApplyInitialize(e)
case LogEntryAddPeer:
l.mustApplyAddPeer(e)
case LogEntryRemovePeer:
l.mustApplyRemovePeer(e)
default:
return fmt.Errorf("unsupported command type: %d", e.Type)
}
// Apply to FSM.
if err := l.FSM.Apply(e); err != nil {
return fmt.Errorf("apply: %s", err)
}
return nil
}
// trim truncates the log based on the applied index and pending writers.
func (l *Log) trim() {
if len(l.entries) == 0 {
return
}
// Determine lowest index to trim to.
index := l.FSM.Index()
for _, w := range l.writers {
if w.snapshotIndex > 0 && w.snapshotIndex < index {
index = w.snapshotIndex
}
}
// Ignore if the index is lower than the first entry.
// This can occur on a new snapshot.
if index < l.entries[0].Index {
return
}
// Reslice entries list.
l.entries = l.entries[index-l.entries[0].Index:]
}
// mustApplyInitialize a log initialization command by parsing and setting the configuration.
func (l *Log) mustApplyInitialize(e *LogEntry) {
// Parse the configuration from the log entry.
@ -1492,19 +1467,18 @@ func (l *Log) writeTo(writer *logWriter, id, term, index uint64) error {
}
// Begin streaming the snapshot.
snapshotIndex, err := l.FSM.Snapshot(w)
if err != nil {
if _, err := l.FSM.WriteTo(w); err != nil {
return err
}
// Write snapshot index at the end and flush.
if err := binary.Write(w, binary.BigEndian, snapshotIndex); err != nil {
return fmt.Errorf("write snapshot index: %s", err)
}
flushWriter(w)
// // Write snapshot index at the end and flush.
// if err := binary.Write(w, binary.BigEndian, snapshotIndex); err != nil {
// return fmt.Errorf("write snapshot index: %s", err)
// }
// Write entries since the snapshot occurred and begin tailing writer.
if err := l.advanceWriter(writer, snapshotIndex); err != nil {
if err := l.advanceWriter(writer); err != nil {
return err
}
@ -1553,7 +1527,7 @@ func (l *Log) initWriter(w io.Writer, id, term, index uint64) (*logWriter, error
writer := &logWriter{
Writer: w,
id: id,
snapshotIndex: l.appliedIndex,
snapshotIndex: l.FSM.Index(),
done: make(chan struct{}),
}
l.writers = append(l.writers, writer)
@ -1562,7 +1536,7 @@ func (l *Log) initWriter(w io.Writer, id, term, index uint64) (*logWriter, error
}
// replays entries since the snapshot's index and begins tailing the log.
func (l *Log) advanceWriter(writer *logWriter, snapshotIndex uint64) error {
func (l *Log) advanceWriter(writer *logWriter) error {
l.mu.Lock()
defer l.mu.Unlock()
@ -1575,9 +1549,8 @@ func (l *Log) advanceWriter(writer *logWriter, snapshotIndex uint64) error {
// Write pending entries.
if len(l.entries) > 0 {
startIndex := l.entries[0].Index
enc := NewLogEntryEncoder(writer.Writer)
for _, e := range l.entries[snapshotIndex-startIndex+1:] {
for _, e := range l.entries[writer.snapshotIndex-l.entries[0].Index+1:] {
if err := enc.Encode(e); err != nil {
return err
}
@ -1663,25 +1636,34 @@ func (l *Log) ReadFrom(r io.ReadCloser) error {
if e.Type == logEntrySnapshot {
l.tracef("ReadFrom: snapshot")
if err := l.FSM.Restore(r); err != nil {
if err := func() error {
l.mu.Lock()
defer l.mu.Unlock()
if _, err := l.FSM.ReadFrom(r); err != nil {
return err
}
// Update the indicies & clear the entries.
index := l.FSM.Index()
l.lastLogIndex = index
l.commitIndex = index
l.entries = nil
return nil
}(); err != nil {
l.tracef("ReadFrom: restore error: %s", err)
return err
}
l.tracef("ReadFrom: snapshot: restored")
// Read the snapshot index off the end of the snapshot.
var index uint64
if err := binary.Read(r, binary.BigEndian, &index); err != nil {
return fmt.Errorf("read snapshot index: %s", err)
}
l.tracef("ReadFrom: snapshot: index=%d", index)
// Update the indicies & clear the entries.
l.mu.Lock()
l.lastLogIndex = index
l.commitIndex = index
l.appliedIndex = index
l.entries = nil
l.mu.Unlock()
// // Read the snapshot index off the end of the snapshot.
// var index uint64
// if err := binary.Read(r, binary.BigEndian, &index); err != nil {
// return fmt.Errorf("read snapshot index: %s", err)
// }
// l.tracef("ReadFrom: snapshot: index=%d", index)
continue
}

View File

@ -249,7 +249,7 @@ func TestCluster_Elect_RealTime(t *testing.T) {
// Create a cluster with a real-time clock.
c := NewRealTimeCluster(3, indexFSMFunc)
minIndex := c.Logs[0].AppliedIndex()
minIndex := c.Logs[0].FSM.Index()
commandN := uint64(1000) - minIndex
// Run a loop to continually apply commands.
@ -297,7 +297,7 @@ func TestCluster_Elect_RealTime(t *testing.T) {
// Verify FSM indicies match.
for i, l := range c.Logs {
fsmIndex, _ := l.FSM.(*raft.IndexFSM).Index()
fsmIndex := l.FSM.Index()
if exp := commandN + minIndex; exp != fsmIndex {
t.Errorf("fsm index mismatch(%d): exp=%d, got=%d", i, exp, fsmIndex)
}
@ -334,7 +334,7 @@ func benchmarkClusterApply(b *testing.B, logN int) {
// Verify FSM indicies match.
for i, l := range c.Logs {
fsmIndex, _ := l.FSM.(*raft.IndexFSM).Index()
fsmIndex := l.FSM.Index()
if index != fsmIndex {
b.Errorf("fsm index mismatch(%d): exp=%d, got=%d", i, index, fsmIndex)
}
@ -592,39 +592,40 @@ type FSM struct {
Commands [][]byte
}
// MustApply updates the max index and appends the command.
func (fsm *FSM) MustApply(entry *raft.LogEntry) {
// Apply updates the max index and appends the command.
func (fsm *FSM) Apply(entry *raft.LogEntry) error {
fsm.MaxIndex = entry.Index
if entry.Type == raft.LogEntryCommand {
fsm.Commands = append(fsm.Commands, entry.Data)
}
return nil
}
// Index returns the highest applied index.
func (fsm *FSM) Index() (uint64, error) { return fsm.MaxIndex, nil }
func (fsm *FSM) Index() uint64 { return fsm.MaxIndex }
// Snapshot begins writing the FSM to a writer.
func (fsm *FSM) Snapshot(w io.Writer) (uint64, error) {
// WriteTo writes a snapshot of the FSM to w.
func (fsm *FSM) WriteTo(w io.Writer) (n int64, err error) {
b, _ := json.Marshal(fsm)
binary.Write(w, binary.BigEndian, uint64(len(b)))
_, err := w.Write(b)
return fsm.MaxIndex, err
_, err = w.Write(b)
return 0, err
}
// Restore reads the snapshot from the reader.
func (fsm *FSM) Restore(r io.Reader) error {
// ReadFrom reads an FSM snapshot from r.
func (fsm *FSM) ReadFrom(r io.Reader) (n int64, err error) {
var sz uint64
if err := binary.Read(r, binary.BigEndian, &sz); err != nil {
return err
return 0, err
}
buf := make([]byte, sz)
if _, err := io.ReadFull(r, buf); err != nil {
return err
return 0, err
}
if err := json.Unmarshal(buf, &fsm); err != nil {
return err
return 0, err
}
return nil
return 0, nil
}
func fsmFunc() raft.FSM { return &FSM{} }