Simplify raft snapshotting, entry apply.
parent
0bcec78527
commit
057309fc8e
|
@ -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.
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
|
|
284
raft/log.go
284
raft/log.go
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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{} }
|
||||
|
|
Loading…
Reference in New Issue