diff --git a/diff_test.go b/diff_test.go index 6030527941f4..9eef22f218b6 100644 --- a/diff_test.go +++ b/diff_test.go @@ -57,6 +57,7 @@ func mustTemp(pre, body string) string { func ltoa(l *raftLog) string { s := fmt.Sprintf("lastIndex: %d\n", l.lastIndex()) s += fmt.Sprintf("applied: %d\n", l.applied) + s += fmt.Sprintf("applying: %d\n", l.applying) for i, e := range l.allEntries() { s += fmt.Sprintf("#%d: %+v\n", i, e) } diff --git a/doc.go b/doc.go index 4febfe6084b8..06253f4e12c7 100644 --- a/doc.go +++ b/doc.go @@ -169,6 +169,94 @@ given ID MUST be used only once even if the old node has been removed. This means that for example IP addresses make poor node IDs since they may be reused. Node IDs must be non-zero. +# Usage with Asynchronous Storage Writes + +The library can be configured with an alternate interface for local storage +writes that can provide better performance in the presence of high proposal +concurrency by minimizing interference between proposals. This feature is called +AsynchronousStorageWrites, and can be enabled using the flag on the Config +struct with the same name. + +When Asynchronous Storage Writes is enabled, the responsibility of code using +the library is different from what was presented above. Users still read from +the Node.Ready() channel. However, they process the updates it contains in a +different manner. Users no longer consult the HardState, Entries, and Snapshot +fields (steps 1 and 3 above). They also no longer call Node.Advance() to +indicate that they have processed all entries in the Ready (step 4 above). +Instead, all local storage operations are also communicated through messages +present in the Ready.Message slice. + +The local storage messages come in two flavors. The first flavor is log append +messages, which target a LocalAppendThread and carry Entries, HardState, and a +Snapshot. The second flavor is entry application messages, which target a +LocalApplyThread and carry CommittedEntries. Messages to the same target must be +reliably processed in order. Messages to different targets can be processed in +any order. + +Each local storage message carries a slice of response messages that must +delivered after the corresponding storage write has been completed. These +responses may target the same node or may target other nodes. + +With Asynchronous Storage Writes enabled, the total state machine handling loop +will look something like this: + + for { + select { + case <-s.Ticker: + n.Tick() + case rd := <-s.Node.Ready(): + for _, m := range rd.Messages { + switch m.To { + case raft.LocalAppendThread: + toAppend <- m + case raft.LocalApplyThread: + toApply <-m + default: + sendOverNetwork(m) + } + } + case <-s.done: + return + } + } + +Usage of Asynchronous Storage Writes will typically also contain a pair of +storage handler threads, one for log writes (append) and one for entry +application to the local state machine (apply). Those will look something like: + + // append thread + go func() { + for { + select { + case m := <-toAppend: + saveToStorage(m.State, m.Entries, m.Snapshot) + send(m.Responses) + case <-s.done: + return + } + } + } + + // apply thread + go func() { + for { + select { + case m := <-toApply: + for _, entry := range m.CommittedEntries { + process(entry) + if entry.Type == raftpb.EntryConfChange { + var cc raftpb.ConfChange + cc.Unmarshal(entry.Data) + s.Node.ApplyConfChange(cc) + } + } + send(m.Responses) + case <-s.done: + return + } + } + } + # Implementation notes This implementation is up to date with the final Raft thesis @@ -295,5 +383,17 @@ stale log entries: that the follower that sent this 'MsgUnreachable' is not reachable, often indicating 'MsgApp' is lost. When follower's progress state is replicate, the leader sets it back to probe. + + 'MsgStorageAppend' is a message from a node to its local append storage + thread to write entries, hard state, and/or a snapshot to stable storage. + The message will carry one or more responses, one of which will be a + 'MsgStorageAppendResp' back to itself. The responses can also contain + 'MsgAppResp', 'MsgVoteResp', and 'MsgPreVoteResp' messages. Used with + AsynchronousStorageWrites. + + 'MsgStorageApply' is a message from a node to its local apply storage + thread to apply committed entries. The message will carry one response, + which will be a 'MsgStorageApplyResp' back to itself. Used with + AsynchronousStorageWrites. */ package raft diff --git a/log.go b/log.go index 133df420f44a..daf7acef7e4d 100644 --- a/log.go +++ b/log.go @@ -32,16 +32,34 @@ type raftLog struct { // committed is the highest log position that is known to be in // stable storage on a quorum of nodes. committed uint64 + // applying is the highest log position that the application has + // been instructed to apply to its state machine. Some of these + // entries may be in the process of applying and have not yet + // reached applied. + // Use: The field is incremented when accepting a Ready struct. + // Invariant: applied <= applying && applying <= committed + applying uint64 // applied is the highest log position that the application has - // been instructed to apply to its state machine. + // successfully applied to its state machine. + // Use: The field is incremented when advancing after the committed + // entries in a Ready struct have been applied (either synchronously + // or asynchronously). // Invariant: applied <= committed applied uint64 logger Logger - // maxNextCommittedEntsSize is the maximum number aggregate byte size of the - // messages returned from calls to nextCommittedEnts. - maxNextCommittedEntsSize uint64 + // maxApplyingEntsSize limits the outstanding byte size of the messages + // returned from calls to nextCommittedEnts that have not been acknowledged + // by a call to appliedTo. + maxApplyingEntsSize entryEncodingSize + // applyingEntsSize is the current outstanding byte size of the messages + // returned from calls to nextCommittedEnts that have not been acknowledged + // by a call to appliedTo. + applyingEntsSize entryEncodingSize + // applyingEntsPaused is true when entry application has been paused until + // enough progress is acknowledged. + applyingEntsPaused bool } // newLog returns log using the given storage and default options. It @@ -53,14 +71,14 @@ func newLog(storage Storage, logger Logger) *raftLog { // newLogWithSize returns a log using the given storage and max // message size. -func newLogWithSize(storage Storage, logger Logger, maxNextCommittedEntsSize uint64) *raftLog { +func newLogWithSize(storage Storage, logger Logger, maxApplyingEntsSize entryEncodingSize) *raftLog { if storage == nil { log.Panic("storage must not be nil") } log := &raftLog{ - storage: storage, - logger: logger, - maxNextCommittedEntsSize: maxNextCommittedEntsSize, + storage: storage, + logger: logger, + maxApplyingEntsSize: maxApplyingEntsSize, } firstIndex, err := storage.FirstIndex() if err != nil { @@ -71,16 +89,19 @@ func newLogWithSize(storage Storage, logger Logger, maxNextCommittedEntsSize uin panic(err) // TODO(bdarnell) } log.unstable.offset = lastIndex + 1 + log.unstable.offsetInProgress = lastIndex + 1 log.unstable.logger = logger // Initialize our committed and applied pointers to the time of the last compaction. log.committed = firstIndex - 1 + log.applying = firstIndex - 1 log.applied = firstIndex - 1 return log } func (l *raftLog) String() string { - return fmt.Sprintf("committed=%d, applied=%d, unstable.offset=%d, len(unstable.Entries)=%d", l.committed, l.applied, l.unstable.offset, len(l.unstable.entries)) + return fmt.Sprintf("committed=%d, applied=%d, applying=%d, unstable.offset=%d, unstable.offsetInProgress=%d, len(unstable.Entries)=%d", + l.committed, l.applied, l.applying, l.unstable.offset, l.unstable.offsetInProgress, len(l.unstable.entries)) } // maybeAppend returns (0, false) if the entries cannot be appended. Otherwise, @@ -170,46 +191,100 @@ func (l *raftLog) findConflictByTerm(index uint64, term uint64) uint64 { return index } -func (l *raftLog) unstableEntries() []pb.Entry { - if len(l.unstable.entries) == 0 { - return nil - } - return l.unstable.entries +// nextUnstableEnts returns all entries that are available to be written to the +// local stable log and are not already in-progress. +func (l *raftLog) nextUnstableEnts() []pb.Entry { + return l.unstable.nextEntries() +} + +// hasNextUnstableEnts returns if there are any entries that are available to be +// written to the local stable log and are not already in-progress. +func (l *raftLog) hasNextUnstableEnts() bool { + return len(l.nextUnstableEnts()) > 0 +} + +// hasNextOrInProgressUnstableEnts returns if there are any entries that are +// available to be written to the local stable log or in the process of being +// written to the local stable log. +func (l *raftLog) hasNextOrInProgressUnstableEnts() bool { + return len(l.unstable.entries) > 0 } // nextCommittedEnts returns all the available entries for execution. -// If applied is smaller than the index of snapshot, it returns all committed -// entries after the index of snapshot. -func (l *raftLog) nextCommittedEnts() (ents []pb.Entry) { - if l.hasPendingSnapshot() { +// Entries can be committed even when the local raft instance has not durably +// appended them to the local raft log yet. If allowUnstable is true, committed +// entries from the unstable log may be returned; otherwise, only entries known +// to reside locally on stable storage will be returned. +func (l *raftLog) nextCommittedEnts(allowUnstable bool) (ents []pb.Entry) { + if l.applyingEntsPaused { + // Entry application outstanding size limit reached. + return nil + } + if l.hasNextOrInProgressSnapshot() { // See comment in hasNextCommittedEnts. return nil } - if l.committed > l.applied { - lo, hi := l.applied+1, l.committed+1 // [lo, hi) - ents, err := l.slice(lo, hi, l.maxNextCommittedEntsSize) - if err != nil { - l.logger.Panicf("unexpected error when getting unapplied entries (%v)", err) - } - return ents + lo, hi := l.applying+1, l.maxAppliableIndex(allowUnstable)+1 // [lo, hi) + if lo >= hi { + // Nothing to apply. + return nil } - return nil + maxSize := l.maxApplyingEntsSize - l.applyingEntsSize + if maxSize <= 0 { + l.logger.Panicf("applying entry size (%d-%d)=%d not positive", + l.maxApplyingEntsSize, l.applyingEntsSize, maxSize) + } + ents, err := l.slice(lo, hi, maxSize) + if err != nil { + l.logger.Panicf("unexpected error when getting unapplied entries (%v)", err) + } + return ents } // hasNextCommittedEnts returns if there is any available entries for execution. // This is a fast check without heavy raftLog.slice() in nextCommittedEnts(). -func (l *raftLog) hasNextCommittedEnts() bool { - if l.hasPendingSnapshot() { +func (l *raftLog) hasNextCommittedEnts(allowUnstable bool) bool { + if l.applyingEntsPaused { + // Entry application outstanding size limit reached. + return false + } + if l.hasNextOrInProgressSnapshot() { // If we have a snapshot to apply, don't also return any committed // entries. Doing so raises questions about what should be applied // first. return false } - return l.committed > l.applied + lo, hi := l.applying+1, l.maxAppliableIndex(allowUnstable)+1 // [lo, hi) + return lo < hi +} + +// maxAppliableIndex returns the maximum committed index that can be applied. +// If allowUnstable is true, committed entries from the unstable log can be +// applied; otherwise, only entries known to reside locally on stable storage +// can be applied. +func (l *raftLog) maxAppliableIndex(allowUnstable bool) uint64 { + hi := l.committed + if !allowUnstable { + hi = min(hi, l.unstable.offset-1) + } + return hi } -// hasPendingSnapshot returns if there is pending snapshot waiting for applying. -func (l *raftLog) hasPendingSnapshot() bool { +// nextUnstableSnapshot returns the snapshot, if present, that is available to +// be applied to the local storage and is not already in-progress. +func (l *raftLog) nextUnstableSnapshot() *pb.Snapshot { + return l.unstable.nextSnapshot() +} + +// hasNextUnstableSnapshot returns if there is a snapshot that is available to +// be applied to the local storage and is not already in-progress. +func (l *raftLog) hasNextUnstableSnapshot() bool { + return l.unstable.nextSnapshot() != nil +} + +// hasNextOrInProgressSnapshot returns if there is pending snapshot waiting for +// applying or in the process of being applied. +func (l *raftLog) hasNextOrInProgressSnapshot() bool { return l.unstable.snapshot != nil } @@ -252,20 +327,51 @@ func (l *raftLog) commitTo(tocommit uint64) { } } -func (l *raftLog) appliedTo(i uint64) { - if i == 0 { - return - } +func (l *raftLog) appliedTo(i uint64, size entryEncodingSize) { if l.committed < i || i < l.applied { l.logger.Panicf("applied(%d) is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed) } l.applied = i + l.applying = max(l.applying, i) + if l.applyingEntsSize > size { + l.applyingEntsSize -= size + } else { + // Defense against underflow. + l.applyingEntsSize = 0 + } + l.applyingEntsPaused = l.applyingEntsSize >= l.maxApplyingEntsSize +} + +func (l *raftLog) acceptApplying(i uint64, size entryEncodingSize, allowUnstable bool) { + if l.committed < i { + l.logger.Panicf("applying(%d) is out of range [prevApplying(%d), committed(%d)]", i, l.applying, l.committed) + } + l.applying = i + l.applyingEntsSize += size + // Determine whether to pause entry application until some progress is + // acknowledged. We pause in two cases: + // 1. the outstanding entry size equals or exceeds the maximum size. + // 2. the outstanding entry size does not equal or exceed the maximum size, + // but we determine that the next entry in the log will push us over the + // limit. We determine this by comparing the last entry returned from + // raftLog.nextCommittedEnts to the maximum entry that the method was + // allowed to return had there been no size limit. If these indexes are + // not equal, then the returned entries slice must have been truncated to + // adhere to the memory limit. + l.applyingEntsPaused = l.applyingEntsSize >= l.maxApplyingEntsSize || + i < l.maxAppliableIndex(allowUnstable) } func (l *raftLog) stableTo(i, t uint64) { l.unstable.stableTo(i, t) } func (l *raftLog) stableSnapTo(i uint64) { l.unstable.stableSnapTo(i) } +// acceptUnstable indicates that the application has started persisting the +// unstable entries in storage, and that the current unstable entries are thus +// to be marked as being in-progress, to avoid returning them with future calls +// to Ready(). +func (l *raftLog) acceptUnstable() { l.unstable.acceptInProgress() } + func (l *raftLog) lastTerm() uint64 { t, err := l.term(l.lastIndex()) if err != nil { @@ -275,17 +381,20 @@ func (l *raftLog) lastTerm() uint64 { } func (l *raftLog) term(i uint64) (uint64, error) { - // the valid term range is [index of dummy entry, last index] + // Check the unstable log first, even before computing the valid term range, + // which may need to access stable Storage. If we find the entry's term in + // the unstable log, we know it was in the valid range. + if t, ok := l.unstable.maybeTerm(i); ok { + return t, nil + } + + // The valid term range is [index of dummy entry, last index]. dummyIndex := l.firstIndex() - 1 if i < dummyIndex || i > l.lastIndex() { // TODO: return an error instead? return 0, nil } - if t, ok := l.unstable.maybeTerm(i); ok { - return t, nil - } - t, err := l.storage.Term(i) if err == nil { return t, nil @@ -296,11 +405,11 @@ func (l *raftLog) term(i uint64) (uint64, error) { panic(err) // TODO(bdarnell) } -func (l *raftLog) entries(i, maxsize uint64) ([]pb.Entry, error) { +func (l *raftLog) entries(i uint64, maxSize entryEncodingSize) ([]pb.Entry, error) { if i > l.lastIndex() { return nil, nil } - return l.slice(i, l.lastIndex()+1, maxsize) + return l.slice(i, l.lastIndex()+1, maxSize) } // allEntries returns all entries in the log. @@ -349,7 +458,7 @@ func (l *raftLog) restore(s pb.Snapshot) { } // slice returns a slice of log entries from lo through hi-1, inclusive. -func (l *raftLog) slice(lo, hi, maxSize uint64) ([]pb.Entry, error) { +func (l *raftLog) slice(lo, hi uint64, maxSize entryEncodingSize) ([]pb.Entry, error) { err := l.mustCheckOutOfBounds(lo, hi) if err != nil { return nil, err @@ -359,7 +468,7 @@ func (l *raftLog) slice(lo, hi, maxSize uint64) ([]pb.Entry, error) { } var ents []pb.Entry if lo < l.unstable.offset { - storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable.offset), maxSize) + storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable.offset), uint64(maxSize)) if err == ErrCompacted { return nil, err } else if err == ErrUnavailable { diff --git a/log_test.go b/log_test.go index d062831b55b6..89d5827e2192 100644 --- a/log_test.go +++ b/log_test.go @@ -270,7 +270,7 @@ func TestCompactionSideEffects(t *testing.T) { } require.True(t, raftLog.maybeCommit(lastIndex, lastTerm)) - raftLog.appliedTo(raftLog.committed) + raftLog.appliedTo(raftLog.committed, 0 /* size */) offset := uint64(500) storage.Compact(offset) @@ -284,7 +284,7 @@ func TestCompactionSideEffects(t *testing.T) { require.True(t, raftLog.matchTerm(j, j)) } - unstableEnts := raftLog.unstableEntries() + unstableEnts := raftLog.nextUnstableEnts() require.Equal(t, 250, len(unstableEnts)) require.Equal(t, uint64(751), unstableEnts[0].Index) @@ -307,32 +307,50 @@ func TestHasNextCommittedEnts(t *testing.T) { {Term: 1, Index: 6}, } tests := []struct { - applied uint64 - snap bool - whasNext bool + applied uint64 + applying uint64 + allowUnstable bool + paused bool + snap bool + whasNext bool }{ - {applied: 0, snap: false, whasNext: true}, - {applied: 3, snap: false, whasNext: true}, - {applied: 4, snap: false, whasNext: true}, - {applied: 5, snap: false, whasNext: false}, + {applied: 3, applying: 3, allowUnstable: true, whasNext: true}, + {applied: 3, applying: 4, allowUnstable: true, whasNext: true}, + {applied: 3, applying: 5, allowUnstable: true, whasNext: false}, + {applied: 4, applying: 4, allowUnstable: true, whasNext: true}, + {applied: 4, applying: 5, allowUnstable: true, whasNext: false}, + {applied: 5, applying: 5, allowUnstable: true, whasNext: false}, + // Don't allow unstable entries. + {applied: 3, applying: 3, allowUnstable: false, whasNext: true}, + {applied: 3, applying: 4, allowUnstable: false, whasNext: false}, + {applied: 3, applying: 5, allowUnstable: false, whasNext: false}, + {applied: 4, applying: 4, allowUnstable: false, whasNext: false}, + {applied: 4, applying: 5, allowUnstable: false, whasNext: false}, + {applied: 5, applying: 5, allowUnstable: false, whasNext: false}, + // Paused. + {applied: 3, applying: 3, allowUnstable: true, paused: true, whasNext: false}, // With snapshot. - {applied: 3, snap: true, whasNext: false}, + {applied: 3, applying: 3, allowUnstable: true, snap: true, whasNext: false}, } for i, tt := range tests { t.Run(fmt.Sprint(i), func(t *testing.T) { storage := NewMemoryStorage() require.NoError(t, storage.ApplySnapshot(snap)) + require.NoError(t, storage.Append(ents[:1])) raftLog := newLog(storage, raftLogger) raftLog.append(ents...) + raftLog.stableTo(4, 1) raftLog.maybeCommit(5, 1) - raftLog.appliedTo(tt.applied) + raftLog.appliedTo(tt.applied, 0 /* size */) + raftLog.acceptApplying(tt.applying, 0 /* size */, tt.allowUnstable) + raftLog.applyingEntsPaused = tt.paused if tt.snap { newSnap := snap newSnap.Metadata.Index++ raftLog.restore(newSnap) } - require.Equal(t, tt.whasNext, raftLog.hasNextCommittedEnts()) + require.Equal(t, tt.whasNext, raftLog.hasNextCommittedEnts(tt.allowUnstable)) }) } } @@ -347,40 +365,164 @@ func TestNextCommittedEnts(t *testing.T) { {Term: 1, Index: 6}, } tests := []struct { - applied uint64 - snap bool - wents []pb.Entry + applied uint64 + applying uint64 + allowUnstable bool + paused bool + snap bool + wents []pb.Entry }{ - {applied: 0, snap: false, wents: ents[:2]}, - {applied: 3, snap: false, wents: ents[:2]}, - {applied: 4, snap: false, wents: ents[1:2]}, - {applied: 5, snap: false, wents: nil}, + {applied: 3, applying: 3, allowUnstable: true, wents: ents[:2]}, + {applied: 3, applying: 4, allowUnstable: true, wents: ents[1:2]}, + {applied: 3, applying: 5, allowUnstable: true, wents: nil}, + {applied: 4, applying: 4, allowUnstable: true, wents: ents[1:2]}, + {applied: 4, applying: 5, allowUnstable: true, wents: nil}, + {applied: 5, applying: 5, allowUnstable: true, wents: nil}, + // Don't allow unstable entries. + {applied: 3, applying: 3, allowUnstable: false, wents: ents[:1]}, + {applied: 3, applying: 4, allowUnstable: false, wents: nil}, + {applied: 3, applying: 5, allowUnstable: false, wents: nil}, + {applied: 4, applying: 4, allowUnstable: false, wents: nil}, + {applied: 4, applying: 5, allowUnstable: false, wents: nil}, + {applied: 5, applying: 5, allowUnstable: false, wents: nil}, + // Paused. + {applied: 3, applying: 3, allowUnstable: true, paused: true, wents: nil}, // With snapshot. - {applied: 3, snap: true, wents: nil}, + {applied: 3, applying: 3, allowUnstable: true, snap: true, wents: nil}, } for i, tt := range tests { t.Run(fmt.Sprint(i), func(t *testing.T) { storage := NewMemoryStorage() require.NoError(t, storage.ApplySnapshot(snap)) + require.NoError(t, storage.Append(ents[:1])) raftLog := newLog(storage, raftLogger) raftLog.append(ents...) + raftLog.stableTo(4, 1) raftLog.maybeCommit(5, 1) - raftLog.appliedTo(tt.applied) + raftLog.appliedTo(tt.applied, 0 /* size */) + raftLog.acceptApplying(tt.applying, 0 /* size */, tt.allowUnstable) + raftLog.applyingEntsPaused = tt.paused if tt.snap { newSnap := snap newSnap.Metadata.Index++ raftLog.restore(newSnap) } - require.Equal(t, tt.wents, raftLog.nextCommittedEnts()) + require.Equal(t, tt.wents, raftLog.nextCommittedEnts(tt.allowUnstable)) }) + } +} + +func TestAcceptApplying(t *testing.T) { + maxSize := entryEncodingSize(100) + snap := pb.Snapshot{ + Metadata: pb.SnapshotMetadata{Term: 1, Index: 3}, + } + ents := []pb.Entry{ + {Term: 1, Index: 4}, + {Term: 1, Index: 5}, + {Term: 1, Index: 6}, + } + tests := []struct { + index uint64 + allowUnstable bool + size entryEncodingSize + wpaused bool + }{ + {index: 3, allowUnstable: true, size: maxSize - 1, wpaused: true}, + {index: 3, allowUnstable: true, size: maxSize, wpaused: true}, + {index: 3, allowUnstable: true, size: maxSize + 1, wpaused: true}, + {index: 4, allowUnstable: true, size: maxSize - 1, wpaused: true}, + {index: 4, allowUnstable: true, size: maxSize, wpaused: true}, + {index: 4, allowUnstable: true, size: maxSize + 1, wpaused: true}, + {index: 5, allowUnstable: true, size: maxSize - 1, wpaused: false}, + {index: 5, allowUnstable: true, size: maxSize, wpaused: true}, + {index: 5, allowUnstable: true, size: maxSize + 1, wpaused: true}, + // Don't allow unstable entries. + {index: 3, allowUnstable: false, size: maxSize - 1, wpaused: true}, + {index: 3, allowUnstable: false, size: maxSize, wpaused: true}, + {index: 3, allowUnstable: false, size: maxSize + 1, wpaused: true}, + {index: 4, allowUnstable: false, size: maxSize - 1, wpaused: false}, + {index: 4, allowUnstable: false, size: maxSize, wpaused: true}, + {index: 4, allowUnstable: false, size: maxSize + 1, wpaused: true}, + {index: 5, allowUnstable: false, size: maxSize - 1, wpaused: false}, + {index: 5, allowUnstable: false, size: maxSize, wpaused: true}, + {index: 5, allowUnstable: false, size: maxSize + 1, wpaused: true}, + } + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + storage := NewMemoryStorage() + require.NoError(t, storage.ApplySnapshot(snap)) + require.NoError(t, storage.Append(ents[:1])) + + raftLog := newLogWithSize(storage, raftLogger, maxSize) + raftLog.append(ents...) + raftLog.stableTo(4, 1) + raftLog.maybeCommit(5, 1) + raftLog.appliedTo(3, 0 /* size */) + + raftLog.acceptApplying(tt.index, tt.size, tt.allowUnstable) + require.Equal(t, tt.wpaused, raftLog.applyingEntsPaused) + }) + } +} + +func TestAppliedTo(t *testing.T) { + maxSize := entryEncodingSize(100) + overshoot := entryEncodingSize(5) + snap := pb.Snapshot{ + Metadata: pb.SnapshotMetadata{Term: 1, Index: 3}, + } + ents := []pb.Entry{ + {Term: 1, Index: 4}, + {Term: 1, Index: 5}, + {Term: 1, Index: 6}, + } + tests := []struct { + index uint64 + size entryEncodingSize + wapplyingSize entryEncodingSize + wpaused bool + }{ + // Apply some of in-progress entries (applying = 5 below). + {index: 4, size: overshoot - 1, wapplyingSize: maxSize + 1, wpaused: true}, + {index: 4, size: overshoot, wapplyingSize: maxSize, wpaused: true}, + {index: 4, size: overshoot + 1, wapplyingSize: maxSize - 1, wpaused: false}, + // Apply all of in-progress entries. + {index: 5, size: overshoot - 1, wapplyingSize: maxSize + 1, wpaused: true}, + {index: 5, size: overshoot, wapplyingSize: maxSize, wpaused: true}, + {index: 5, size: overshoot + 1, wapplyingSize: maxSize - 1, wpaused: false}, + // Apply all of outstanding bytes. + {index: 4, size: maxSize + overshoot, wapplyingSize: 0, wpaused: false}, + // Apply more than outstanding bytes. + // Incorrect accounting doesn't underflow applyingSize. + {index: 4, size: maxSize + overshoot + 1, wapplyingSize: 0, wpaused: false}, + } + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + storage := NewMemoryStorage() + require.NoError(t, storage.ApplySnapshot(snap)) + require.NoError(t, storage.Append(ents[:1])) + raftLog := newLogWithSize(storage, raftLogger, maxSize) + raftLog.append(ents...) + raftLog.stableTo(4, 1) + raftLog.maybeCommit(5, 1) + raftLog.appliedTo(3, 0 /* size */) + raftLog.acceptApplying(5, maxSize+overshoot, false /* allowUnstable */) + + raftLog.appliedTo(tt.index, tt.size) + require.Equal(t, tt.index, raftLog.applied) + require.Equal(t, uint64(5), raftLog.applying) + require.Equal(t, tt.wapplyingSize, raftLog.applyingEntsSize) + require.Equal(t, tt.wpaused, raftLog.applyingEntsPaused) + }) } } -// TestUnstableEnts ensures unstableEntries returns the unstable part of the +// TestNextUnstableEnts ensures unstableEntries returns the unstable part of the // entries correctly. -func TestUnstableEnts(t *testing.T) { +func TestNextUnstableEnts(t *testing.T) { previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}} tests := []struct { unstable uint64 @@ -400,7 +542,7 @@ func TestUnstableEnts(t *testing.T) { raftLog := newLog(storage, raftLogger) raftLog.append(previousEnts[tt.unstable-1:]...) - ents := raftLog.unstableEntries() + ents := raftLog.nextUnstableEnts() if l := len(ents); l > 0 { raftLog.stableTo(ents[l-1].Index, ents[l-1].Term) } @@ -526,7 +668,7 @@ func TestCompaction(t *testing.T) { raftLog := newLog(storage, raftLogger) raftLog.maybeCommit(tt.lastIndex, 0) - raftLog.appliedTo(raftLog.committed) + raftLog.appliedTo(raftLog.committed, 0 /* size */) for j := 0; j < len(tt.compact); j++ { err := storage.Compact(tt.compact[j]) if err != nil { @@ -738,7 +880,7 @@ func TestSlice(t *testing.T) { require.True(t, tt.wpanic) } }() - g, err := l.slice(tt.from, tt.to, tt.limit) + g, err := l.slice(tt.from, tt.to, entryEncodingSize(tt.limit)) require.False(t, tt.from <= offset && err != ErrCompacted) require.False(t, tt.from > offset && err != nil) require.Equal(t, tt.w, g) diff --git a/log_unstable.go b/log_unstable.go index 0e4ba183949c..740367ba048f 100644 --- a/log_unstable.go +++ b/log_unstable.go @@ -16,6 +16,16 @@ package raft import pb "go.etcd.io/raft/v3/raftpb" +// unstable contains "unstable" log entries and snapshot state that has +// not yet been written to Storage. The type serves two roles. First, it +// holds on to new log entries and an optional snapshot until they are +// handed to a Ready struct for persistence. Second, it continues to +// hold on to this state after it has been handed off to provide raftLog +// with a view of the in-progress log entries and snapshot until their +// writes have been stabilized and are guaranteed to be reflected in +// queries of Storage. After this point, the corresponding log entries +// and/or snapshot can be cleared from unstable. +// // unstable.entries[i] has raft log position i+unstable.offset. // Note that unstable.offset may be less than the highest log // position in storage; this means that the next write to storage @@ -25,7 +35,16 @@ type unstable struct { snapshot *pb.Snapshot // all entries that have not yet been written to storage. entries []pb.Entry - offset uint64 + // entries[i] has raft log position i+offset. + offset uint64 + + // if true, snapshot is being written to storage. + snapshotInProgress bool + // entries[:offsetInProgress-offset] are being written to storage. + // Like offset, offsetInProgress is exclusive, meaning that it + // contains the index following the largest in-progress entry. + // Invariant: offset <= offsetInProgress + offsetInProgress uint64 logger Logger } @@ -72,22 +91,71 @@ func (u *unstable) maybeTerm(i uint64) (uint64, bool) { return u.entries[i-u.offset].Term, true } +// nextEntries returns the unstable entries that are not already in the process +// of being written to storage. +func (u *unstable) nextEntries() []pb.Entry { + inProgress := int(u.offsetInProgress - u.offset) + if len(u.entries) == inProgress { + return nil + } + return u.entries[inProgress:] +} + +// nextSnapshot returns the unstable snapshot, if one exists that is not already +// in the process of being written to storage. +func (u *unstable) nextSnapshot() *pb.Snapshot { + if u.snapshot == nil || u.snapshotInProgress { + return nil + } + return u.snapshot +} + +// acceptInProgress marks all entries and the snapshot, if any, in the unstable +// as having begun the process of being written to storage. The entries/snapshot +// will no longer be returned from nextEntries/nextSnapshot. However, new +// entries/snapshots added after a call to acceptInProgress will be returned +// from those methods, until the next call to acceptInProgress. +func (u *unstable) acceptInProgress() { + if len(u.entries) > 0 { + // NOTE: +1 because offsetInProgress is exclusive, like offset. + u.offsetInProgress = u.entries[len(u.entries)-1].Index + 1 + } + if u.snapshot != nil { + u.snapshotInProgress = true + } +} + +// stableTo marks entries up to the entry with the specified (index, term) as +// being successfully written to stable storage. +// +// The method should only be called when the caller can attest that the entries +// can not be overwritten by an in-progress log append. See the related comment +// in newStorageAppendRespMsg. func (u *unstable) stableTo(i, t uint64) { gt, ok := u.maybeTerm(i) if !ok { // Unstable entry missing. Ignore. + u.logger.Infof("entry at index %d missing from unstable log; ignoring", i) return } if i < u.offset { // Index matched unstable snapshot, not unstable entry. Ignore. + u.logger.Infof("entry at index %d matched unstable snapshot; ignoring", i) return } if gt != t { // Term mismatch between unstable entry and specified entry. Ignore. + // This is possible if part or all of the unstable log was replaced + // between that time that a set of entries started to be written to + // stable storage and when they finished. + u.logger.Infof("entry at (index,term)=(%d,%d) mismatched with "+ + "entry at (%d,%d) in unstable log; ignoring", i, t, i, gt) return } - u.entries = u.entries[i+1-u.offset:] + num := int(i + 1 - u.offset) + u.entries = u.entries[num:] u.offset = i + 1 + u.offsetInProgress = max(u.offsetInProgress, u.offset) u.shrinkEntriesArray() } @@ -113,37 +181,47 @@ func (u *unstable) shrinkEntriesArray() { func (u *unstable) stableSnapTo(i uint64) { if u.snapshot != nil && u.snapshot.Metadata.Index == i { u.snapshot = nil + u.snapshotInProgress = false } } func (u *unstable) restore(s pb.Snapshot) { u.offset = s.Metadata.Index + 1 + u.offsetInProgress = u.offset u.entries = nil u.snapshot = &s + u.snapshotInProgress = false } func (u *unstable) truncateAndAppend(ents []pb.Entry) { + // TODO(nvanbenschoten): rename this variable to firstAppIndex. after := ents[0].Index switch { case after == u.offset+uint64(len(u.entries)): - // after is the next index in the u.entries - // directly append + // after is the next index in the u.entries, so append directly. u.entries = append(u.entries, ents...) case after <= u.offset: u.logger.Infof("replace the unstable entries from index %d", after) // The log is being truncated to before our current offset - // portion, so set the offset and replace the entries - u.offset = after + // portion, so set the offset and replace the entries. u.entries = ents + u.offset = after + u.offsetInProgress = u.offset default: - // truncate to after and copy to u.entries - // then append + // truncate to after and copy to u.entries then append. u.logger.Infof("truncate the unstable entries before index %d", after) - u.entries = append([]pb.Entry{}, u.slice(u.offset, after)...) + keep := u.slice(u.offset, after) + u.entries = append([]pb.Entry{}, keep...) u.entries = append(u.entries, ents...) + // Only in-progress entries before after are still considered to be + // in-progress. + u.offsetInProgress = min(u.offsetInProgress, after) } } +// slice returns the entries from the unstable log with indexes in the +// range [lo, hi). The entire range must be stored in the unstable log +// or the method will panic. func (u *unstable) slice(lo uint64, hi uint64) []pb.Entry { u.mustCheckOutOfBounds(lo, hi) return u.entries[lo-u.offset : hi-u.offset] diff --git a/log_unstable_test.go b/log_unstable_test.go index bf03f2487b36..a0b4f8e3e3c0 100644 --- a/log_unstable_test.go +++ b/log_unstable_test.go @@ -53,7 +53,6 @@ func TestUnstableMaybeFirstIndex(t *testing.T) { } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ entries: tt.entries, @@ -99,7 +98,6 @@ func TestMaybeLastIndex(t *testing.T) { } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ entries: tt.entries, @@ -179,7 +177,6 @@ func TestUnstableMaybeTerm(t *testing.T) { } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ entries: tt.entries, @@ -196,98 +193,309 @@ func TestUnstableMaybeTerm(t *testing.T) { func TestUnstableRestore(t *testing.T) { u := unstable{ - entries: []pb.Entry{{Index: 5, Term: 1}}, - offset: 5, - snapshot: &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, - logger: raftLogger, + entries: []pb.Entry{{Index: 5, Term: 1}}, + offset: 5, + offsetInProgress: 6, + snapshot: &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + snapshotInProgress: true, + logger: raftLogger, } s := pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 6, Term: 2}} u.restore(s) require.Equal(t, s.Metadata.Index+1, u.offset) + require.Equal(t, s.Metadata.Index+1, u.offsetInProgress) require.Zero(t, len(u.entries)) require.Equal(t, &s, u.snapshot) + require.False(t, u.snapshotInProgress) +} + +func TestUnstableNextEntries(t *testing.T) { + tests := []struct { + entries []pb.Entry + offset uint64 + offsetInProgress uint64 + + wentries []pb.Entry + }{ + // nothing in progress + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 5, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, + }, + // partially in progress + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 6, + []pb.Entry{{Index: 6, Term: 1}}, + }, + // everything in progress + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 7, + nil, // nil, not empty slice + }, + } + + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + u := unstable{ + entries: tt.entries, + offset: tt.offset, + offsetInProgress: tt.offsetInProgress, + logger: raftLogger, + } + res := u.nextEntries() + require.Equal(t, tt.wentries, res) + }) + } +} + +func TestUnstableNextSnapshot(t *testing.T) { + s := &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}} + tests := []struct { + snapshot *pb.Snapshot + snapshotInProgress bool + + wsnapshot *pb.Snapshot + }{ + // snapshot not unstable + { + nil, false, + nil, + }, + // snapshot not in progress + { + s, false, + s, + }, + // snapshot in progress + { + s, true, + nil, + }, + } + + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + u := unstable{ + snapshot: tt.snapshot, + snapshotInProgress: tt.snapshotInProgress, + } + res := u.nextSnapshot() + require.Equal(t, tt.wsnapshot, res) + }) + } +} + +func TestUnstableAcceptInProgress(t *testing.T) { + tests := []struct { + entries []pb.Entry + snapshot *pb.Snapshot + offsetInProgress uint64 + snapshotInProgress bool + + woffsetInProgress uint64 + wsnapshotInProgress bool + }{ + { + []pb.Entry{}, nil, + 5, // no entries + false, // snapshot not already in progress + 5, false, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, nil, + 5, // entries not in progress + false, // snapshot not already in progress + 6, false, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, nil, + 5, // entries not in progress + false, // snapshot not already in progress + 7, false, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, nil, + 6, // in-progress to the first entry + false, // snapshot not already in progress + 7, false, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, nil, + 7, // in-progress to the second entry + false, // snapshot not already in progress + 7, false, + }, + // with snapshot + { + []pb.Entry{}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // no entries + false, // snapshot not already in progress + 5, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + false, // snapshot not already in progress + 6, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + false, // snapshot not already in progress + 7, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 6, // in-progress to the first entry + false, // snapshot not already in progress + 7, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 7, // in-progress to the second entry + false, // snapshot not already in progress + 7, true, + }, + { + []pb.Entry{}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + true, // snapshot already in progress + 5, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + true, // snapshot already in progress + 6, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, // entries not in progress + true, // snapshot already in progress + 7, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 6, // in-progress to the first entry + true, // snapshot already in progress + 7, true, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 7, // in-progress to the second entry + true, // snapshot already in progress + 7, true, + }, + } + + for i, tt := range tests { + t.Run(fmt.Sprint(i), func(t *testing.T) { + u := unstable{ + entries: tt.entries, + snapshot: tt.snapshot, + offsetInProgress: tt.offsetInProgress, + snapshotInProgress: tt.snapshotInProgress, + } + u.acceptInProgress() + require.Equal(t, tt.woffsetInProgress, u.offsetInProgress) + require.Equal(t, tt.wsnapshotInProgress, u.snapshotInProgress) + }) + } } func TestUnstableStableTo(t *testing.T) { tests := []struct { - entries []pb.Entry - offset uint64 - snap *pb.Snapshot - index, term uint64 + entries []pb.Entry + offset uint64 + offsetInProgress uint64 + snap *pb.Snapshot + index, term uint64 - woffset uint64 - wlen int + woffset uint64 + woffsetInProgress uint64 + wlen int }{ { - []pb.Entry{}, 0, nil, + []pb.Entry{}, 0, 0, nil, 5, 1, - 0, 0, + 0, 0, 0, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, 5, 1, // stable to the first entry - 6, 0, + 6, 6, 0, }, { - []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 6, nil, 5, 1, // stable to the first entry - 6, 1, + 6, 6, 1, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 7, nil, + 5, 1, // stable to the first entry and in-progress ahead + 6, 7, 1, }, { - []pb.Entry{{Index: 6, Term: 2}}, 6, nil, + []pb.Entry{{Index: 6, Term: 2}}, 6, 7, nil, 6, 1, // stable to the first entry and term mismatch - 6, 1, + 6, 7, 1, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, 4, 1, // stable to old entry - 5, 1, + 5, 6, 1, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, 4, 2, // stable to old entry - 5, 1, + 5, 6, 1, }, // with snapshot { - []pb.Entry{{Index: 5, Term: 1}}, 5, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, 5, 1, // stable to the first entry - 6, 0, + 6, 6, 0, }, { - []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, 5, 1, // stable to the first entry - 6, 1, + 6, 6, 1, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}}, 5, 7, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + 5, 1, // stable to the first entry and in-progress ahead + 6, 7, 1, }, { - []pb.Entry{{Index: 6, Term: 2}}, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 5, Term: 1}}, + []pb.Entry{{Index: 6, Term: 2}}, 6, 7, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 5, Term: 1}}, 6, 1, // stable to the first entry and term mismatch - 6, 1, + 6, 7, 1, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 1}}, 4, 1, // stable to snapshot - 5, 1, + 5, 6, 1, }, { - []pb.Entry{{Index: 5, Term: 2}}, 5, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 2}}, + []pb.Entry{{Index: 5, Term: 2}}, 5, 6, &pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: 4, Term: 2}}, 4, 1, // stable to old entry - 5, 1, + 5, 6, 1, }, } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ - entries: tt.entries, - offset: tt.offset, - snapshot: tt.snap, - logger: raftLogger, + entries: tt.entries, + offset: tt.offset, + offsetInProgress: tt.offsetInProgress, + snapshot: tt.snap, + logger: raftLogger, } u.stableTo(tt.index, tt.term) require.Equal(t, tt.woffset, u.offset) + require.Equal(t, tt.woffsetInProgress, u.offsetInProgress) require.Equal(t, tt.wlen, len(u.entries)) }) } @@ -295,55 +503,78 @@ func TestUnstableStableTo(t *testing.T) { func TestUnstableTruncateAndAppend(t *testing.T) { tests := []struct { - entries []pb.Entry - offset uint64 - snap *pb.Snapshot - toappend []pb.Entry + entries []pb.Entry + offset uint64 + offsetInProgress uint64 + snap *pb.Snapshot + toappend []pb.Entry - woffset uint64 - wentries []pb.Entry + woffset uint64 + woffsetInProgress uint64 + wentries []pb.Entry }{ // append to the end { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 5, nil, + []pb.Entry{{Index: 6, Term: 1}, {Index: 7, Term: 1}}, + 5, 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, []pb.Entry{{Index: 6, Term: 1}, {Index: 7, Term: 1}}, - 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, + 5, 6, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, }, // replace the unstable entries { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 5, nil, []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, - 5, []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, + 5, 5, []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, }, { - []pb.Entry{{Index: 5, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}}, 5, 5, nil, []pb.Entry{{Index: 4, Term: 2}, {Index: 5, Term: 2}, {Index: 6, Term: 2}}, - 4, []pb.Entry{{Index: 4, Term: 2}, {Index: 5, Term: 2}, {Index: 6, Term: 2}}, + 4, 4, []pb.Entry{{Index: 4, Term: 2}, {Index: 5, Term: 2}, {Index: 6, Term: 2}}, + }, + { + []pb.Entry{{Index: 5, Term: 1}}, 5, 6, nil, + []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, + 5, 5, []pb.Entry{{Index: 5, Term: 2}, {Index: 6, Term: 2}}, }, // truncate the existing entries and append { - []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, 5, nil, []pb.Entry{{Index: 6, Term: 2}}, - 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 2}}, + 5, 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 2}}, }, { - []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, nil, + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, 5, nil, []pb.Entry{{Index: 7, Term: 2}, {Index: 8, Term: 2}}, - 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 2}, {Index: 8, Term: 2}}, + 5, 5, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 2}, {Index: 8, Term: 2}}, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, 6, nil, + []pb.Entry{{Index: 6, Term: 2}}, + 5, 6, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 2}}, + }, + { + []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 1}, {Index: 7, Term: 1}}, 5, 7, nil, + []pb.Entry{{Index: 6, Term: 2}}, + 5, 6, []pb.Entry{{Index: 5, Term: 1}, {Index: 6, Term: 2}}, }, } for i, tt := range tests { - tt := tt t.Run(fmt.Sprint(i), func(t *testing.T) { u := unstable{ - entries: tt.entries, - offset: tt.offset, - snapshot: tt.snap, - logger: raftLogger, + entries: tt.entries, + offset: tt.offset, + offsetInProgress: tt.offsetInProgress, + snapshot: tt.snap, + logger: raftLogger, } u.truncateAndAppend(tt.toappend) require.Equal(t, tt.woffset, u.offset) + require.Equal(t, tt.woffsetInProgress, u.offsetInProgress) require.Equal(t, tt.wentries, u.entries) }) } diff --git a/node.go b/node.go index 62288a17dbba..9c53aed29d45 100644 --- a/node.go +++ b/node.go @@ -57,7 +57,12 @@ type Ready struct { // The current state of a Node to be saved to stable storage BEFORE // Messages are sent. + // // HardState will be equal to empty state if there is no update. + // + // If async storage writes are enabled, this field does not need to be acted + // on immediately. It will be reflected in a MsgStorageAppend message in the + // Messages slice. pb.HardState // ReadStates can be used for node to serve linearizable read requests locally @@ -68,24 +73,44 @@ type Ready struct { // Entries specifies entries to be saved to stable storage BEFORE // Messages are sent. + // + // If async storage writes are enabled, this field does not need to be acted + // on immediately. It will be reflected in a MsgStorageAppend message in the + // Messages slice. Entries []pb.Entry // Snapshot specifies the snapshot to be saved to stable storage. + // + // If async storage writes are enabled, this field does not need to be acted + // on immediately. It will be reflected in a MsgStorageAppend message in the + // Messages slice. Snapshot pb.Snapshot // CommittedEntries specifies entries to be committed to a - // store/state-machine. These have previously been committed to stable - // store. + // store/state-machine. These have previously been appended to stable + // storage. + // + // If async storage writes are enabled, this field does not need to be acted + // on immediately. It will be reflected in a MsgStorageApply message in the + // Messages slice. CommittedEntries []pb.Entry - // Messages specifies outbound messages to be sent AFTER Entries are - // committed to stable storage. + // Messages specifies outbound messages. + // + // If async storage writes are not enabled, these messages must be sent + // AFTER Entries are appended to stable storage. + // + // If async storage writes are enabled, these messages can be sent + // immediately as the messages that have the completion of the async writes + // as a precondition are attached to the individual MsgStorage{Append,Apply} + // messages instead. + // // If it contains a MsgSnap message, the application MUST report back to raft // when the snapshot has been received or has failed by calling ReportSnapshot. Messages []pb.Message - // MustSync indicates whether the HardState and Entries must be synchronously - // written to disk or if an asynchronous write is permissible. + // MustSync indicates whether the HardState and Entries must be durably + // written to disk or if a non-durable write is permissible. MustSync bool } @@ -103,19 +128,6 @@ func IsEmptySnap(sp pb.Snapshot) bool { return sp.Metadata.Index == 0 } -// appliedCursor extracts from the Ready the highest index the client has -// applied (once the Ready is confirmed via Advance). If no information is -// contained in the Ready, returns zero. -func (rd Ready) appliedCursor() uint64 { - if n := len(rd.CommittedEntries); n > 0 { - return rd.CommittedEntries[n-1].Index - } - if index := rd.Snapshot.Metadata.Index; index > 0 { - return index - } - return 0 -} - // Node represents a node in a raft cluster. type Node interface { // Tick increments the internal logical clock for the Node by a single tick. Election @@ -144,7 +156,8 @@ type Node interface { Step(ctx context.Context, msg pb.Message) error // Ready returns a channel that returns the current point-in-time state. - // Users of the Node must call Advance after retrieving the state returned by Ready. + // Users of the Node must call Advance after retrieving the state returned by Ready (unless + // async storage writes is enabled, in which case it should never be called). // // NOTE: No committed entries from the next Ready may be applied until all committed entries // and snapshots from the previous one have finished. @@ -159,6 +172,9 @@ type Node interface { // commands. For example. when the last Ready contains a snapshot, the application might take // a long time to apply the snapshot data. To continue receiving Ready without blocking raft // progress, it can call Advance before finishing applying the last ready. + // + // NOTE: Advance must not be called when using AsyncStorageWrites. Response messages from the + // local append and apply threads take its place. Advance() // ApplyConfChange applies a config change (previously passed to // ProposeConfChange) to the node. This must be called whenever a config @@ -309,9 +325,7 @@ func (n *node) run() { lead := None for { - if advancec != nil { - readyc = nil - } else if n.rn.HasReady() { + if advancec == nil && n.rn.HasReady() { // Populate a Ready. Note that this Ready is not guaranteed to // actually be handled. We will arm readyc, but there's no guarantee // that we will actually send on it. It's possible that we will @@ -352,10 +366,11 @@ func (n *node) run() { close(pm.result) } case m := <-n.recvc: - // filter out response message from unknown From. - if pr := r.prs.Progress[m.From]; pr != nil || !IsResponseMsg(m.Type) { - r.Step(m) + if IsResponseMsg(m.Type) && !IsLocalMsgTarget(m.From) && r.prs.Progress[m.From] == nil { + // Filter out response message from unknown From. + break } + r.Step(m) case cc := <-n.confc: _, okBefore := r.prs.Progress[r.id] cs := r.applyConfChange(cc) @@ -393,7 +408,12 @@ func (n *node) run() { n.rn.Tick() case readyc <- rd: n.rn.acceptReady(rd) - advancec = n.advancec + if !n.rn.asyncStorageWrites { + advancec = n.advancec + } else { + rd = Ready{} + } + readyc = nil case <-advancec: n.rn.Advance(rd) rd = Ready{} @@ -425,8 +445,8 @@ func (n *node) Propose(ctx context.Context, data []byte) error { } func (n *node) Step(ctx context.Context, m pb.Message) error { - // ignore unexpected local messages receiving over network - if IsLocalMsg(m.Type) { + // Ignore unexpected local messages receiving over network. + if IsLocalMsg(m.Type) && !IsLocalMsgTarget(m.From) { // TODO: return an error? return nil } @@ -558,36 +578,3 @@ func (n *node) TransferLeadership(ctx context.Context, lead, transferee uint64) func (n *node) ReadIndex(ctx context.Context, rctx []byte) error { return n.step(ctx, pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}}) } - -func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready { - rd := Ready{ - Entries: r.raftLog.unstableEntries(), - CommittedEntries: r.raftLog.nextCommittedEnts(), - Messages: r.msgs, - } - if softSt := r.softState(); !softSt.equal(prevSoftSt) { - rd.SoftState = softSt - } - if hardSt := r.hardState(); !isHardStateEqual(hardSt, prevHardSt) { - rd.HardState = hardSt - } - if r.raftLog.unstable.snapshot != nil { - rd.Snapshot = *r.raftLog.unstable.snapshot - } - if len(r.readStates) != 0 { - rd.ReadStates = r.readStates - } - rd.MustSync = MustSync(r.hardState(), prevHardSt, len(rd.Entries)) - return rd -} - -// MustSync returns true if the hard state and count of Raft entries indicate -// that a synchronous write to persistent storage is required. -func MustSync(st, prevst pb.HardState, entsnum int) bool { - // Persistent state on all servers: - // (Updated on stable storage before responding to RPCs) - // currentTerm - // votedFor - // log entries[] - return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term -} diff --git a/node_test.go b/node_test.go index d0da8a2f92a3..068002b85c4c 100644 --- a/node_test.go +++ b/node_test.go @@ -24,6 +24,8 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "go.etcd.io/raft/v3/raftpb" ) @@ -393,7 +395,8 @@ func TestNodeProposeAddDuplicateNode(t *testing.T) { // know who is the current leader; node will accept proposal when it knows // who is the current leader. func TestBlockProposal(t *testing.T) { - rn := newTestRawNode(1, 10, 1, newTestMemoryStorage(withPeers(1))) + s := newTestMemoryStorage(withPeers(1)) + rn := newTestRawNode(1, 10, 1, s) n := newNode(rn) go n.run() defer n.Stop() @@ -412,6 +415,9 @@ func TestBlockProposal(t *testing.T) { } n.Campaign(context.TODO()) + rd := <-n.Ready() + s.Append(rd.Entries) + n.Advance() select { case err := <-errc: if err != nil { @@ -586,9 +592,14 @@ func TestNodeStart(t *testing.T) { } { + // Persist vote. rd := <-n.Ready() storage.Append(rd.Entries) n.Advance() + // Append empty entry. + rd = <-n.Ready() + storage.Append(rd.Entries) + n.Advance() } n.Propose(ctx, []byte("foo")) @@ -723,8 +734,12 @@ func TestNodeAdvance(t *testing.T) { defer cancel() n.Campaign(ctx) + // Persist vote. rd := readyWithTimeout(n) - // Commit empty entry. + storage.Append(rd.Entries) + n.Advance() + // Append empty entry. + rd = readyWithTimeout(n) storage.Append(rd.Entries) n.Advance() @@ -878,9 +893,15 @@ func TestCommitPagination(t *testing.T) { defer cancel() n.Campaign(ctx) + // Persist vote. rd := readyWithTimeout(n) s.Append(rd.Entries) n.Advance() + // Append empty entry. + rd = readyWithTimeout(n) + s.Append(rd.Entries) + n.Advance() + // Apply empty entry. rd = readyWithTimeout(n) if len(rd.CommittedEntries) != 1 { t.Fatalf("expected 1 (empty) entry, got %d", len(rd.CommittedEntries)) @@ -918,6 +939,153 @@ func TestCommitPagination(t *testing.T) { n.Advance() } +func TestCommitPaginationWithAsyncStorageWrites(t *testing.T) { + s := newTestMemoryStorage(withPeers(1)) + cfg := newTestConfig(1, 10, 1, s) + cfg.MaxCommittedSizePerReady = 2048 + cfg.AsyncStorageWrites = true + ctx, cancel, n := newNodeTestHarness(context.Background(), t, cfg) + defer cancel() + n.Campaign(ctx) + + // Persist vote. + rd := readyWithTimeout(n) + require.Len(t, rd.Messages, 1) + m := rd.Messages[0] + require.Equal(t, raftpb.MsgStorageAppend, m.Type) + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + // Append empty entry. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 1) + m = rd.Messages[0] + require.Equal(t, raftpb.MsgStorageAppend, m.Type) + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + // Apply empty entry. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 2) + for _, m := range rd.Messages { + switch m.Type { + case raftpb.MsgStorageAppend: + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + case raftpb.MsgStorageApply: + if len(m.Entries) != 1 { + t.Fatalf("expected 1 (empty) entry, got %d", len(m.Entries)) + } + require.Len(t, m.Responses, 1) + require.NoError(t, n.Step(ctx, m.Responses[0])) + default: + t.Fatalf("unexpected: %v", m) + } + } + + // Propose first entry. + blob := []byte(strings.Repeat("a", 1024)) + require.NoError(t, n.Propose(ctx, blob)) + + // Append first entry. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 1) + m = rd.Messages[0] + require.Equal(t, raftpb.MsgStorageAppend, m.Type) + require.Len(t, m.Entries, 1) + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + + // Propose second entry. + require.NoError(t, n.Propose(ctx, blob)) + + // Append second entry. Don't apply first entry yet. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 2) + var applyResps []raftpb.Message + for _, m := range rd.Messages { + switch m.Type { + case raftpb.MsgStorageAppend: + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + case raftpb.MsgStorageApply: + if len(m.Entries) != 1 { + t.Fatalf("expected 1 (empty) entry, got %d", len(m.Entries)) + } + require.Len(t, m.Responses, 1) + applyResps = append(applyResps, m.Responses[0]) + default: + t.Fatalf("unexpected: %v", m) + } + } + + // Propose third entry. + require.NoError(t, n.Propose(ctx, blob)) + + // Append third entry. Don't apply second entry yet. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 2) + for _, m := range rd.Messages { + switch m.Type { + case raftpb.MsgStorageAppend: + require.NoError(t, s.Append(m.Entries)) + for _, resp := range m.Responses { + require.NoError(t, n.Step(ctx, resp)) + } + case raftpb.MsgStorageApply: + if len(m.Entries) != 1 { + t.Fatalf("expected 1 (empty) entry, got %d", len(m.Entries)) + } + require.Len(t, m.Responses, 1) + applyResps = append(applyResps, m.Responses[0]) + default: + t.Fatalf("unexpected: %v", m) + } + } + + // Third entry should not be returned to be applied until first entry's + // application is acknowledged. + drain := true + for drain { + select { + case rd := <-n.Ready(): + for _, m := range rd.Messages { + if m.Type == raftpb.MsgStorageApply { + t.Fatalf("expected MsgStorageApply, %v", m) + } + } + case <-time.After(10 * time.Millisecond): + drain = false + } + } + + // Acknowledged first entry application. + require.NoError(t, n.Step(ctx, applyResps[0])) + applyResps = applyResps[1:] + + // Third entry now returned for application. + rd = readyWithTimeout(n) + require.Len(t, rd.Messages, 1) + m = rd.Messages[0] + require.Equal(t, raftpb.MsgStorageApply, m.Type) + require.Len(t, m.Entries, 1) + applyResps = append(applyResps, m.Responses[0]) + + // Acknowledged second and third entry application. + for _, resp := range applyResps { + require.NoError(t, n.Step(ctx, resp)) + } + applyResps = nil +} + type ignoreSizeHintMemStorage struct { *MemoryStorage } diff --git a/raft.go b/raft.go index fd7cfddfab62..ee7859456ae6 100644 --- a/raft.go +++ b/raft.go @@ -31,9 +31,18 @@ import ( "go.etcd.io/raft/v3/tracker" ) -// None is a placeholder node ID used when there is no leader. -const None uint64 = 0 -const noLimit = math.MaxUint64 +const ( + // None is a placeholder node ID used when there is no leader. + None uint64 = 0 + // LocalAppendThread is a reference to a local thread that saves unstable + // log entries and snapshots to stable storage. The identifier is used as a + // target for MsgStorageAppend messages when AsyncStorageWrites is enabled. + LocalAppendThread uint64 = math.MaxUint64 + // LocalApplyThread is a reference to a local thread that applies committed + // log entries to the local state machine. The identifier is used as a + // target for MsgStorageApply messages when AsyncStorageWrites is enabled. + LocalApplyThread uint64 = math.MaxUint64 - 1 +) // Possible values for StateType. const ( @@ -70,6 +79,8 @@ const ( campaignTransfer CampaignType = "CampaignTransfer" ) +const noLimit = math.MaxUint64 + // ErrProposalDropped is returned when the proposal is ignored by some cases, // so that the proposer can be notified and fail fast. var ErrProposalDropped = errors.New("raft proposal dropped") @@ -140,6 +151,42 @@ type Config struct { // applied entries. This is a very application dependent configuration. Applied uint64 + // AsyncStorageWrites configures the raft node to write to its local storage + // (raft log and state machine) using a request/response message passing + // interface instead of the default Ready/Advance function call interface. + // Local storage messages can be pipelined and processed asynchronously + // (with respect to Ready iteration), facilitating reduced interference + // between Raft proposals and increased batching of log appends and state + // machine application. As a result, use of asynchronous storage writes can + // reduce end-to-end commit latency and increase maximum throughput. + // + // When true, the Ready.Message slice will include MsgStorageAppend and + // MsgStorageApply messages. The messages will target a LocalAppendThread + // and a LocalApplyThread, respectively. Messages to the same target must be + // reliably processed in order. In other words, they can't be dropped (like + // messages over the network) and those targeted at the same thread can't be + // reordered. Messages to different targets can be processed in any order. + // + // MsgStorageAppend carries Raft log entries to append, election votes / + // term changes / updated commit indexes to persist, and snapshots to apply. + // All writes performed in service of a MsgStorageAppend must be durable + // before response messages are delivered. However, if the MsgStorageAppend + // carries no response messages, durability is not required. The message + // assumes the role of the Entries, HardState, and Snapshot fields in Ready. + // + // MsgStorageApply carries committed entries to apply. Writes performed in + // service of a MsgStorageApply need not be durable before response messages + // are delivered. The message assumes the role of the CommittedEntries field + // in Ready. + // + // Local messages each carry one or more response messages which should be + // delivered after the corresponding storage write has been completed. These + // responses may target the same node or may target other nodes. The storage + // threads are not responsible for understanding the response messages, only + // for delivering them to the correct target after performing the storage + // write. + AsyncStorageWrites bool + // MaxSizePerMsg limits the max byte size of each append message. Smaller // value lowers the raft recovery cost(initial probing and message lost // during normal operation). On the other side, it might affect the @@ -147,7 +194,11 @@ type Config struct { // 0 for at most one entry per message. MaxSizePerMsg uint64 // MaxCommittedSizePerReady limits the size of the committed entries which - // can be applied. + // can be applying at the same time. + // + // Despite its name (preserved for compatibility), this quota applies across + // Ready structs to encompass all outstanding entries in unacknowledged + // MsgStorageApply messages when AsyncStorageWrites is enabled. MaxCommittedSizePerReady uint64 // MaxUncommittedEntriesSize limits the aggregate byte size of the // uncommitted entries that may be appended to a leader's log. Once this @@ -212,6 +263,9 @@ func (c *Config) validate() error { if c.ID == None { return errors.New("cannot use none as id") } + if IsLocalMsgTarget(c.ID) { + return errors.New("cannot use local target as id") + } if c.HeartbeatTick <= 0 { return errors.New("heartbeat tick must be greater than 0") @@ -266,8 +320,8 @@ type raft struct { // the log raftLog *raftLog - maxMsgSize uint64 - maxUncommittedSize uint64 + maxMsgSize entryEncodingSize + maxUncommittedSize entryPayloadSize // TODO(tbg): rename to trk. prs tracker.ProgressTracker @@ -276,7 +330,22 @@ type raft struct { // isLearner is true if the local raft node is a learner. isLearner bool + // msgs contains the list of messages that should be sent out immediately to + // other nodes. + // + // Messages in this list must target other nodes. msgs []pb.Message + // msgsAfterAppend contains the list of messages that should be sent after + // the accumulated unstable state (e.g. term, vote, []entry, and snapshot) + // has been persisted to durable storage. This includes waiting for any + // unstable state that is already in the process of being persisted (i.e. + // has already been handed out in a prior Ready struct) to complete. + // + // Messages in this list may target other nodes or may target this node. + // + // Messages in this list have the type MsgAppResp, MsgVoteResp, or + // MsgPreVoteResp. See the comment in raft.send for details. + msgsAfterAppend []pb.Message // the leader id lead uint64 @@ -293,7 +362,7 @@ type raft struct { // an estimate of the size of the uncommitted tail of the Raft log. Used to // prevent unbounded log growth. Only maintained by the leader. Reset on // term changes. - uncommittedSize uint64 + uncommittedSize entryPayloadSize readOnly *readOnly @@ -334,7 +403,7 @@ func newRaft(c *Config) *raft { if err := c.validate(); err != nil { panic(err.Error()) } - raftlog := newLogWithSize(c.Storage, c.Logger, c.MaxCommittedSizePerReady) + raftlog := newLogWithSize(c.Storage, c.Logger, entryEncodingSize(c.MaxCommittedSizePerReady)) hs, cs, err := c.Storage.InitialState() if err != nil { panic(err) // TODO(bdarnell) @@ -345,8 +414,8 @@ func newRaft(c *Config) *raft { lead: None, isLearner: false, raftLog: raftlog, - maxMsgSize: c.MaxSizePerMsg, - maxUncommittedSize: c.MaxUncommittedEntriesSize, + maxMsgSize: entryEncodingSize(c.MaxSizePerMsg), + maxUncommittedSize: entryPayloadSize(c.MaxUncommittedEntriesSize), prs: tracker.MakeProgressTracker(c.MaxInflightMsgs, c.MaxInflightBytes), electionTimeout: c.ElectionTick, heartbeatTimeout: c.HeartbeatTick, @@ -370,7 +439,7 @@ func newRaft(c *Config) *raft { r.loadState(hs) } if c.Applied > 0 { - raftlog.appliedTo(c.Applied) + raftlog.appliedTo(c.Applied, 0 /* size */) } r.becomeFollower(r.Term, None) @@ -416,11 +485,11 @@ func (r *raft) send(m pb.Message) { // - MsgPreVoteResp: m.Term is the term received in the original // MsgPreVote if the pre-vote was granted, non-zero for the // same reasons MsgPreVote is - panic(fmt.Sprintf("term should be set when sending %s", m.Type)) + r.logger.Panicf("term should be set when sending %s", m.Type) } } else { if m.Term != 0 { - panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term)) + r.logger.Panicf("term should not be set when sending %s (was %d)", m.Type, m.Term) } // do not attach term to MsgProp, MsgReadIndex // proposals are a way to forward to the leader and @@ -430,10 +499,59 @@ func (r *raft) send(m pb.Message) { m.Term = r.Term } } - if m.To == r.id { - r.logger.Panicf("message should not be self-addressed when sending %s", m.Type) + if m.Type == pb.MsgAppResp || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVoteResp { + // If async storage writes are enabled, messages added to the msgs slice + // are allowed to be sent out before unstable state (e.g. log entry + // writes and election votes) have been durably synced to the local + // disk. + // + // For most message types, this is not an issue. However, response + // messages that relate to "voting" on either leader election or log + // appends require durability before they can be sent. It would be + // incorrect to publish a vote in an election before that vote has been + // synced to stable storage locally. Similarly, it would be incorrect to + // acknowledge a log append to the leader before that entry has been + // synced to stable storage locally. + // + // Per the Raft thesis, section 3.8 Persisted state and server restarts: + // + // > Raft servers must persist enough information to stable storage to + // > survive server restarts safely. In particular, each server persists + // > its current term and vote; this is necessary to prevent the server + // > from voting twice in the same term or replacing log entries from a + // > newer leader with those from a deposed leader. Each server also + // > persists new log entries before they are counted towards the entries’ + // > commitment; this prevents committed entries from being lost or + // > “uncommitted” when servers restart + // + // To enforce this durability requirement, these response messages are + // queued to be sent out as soon as the current collection of unstable + // state (the state that the response message was predicated upon) has + // been durably persisted. This unstable state may have already been + // passed to a Ready struct whose persistence is in progress or may be + // waiting for the next Ready struct to begin being written to Storage. + // These messages must wait for all of this state to be durable before + // being published. + // + // Rejected responses (m.Reject == true) present an interesting case + // where the durability requirement is less unambiguous. A rejection may + // be predicated upon unstable state. For instance, a node may reject a + // vote for one peer because it has already begun syncing its vote for + // another peer. Or it may reject a vote from one peer because it has + // unstable log entries that indicate that the peer is behind on its + // log. In these cases, it is likely safe to send out the rejection + // response immediately without compromising safety in the presence of a + // server restart. However, because these rejections are rare and + // because the safety of such behavior has not been formally verified, + // we err on the side of safety and omit a `&& !m.Reject` condition + // above. + r.msgsAfterAppend = append(r.msgsAfterAppend, m) + } else { + if m.To == r.id { + r.logger.Panicf("message should not be self-addressed when sending %s", m.Type) + } + r.msgs = append(r.msgs, m) } - r.msgs = append(r.msgs, m) } // sendAppend sends an append RPC with new entries (if any) and the @@ -499,7 +617,7 @@ func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool { // Send the actual MsgApp otherwise, and update the progress accordingly. next := pr.Next // save Next for later, as the progress update can change it - if err := pr.UpdateOnEntriesSend(len(ents), payloadsSize(ents), next); err != nil { + if err := pr.UpdateOnEntriesSend(len(ents), uint64(payloadsSize(ents)), next); err != nil { r.logger.Panicf("%x: %v", r.id, err) } r.send(pb.Message{ @@ -562,62 +680,41 @@ func (r *raft) bcastHeartbeatWithCtx(ctx []byte) { }) } -func (r *raft) advance(rd Ready) { - r.reduceUncommittedSize(rd.CommittedEntries) - - // If entries were applied (or a snapshot), update our cursor for - // the next Ready. Note that if the current HardState contains a - // new Commit index, this does not mean that we're also applying - // all of the new entries due to commit pagination by size. - if newApplied := rd.appliedCursor(); newApplied > 0 { - r.raftLog.appliedTo(newApplied) - - if r.prs.Config.AutoLeave && newApplied >= r.pendingConfIndex && r.state == StateLeader { - // If the current (and most recent, at least for this leader's term) - // configuration should be auto-left, initiate that now. We use a - // nil Data which unmarshals into an empty ConfChangeV2 and has the - // benefit that appendEntry can never refuse it based on its size - // (which registers as zero). - m, err := confChangeToMsg(nil) - if err != nil { - panic(err) - } - // NB: this proposal can't be dropped due to size, but can be - // dropped if a leadership transfer is in progress. We'll keep - // checking this condition on each applied entry, so either the - // leadership transfer will succeed and the new leader will leave - // the joint configuration, or the leadership transfer will fail, - // and we will propose the config change on the next advance. - if err := r.Step(m); err != nil { - r.logger.Debugf("not initiating automatic transition out of joint configuration %s: %v", r.prs.Config, err) - } else { - r.logger.Infof("initiating automatic transition out of joint configuration %s", r.prs.Config) - } +func (r *raft) appliedTo(index uint64, size entryEncodingSize) { + oldApplied := r.raftLog.applied + newApplied := max(index, oldApplied) + r.raftLog.appliedTo(newApplied, size) + + if r.prs.Config.AutoLeave && newApplied >= r.pendingConfIndex && r.state == StateLeader { + // If the current (and most recent, at least for this leader's term) + // configuration should be auto-left, initiate that now. We use a + // nil Data which unmarshals into an empty ConfChangeV2 and has the + // benefit that appendEntry can never refuse it based on its size + // (which registers as zero). + m, err := confChangeToMsg(nil) + if err != nil { + panic(err) } - } - - if len(rd.Entries) > 0 { - e := rd.Entries[len(rd.Entries)-1] - if r.id == r.lead { - // The leader needs to self-ack the entries just appended (since it doesn't - // send an MsgApp to itself). This is roughly equivalent to: - // - // r.prs.Progress[r.id].MaybeUpdate(e.Index) - // if r.maybeCommit() { - // r.bcastAppend() - // } - _ = r.Step(pb.Message{From: r.id, Type: pb.MsgAppResp, Index: e.Index}) + // NB: this proposal can't be dropped due to size, but can be + // dropped if a leadership transfer is in progress. We'll keep + // checking this condition on each applied entry, so either the + // leadership transfer will succeed and the new leader will leave + // the joint configuration, or the leadership transfer will fail, + // and we will propose the config change on the next advance. + if err := r.Step(m); err != nil { + r.logger.Debugf("not initiating automatic transition out of joint configuration %s: %v", r.prs.Config, err) + } else { + r.logger.Infof("initiating automatic transition out of joint configuration %s", r.prs.Config) } - // NB: it's important for performance that this call happens after - // r.Step above on the leader. This is because r.Step can then use - // a fast-path for `r.raftLog.term()`. - r.raftLog.stableTo(e.Index, e.Term) - } - if !IsEmptySnap(rd.Snapshot) { - r.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index) } } +func (r *raft) appliedSnap(snap *pb.Snapshot) { + index := snap.Metadata.Index + r.raftLog.stableSnapTo(index) + r.appliedTo(index, 0 /* size */) +} + // maybeCommit attempts to advance the commit index. Returns true if // the commit index changed (in which case the caller should call // r.bcastAppend). @@ -673,7 +770,18 @@ func (r *raft) appendEntry(es ...pb.Entry) (accepted bool) { return false } // use latest "last" index after truncate/append - r.raftLog.append(es...) + li = r.raftLog.append(es...) + // The leader needs to self-ack the entries just appended once they have + // been durably persisted (since it doesn't send an MsgApp to itself). This + // response message will be added to msgsAfterAppend and delivered back to + // this node after these entries have been written to stable storage. When + // handled, this is roughly equivalent to: + // + // r.prs.Progress[r.id].MaybeUpdate(e.Index) + // if r.maybeCommit() { + // r.bcastAppend() + // } + r.send(pb.Message{To: r.id, Type: pb.MsgAppResp, Index: li}) return true } @@ -793,7 +901,7 @@ func (r *raft) becomeLeader() { // uncommitted log quota. This is because we want to preserve the // behavior of allowing one entry larger than quota if the current // usage is zero. - r.reduceUncommittedSize([]pb.Entry{emptyEnt}) + r.reduceUncommittedSize(payloadSize(emptyEnt)) r.logger.Infof("%x became leader at term %d", r.id, r.Term) } @@ -840,16 +948,6 @@ func (r *raft) campaign(t CampaignType) { voteMsg = pb.MsgVote term = r.Term } - if _, _, res := r.poll(r.id, voteRespMsgType(voteMsg), true); res == quorum.VoteWon { - // We won the election after voting for ourselves (which must mean that - // this is a single-node cluster). Advance to the next state. - if t == campaignPreElection { - r.campaign(campaignElection) - } else { - r.becomeLeader() - } - return - } var ids []uint64 { idMap := r.prs.Voters.IDs() @@ -861,6 +959,12 @@ func (r *raft) campaign(t CampaignType) { } for _, id := range ids { if id == r.id { + // The candidate votes for itself and should account for this self + // vote once the vote has been durably persisted (since it doesn't + // send a MsgVote to itself). This response message will be added to + // msgsAfterAppend and delivered back to this node after the vote + // has been written to stable storage. + r.send(pb.Message{To: id, Term: term, Type: voteRespMsgType(voteMsg)}) continue } r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d", @@ -870,7 +974,7 @@ func (r *raft) campaign(t CampaignType) { if t == campaignTransfer { ctx = []byte(t) } - r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx}) + r.send(pb.Message{To: id, Term: term, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx}) } } @@ -951,6 +1055,21 @@ func (r *raft) Step(m pb.Message) error { r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d", r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term) r.send(pb.Message{To: m.From, Term: r.Term, Type: pb.MsgPreVoteResp, Reject: true}) + } else if m.Type == pb.MsgStorageAppendResp { + if m.Index != 0 { + // Don't consider the appended log entries to be stable because + // they may have been overwritten in the unstable log during a + // later term. See the comment in newStorageAppendResp for more + // about this race. + r.logger.Infof("%x [term: %d] ignored entry appends from a %s message with lower term [term: %d]", + r.id, r.Term, m.Type, m.Term) + } + if m.Snapshot != nil { + // Even if the snapshot applied under a different term, its + // application is still valid. Snapshots carry committed + // (term-independent) state. + r.appliedSnap(m.Snapshot) + } } else { // ignore other cases r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]", @@ -967,6 +1086,21 @@ func (r *raft) Step(m pb.Message) error { r.hup(campaignElection) } + case pb.MsgStorageAppendResp: + if m.Index != 0 { + r.raftLog.stableTo(m.Index, m.LogTerm) + } + if m.Snapshot != nil { + r.appliedSnap(m.Snapshot) + } + + case pb.MsgStorageApplyResp: + if len(m.Entries) > 0 { + index := m.Entries[len(m.Entries)-1].Index + r.appliedTo(index, entsSize(m.Entries)) + r.reduceUncommittedSize(payloadsSize(m.Entries)) + } + case pb.MsgVote, pb.MsgPreVote: // We can vote if this is a repeat of a vote we've already cast... canVote := r.Vote == m.From || @@ -1659,7 +1793,7 @@ func (r *raft) restore(s pb.Snapshot) bool { // which is true when its own id is in progress list. func (r *raft) promotable() bool { pr := r.prs.Progress[r.id] - return pr != nil && !pr.IsLearner && !r.raftLog.hasPendingSnapshot() + return pr != nil && !pr.IsLearner && !r.raftLog.hasNextOrInProgressSnapshot() } func (r *raft) applyConfChange(cc pb.ConfChangeV2) pb.ConfState { @@ -1821,12 +1955,8 @@ func (r *raft) increaseUncommittedSize(ents []pb.Entry) bool { // reduceUncommittedSize accounts for the newly committed entries by decreasing // the uncommitted entry size limit. -func (r *raft) reduceUncommittedSize(ents []pb.Entry) { - if r.uncommittedSize == 0 { - // Fast-path for followers, who do not track or enforce the limit. - return - } - if s := payloadsSize(ents); s > r.uncommittedSize { +func (r *raft) reduceUncommittedSize(s entryPayloadSize) { + if s > r.uncommittedSize { // uncommittedSize may underestimate the size of the uncommitted Raft // log tail but will never overestimate it. Saturate at 0 instead of // allowing overflow. @@ -1836,14 +1966,6 @@ func (r *raft) reduceUncommittedSize(ents []pb.Entry) { } } -func payloadsSize(ents []pb.Entry) uint64 { - var s uint64 - for _, e := range ents { - s += uint64(PayloadSize(e)) - } - return s -} - func numOfPendingConf(ents []pb.Entry) int { n := 0 for i := range ents { diff --git a/raft_paper_test.go b/raft_paper_test.go index b8000e4e772c..d7e9949c2389 100644 --- a/raft_paper_test.go +++ b/raft_paper_test.go @@ -162,6 +162,7 @@ func testNonleaderStartElection(t *testing.T, state StateType) { for i := 1; i < 2*et; i++ { r.tick() } + r.advanceMessagesAfterAppend() if r.Term != 2 { t.Errorf("term = %d, want 2", r.Term) @@ -218,6 +219,7 @@ func TestLeaderElectionInOneRoundRPC(t *testing.T) { r := newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(idsBySize(tt.size)...))) r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + r.advanceMessagesAfterAppend() for id, vote := range tt.votes { r.Step(pb.Message{From: id, To: 1, Term: r.Term, Type: pb.MsgVoteResp, Reject: !vote}) } @@ -253,7 +255,7 @@ func TestFollowerVote(t *testing.T) { r.Step(pb.Message{From: tt.nvote, To: 1, Term: 1, Type: pb.MsgVote}) - msgs := r.readMessages() + msgs := r.msgsAfterAppend wmsgs := []pb.Message{ {From: 1, To: tt.nvote, Term: 1, Type: pb.MsgVoteResp, Reject: tt.wreject}, } @@ -421,7 +423,7 @@ func TestLeaderStartReplication(t *testing.T) { if !reflect.DeepEqual(msgs, wmsgs) { t.Errorf("msgs = %+v, want %+v", msgs, wmsgs) } - if g := r.raftLog.unstableEntries(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.nextUnstableEnts(); !reflect.DeepEqual(g, wents) { t.Errorf("ents = %+v, want %+v", g, wents) } } @@ -450,7 +452,7 @@ func TestLeaderCommitEntry(t *testing.T) { t.Errorf("committed = %d, want %d", g, li+1) } wents := []pb.Entry{{Index: li + 1, Term: 1, Data: []byte("some data")}} - if g := r.raftLog.nextCommittedEnts(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.nextCommittedEnts(true); !reflect.DeepEqual(g, wents) { t.Errorf("nextCommittedEnts = %+v, want %+v", g, wents) } msgs := r.readMessages() @@ -495,11 +497,8 @@ func TestLeaderAcknowledgeCommit(t *testing.T) { commitNoopEntry(r, s) li := r.raftLog.lastIndex() r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) - - rd := newReady(r, &SoftState{}, pb.HardState{}) - s.Append(rd.Entries) - r.advance(rd) // simulate having appended entry on leader - for _, m := range rd.Messages { + r.advanceMessagesAfterAppend() + for _, m := range r.msgs { if tt.nonLeaderAcceptors[m.To] { r.Step(acceptAndReply(m)) } @@ -538,7 +537,7 @@ func TestLeaderCommitPrecedingEntries(t *testing.T) { li := uint64(len(tt)) wents := append(tt, pb.Entry{Term: 3, Index: li + 1}, pb.Entry{Term: 3, Index: li + 2, Data: []byte("some data")}) - if g := r.raftLog.nextCommittedEnts(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.nextCommittedEnts(true); !reflect.DeepEqual(g, wents) { t.Errorf("#%d: ents = %+v, want %+v", i, g, wents) } } @@ -590,7 +589,7 @@ func TestFollowerCommitEntry(t *testing.T) { t.Errorf("#%d: committed = %d, want %d", i, g, tt.commit) } wents := tt.ents[:int(tt.commit)] - if g := r.raftLog.nextCommittedEnts(); !reflect.DeepEqual(g, wents) { + if g := r.raftLog.nextCommittedEnts(true); !reflect.DeepEqual(g, wents) { t.Errorf("#%d: nextCommittedEnts = %v, want %v", i, g, wents) } } @@ -689,7 +688,7 @@ func TestFollowerAppendEntries(t *testing.T) { if g := r.raftLog.allEntries(); !reflect.DeepEqual(g, tt.wents) { t.Errorf("#%d: ents = %+v, want %+v", i, g, tt.wents) } - if g := r.raftLog.unstableEntries(); !reflect.DeepEqual(g, tt.wunstable) { + if g := r.raftLog.nextUnstableEnts(); !reflect.DeepEqual(g, tt.wunstable) { t.Errorf("#%d: unstableEnts = %+v, want %+v", i, g, tt.wunstable) } } @@ -894,9 +893,7 @@ func TestLeaderOnlyCommitsLogFromCurrentTerm(t *testing.T) { r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{}}}) r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Term: r.Term, Index: tt.index}) - rd := newReady(r, &SoftState{}, pb.HardState{}) - storage.Append(rd.Entries) - r.advance(rd) + r.advanceMessagesAfterAppend() if r.raftLog.committed != tt.wcommit { t.Errorf("#%d: commit = %d, want %d", i, r.raftLog.committed, tt.wcommit) } @@ -924,8 +921,8 @@ func commitNoopEntry(r *raft, s *MemoryStorage) { } // ignore further messages to refresh followers' commit index r.readMessages() - s.Append(r.raftLog.unstableEntries()) - r.raftLog.appliedTo(r.raftLog.committed) + s.Append(r.raftLog.nextUnstableEnts()) + r.raftLog.appliedTo(r.raftLog.committed, 0 /* size */) r.raftLog.stableTo(r.raftLog.lastIndex(), r.raftLog.lastTerm()) } diff --git a/raft_test.go b/raft_test.go index 7f4f660cee3e..6abda3cfe016 100644 --- a/raft_test.go +++ b/raft_test.go @@ -27,17 +27,19 @@ import ( "go.etcd.io/raft/v3/tracker" ) -// nextEnts returns the appliable entries and updates the applied index +// nextEnts returns the appliable entries and updates the applied index. func nextEnts(r *raft, s *MemoryStorage) (ents []pb.Entry) { - for { - rd := newReady(r, &SoftState{}, pb.HardState{}) - s.Append(rd.Entries) - r.advance(rd) - if len(rd.Entries)+len(rd.CommittedEntries) == 0 { - return ents - } - ents = append(ents, rd.CommittedEntries...) - } + // Append unstable entries. + s.Append(r.raftLog.nextUnstableEnts()) + r.raftLog.stableTo(r.raftLog.lastIndex(), r.raftLog.lastTerm()) + + // Run post-append steps. + r.advanceMessagesAfterAppend() + + // Return committed entries. + ents = r.raftLog.nextCommittedEnts(true) + r.raftLog.appliedTo(r.raftLog.committed, 0 /* size */) + return ents } func mustAppendEntry(r *raft, ents ...pb.Entry) { @@ -49,15 +51,45 @@ func mustAppendEntry(r *raft, ents ...pb.Entry) { type stateMachine interface { Step(m pb.Message) error readMessages() []pb.Message + advanceMessagesAfterAppend() } func (r *raft) readMessages() []pb.Message { + r.advanceMessagesAfterAppend() msgs := r.msgs - r.msgs = make([]pb.Message, 0) + r.msgs = nil + return msgs +} +func (r *raft) advanceMessagesAfterAppend() { + for { + msgs := r.takeMessagesAfterAppend() + if len(msgs) == 0 { + break + } + r.stepOrSend(msgs) + } +} + +func (r *raft) takeMessagesAfterAppend() []pb.Message { + msgs := r.msgsAfterAppend + r.msgsAfterAppend = nil return msgs } +func (r *raft) stepOrSend(msgs []pb.Message) error { + for _, m := range msgs { + if m.To == r.id { + if err := r.Step(m); err != nil { + return err + } + } else { + r.msgs = append(r.msgs, m) + } + } + return nil +} + func TestProgressLeader(t *testing.T) { s := newTestMemoryStorage(withPeers(1, 2)) r := newTestRaft(1, 5, 1, s) @@ -75,11 +107,11 @@ func TestProgressLeader(t *testing.T) { if m := r.prs.Progress[1].Match; m != 0 { t.Fatalf("expected zero match, got %d", m) } - rd := newReady(r, &SoftState{}, pb.HardState{}) - if len(rd.Entries) != 6 || len(rd.Entries[0].Data) > 0 || string(rd.Entries[5].Data) != "foo" { - t.Fatalf("unexpected Entries: %s", DescribeReady(rd, nil)) + ents := r.raftLog.nextUnstableEnts() + if len(ents) != 6 || len(ents[0].Data) > 0 || string(ents[5].Data) != "foo" { + t.Fatalf("unexpected entries: %v", ents) } - r.advance(rd) + r.advanceMessagesAfterAppend() if m := r.prs.Progress[1].Match; m != 6 { t.Fatalf("unexpected Match %d", m) } @@ -208,9 +240,9 @@ func TestUncommittedEntryLimit(t *testing.T) { // writing, the former). const maxEntries = 1024 testEntry := pb.Entry{Data: []byte("testdata")} - maxEntrySize := maxEntries * PayloadSize(testEntry) + maxEntrySize := maxEntries * payloadSize(testEntry) - if n := PayloadSize(pb.Entry{Data: nil}); n != 0 { + if n := payloadSize(pb.Entry{Data: nil}); n != 0 { t.Fatal("entry with no Data must have zero payload size") } @@ -251,7 +283,7 @@ func TestUncommittedEntryLimit(t *testing.T) { if e := maxEntries * numFollowers; len(ms) != e { t.Fatalf("expected %d messages, got %d", e, len(ms)) } - r.reduceUncommittedSize(propEnts) + r.reduceUncommittedSize(payloadsSize(propEnts)) if r.uncommittedSize != 0 { t.Fatalf("committed everything, but still tracking %d", r.uncommittedSize) } @@ -287,7 +319,7 @@ func TestUncommittedEntryLimit(t *testing.T) { if e := 2 * numFollowers; len(ms) != e { t.Fatalf("expected %d messages, got %d", e, len(ms)) } - r.reduceUncommittedSize(propEnts) + r.reduceUncommittedSize(payloadsSize(propEnts)) if n := r.uncommittedSize; n != 0 { t.Fatalf("expected zero uncommitted size, got %d", n) } @@ -383,6 +415,7 @@ func TestLearnerPromotion(t *testing.T) { for i := 0; i < n1.electionTimeout; i++ { n1.tick() } + n1.advanceMessagesAfterAppend() if n1.state != StateLeader { t.Errorf("peer 1 state: %s, want %s", n1.state, StateLeader) @@ -404,6 +437,7 @@ func TestLearnerPromotion(t *testing.T) { for i := 0; i < n2.electionTimeout; i++ { n2.tick() } + n2.advanceMessagesAfterAppend() nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgBeat}) @@ -424,10 +458,11 @@ func TestLearnerCanVote(t *testing.T) { n2.Step(pb.Message{From: 1, To: 2, Term: 2, Type: pb.MsgVote, LogTerm: 11, Index: 11}) - if len(n2.msgs) != 1 { - t.Fatalf("expected exactly one message, not %+v", n2.msgs) + msgs := n2.readMessages() + if len(msgs) != 1 { + t.Fatalf("expected exactly one message, not %+v", msgs) } - msg := n2.msgs[0] + msg := msgs[0] if msg.Type != pb.MsgVoteResp && !msg.Reject { t.Fatal("expected learner to not reject vote") } @@ -585,10 +620,11 @@ func testVoteFromAnyState(t *testing.T, vt pb.MessageType) { if err := r.Step(msg); err != nil { t.Errorf("%s,%s: Step failed: %s", vt, st, err) } - if len(r.msgs) != 1 { - t.Errorf("%s,%s: %d response messages, want 1: %+v", vt, st, len(r.msgs), r.msgs) + msgs := r.readMessages() + if len(msgs) != 1 { + t.Errorf("%s,%s: %d response messages, want 1: %+v", vt, st, len(msgs), msgs) } else { - resp := r.msgs[0] + resp := msgs[0] if resp.Type != voteRespMsgType(vt) { t.Errorf("%s,%s: response message is %s, want %s", vt, st, resp.Type, voteRespMsgType(vt)) @@ -701,6 +737,7 @@ func TestLearnerLogReplication(t *testing.T) { for i := 0; i < n1.electionTimeout; i++ { n1.tick() } + n1.advanceMessagesAfterAppend() nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgBeat}) @@ -715,20 +752,11 @@ func TestLearnerLogReplication(t *testing.T) { nextCommitted := uint64(2) { nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}}) - rd := newReady(n1, &SoftState{}, pb.HardState{}) - nt.send(rd.Messages...) - s1.Append(rd.Entries) - n1.advance(rd) } if n1.raftLog.committed != nextCommitted { t.Errorf("peer 1 wants committed to %d, but still %d", nextCommitted, n1.raftLog.committed) } - { - rd := newReady(n1, &SoftState{}, pb.HardState{}) - nt.send(rd.Messages...) - } - if n1.raftLog.committed != n2.raftLog.committed { t.Errorf("peer 2 wants committed to %d, but still %d", n1.raftLog.committed, n2.raftLog.committed) } @@ -748,10 +776,6 @@ func TestSingleNodeCommit(t *testing.T) { tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("some data")}}}) - rd := newReady(r, &SoftState{}, pb.HardState{}) - s.Append(rd.Entries) - r.advance(rd) - sm := tt.peers[1].(*raft) if sm.raftLog.committed != 3 { t.Errorf("committed = %d, want %d", sm.raftLog.committed, 3) @@ -1752,6 +1776,7 @@ func testCandidateResetTerm(t *testing.T, mt pb.MessageType) { for i := 0; i < c.randomizedElectionTimeout; i++ { c.tick() } + c.advanceMessagesAfterAppend() if c.state != StateCandidate { t.Errorf("state = %s, want %s", c.state, StateCandidate) @@ -1773,6 +1798,118 @@ func testCandidateResetTerm(t *testing.T, mt pb.MessageType) { } } +// The following three tests exercise the behavior of a (pre-)candidate when its +// own self-vote is delivered back to itself after the peer has already learned +// that it has lost the election. The self-vote should be ignored in these cases. + +func TestCandidateSelfVoteAfterLostElection(t *testing.T) { + testCandidateSelfVoteAfterLostElection(t, false) +} + +func TestCandidateSelfVoteAfterLostElectionPreVote(t *testing.T) { + testCandidateSelfVoteAfterLostElection(t, true) +} + +func testCandidateSelfVoteAfterLostElection(t *testing.T, preVote bool) { + sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) + sm.preVote = preVote + + // n1 calls an election. + sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + steps := sm.takeMessagesAfterAppend() + + // n1 hears that n2 already won the election before it has had a + // change to sync its vote to disk and account for its self-vote. + // Becomes a follower. + sm.Step(pb.Message{From: 2, To: 1, Term: sm.Term, Type: pb.MsgHeartbeat}) + if sm.state != StateFollower { + t.Errorf("state = %v, want %v", sm.state, StateFollower) + } + + // n1 remains a follower even after its self-vote is delivered. + sm.stepOrSend(steps) + if sm.state != StateFollower { + t.Errorf("state = %v, want %v", sm.state, StateFollower) + } + + // Its self-vote does not make its way to its ProgressTracker. + granted, _, _ := sm.prs.TallyVotes() + if granted != 0 { + t.Errorf("granted = %v, want %v", granted, 0) + } +} + +func TestCandidateDeliversPreCandidateSelfVoteAfterBecomingCandidate(t *testing.T) { + sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) + sm.preVote = true + + // n1 calls an election. + sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + if sm.state != StatePreCandidate { + t.Errorf("state = %v, want %v", sm.state, StatePreCandidate) + } + steps := sm.takeMessagesAfterAppend() + + // n1 receives pre-candidate votes from both other peers before + // voting for itself. n1 becomes a candidate. + // NB: pre-vote messages carry the local term + 1. + sm.Step(pb.Message{From: 2, To: 1, Term: sm.Term + 1, Type: pb.MsgPreVoteResp}) + sm.Step(pb.Message{From: 3, To: 1, Term: sm.Term + 1, Type: pb.MsgPreVoteResp}) + if sm.state != StateCandidate { + t.Errorf("state = %v, want %v", sm.state, StateCandidate) + } + + // n1 remains a candidate even after its delayed pre-vote self-vote is + // delivered. + sm.stepOrSend(steps) + if sm.state != StateCandidate { + t.Errorf("state = %v, want %v", sm.state, StateCandidate) + } + steps = sm.takeMessagesAfterAppend() + + // Its pre-vote self-vote does not make its way to its ProgressTracker. + granted, _, _ := sm.prs.TallyVotes() + if granted != 0 { + t.Errorf("granted = %v, want %v", granted, 0) + } + + // A single vote from n2 does not move n1 to the leader. + sm.Step(pb.Message{From: 2, To: 1, Term: sm.Term, Type: pb.MsgVoteResp}) + if sm.state != StateCandidate { + t.Errorf("state = %v, want %v", sm.state, StateCandidate) + } + + // n1 becomes the leader once its self-vote is received because now + // quorum is reached. + sm.stepOrSend(steps) + if sm.state != StateLeader { + t.Errorf("state = %v, want %v", sm.state, StateLeader) + } +} + +func TestLeaderMsgAppSelfAckAfterTermChange(t *testing.T) { + sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) + sm.becomeCandidate() + sm.becomeLeader() + + // n1 proposes a write. + sm.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}}) + steps := sm.takeMessagesAfterAppend() + + // n1 hears that n2 is the new leader. + sm.Step(pb.Message{From: 2, To: 1, Term: sm.Term + 1, Type: pb.MsgHeartbeat}) + if sm.state != StateFollower { + t.Errorf("state = %v, want %v", sm.state, StateFollower) + } + + // n1 advances, ignoring its earlier self-ack of its MsgApp. The + // corresponding MsgAppResp is ignored because it carries an earlier term. + sm.stepOrSend(steps) + if sm.state != StateFollower { + t.Errorf("state = %v, want %v", sm.state, StateFollower) + } +} + func TestLeaderStepdownWhenQuorumActive(t *testing.T) { sm := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2, 3))) @@ -2531,6 +2668,8 @@ func TestBcastBeat(t *testing.T) { for i := 0; i < 10; i++ { mustAppendEntry(sm, pb.Entry{Index: uint64(i) + 1}) } + sm.advanceMessagesAfterAppend() + // slow follower sm.prs.Progress[2].Match, sm.prs.Progress[2].Next = 5, 6 // normal follower @@ -2958,12 +3097,9 @@ func TestLearnerReceiveSnapshot(t *testing.T) { n2 := newTestLearnerRaft(2, 10, 1, newTestMemoryStorage(withPeers(1), withLearners(2))) n1.restore(s) - ready := newReady(n1, &SoftState{}, pb.HardState{}) - store.ApplySnapshot(ready.Snapshot) - n1.advance(ready) - - // Force set n1 appplied index. - n1.raftLog.appliedTo(n1.raftLog.committed) + snap := n1.raftLog.nextUnstableSnapshot() + store.ApplySnapshot(*snap) + n1.appliedSnap(snap) nt := newNetwork(n1, n2) @@ -3382,11 +3518,13 @@ func testCampaignWhileLeader(t *testing.T, preVote bool) { // We don't call campaign() directly because it comes after the check // for our current state. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + r.advanceMessagesAfterAppend() if r.state != StateLeader { t.Errorf("expected single-node election to become leader but got %s", r.state) } term := r.Term r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) + r.advanceMessagesAfterAppend() if r.state != StateLeader { t.Errorf("expected to remain leader but got %s", r.state) } @@ -3607,9 +3745,9 @@ func TestLeaderTransferAfterSnapshot(t *testing.T) { // Apply snapshot and resume progress follower := nt.peers[3].(*raft) - ready := newReady(follower, &SoftState{}, pb.HardState{}) - nt.storage[3].ApplySnapshot(ready.Snapshot) - follower.advance(ready) + snap := follower.raftLog.nextUnstableSnapshot() + nt.storage[3].ApplySnapshot(*snap) + follower.appliedSnap(snap) nt.msgHook = nil nt.send(filtered) @@ -4753,7 +4891,8 @@ func (nw *network) send(msgs ...pb.Message) { if nw.t != nil { nw.t.Log(DescribeMessage(m, nil)) } - p.Step(m) + _ = p.Step(m) + p.advanceMessagesAfterAppend() msgs = append(msgs[1:], nw.filter(p.readMessages())...) } } @@ -4818,8 +4957,9 @@ type connem struct { type blackHole struct{} -func (blackHole) Step(pb.Message) error { return nil } -func (blackHole) readMessages() []pb.Message { return nil } +func (blackHole) Step(pb.Message) error { return nil } +func (blackHole) readMessages() []pb.Message { return nil } +func (blackHole) advanceMessagesAfterAppend() {} var nopStepper = &blackHole{} diff --git a/raftpb/raft.pb.go b/raftpb/raft.pb.go index 1829dfc557e9..148026397933 100644 --- a/raftpb/raft.pb.go +++ b/raftpb/raft.pb.go @@ -72,25 +72,29 @@ func (EntryType) EnumDescriptor() ([]byte, []int) { type MessageType int32 const ( - MsgHup MessageType = 0 - MsgBeat MessageType = 1 - MsgProp MessageType = 2 - MsgApp MessageType = 3 - MsgAppResp MessageType = 4 - MsgVote MessageType = 5 - MsgVoteResp MessageType = 6 - MsgSnap MessageType = 7 - MsgHeartbeat MessageType = 8 - MsgHeartbeatResp MessageType = 9 - MsgUnreachable MessageType = 10 - MsgSnapStatus MessageType = 11 - MsgCheckQuorum MessageType = 12 - MsgTransferLeader MessageType = 13 - MsgTimeoutNow MessageType = 14 - MsgReadIndex MessageType = 15 - MsgReadIndexResp MessageType = 16 - MsgPreVote MessageType = 17 - MsgPreVoteResp MessageType = 18 + MsgHup MessageType = 0 + MsgBeat MessageType = 1 + MsgProp MessageType = 2 + MsgApp MessageType = 3 + MsgAppResp MessageType = 4 + MsgVote MessageType = 5 + MsgVoteResp MessageType = 6 + MsgSnap MessageType = 7 + MsgHeartbeat MessageType = 8 + MsgHeartbeatResp MessageType = 9 + MsgUnreachable MessageType = 10 + MsgSnapStatus MessageType = 11 + MsgCheckQuorum MessageType = 12 + MsgTransferLeader MessageType = 13 + MsgTimeoutNow MessageType = 14 + MsgReadIndex MessageType = 15 + MsgReadIndexResp MessageType = 16 + MsgPreVote MessageType = 17 + MsgPreVoteResp MessageType = 18 + MsgStorageAppend MessageType = 19 + MsgStorageAppendResp MessageType = 20 + MsgStorageApply MessageType = 21 + MsgStorageApplyResp MessageType = 22 ) var MessageType_name = map[int32]string{ @@ -113,28 +117,36 @@ var MessageType_name = map[int32]string{ 16: "MsgReadIndexResp", 17: "MsgPreVote", 18: "MsgPreVoteResp", + 19: "MsgStorageAppend", + 20: "MsgStorageAppendResp", + 21: "MsgStorageApply", + 22: "MsgStorageApplyResp", } var MessageType_value = map[string]int32{ - "MsgHup": 0, - "MsgBeat": 1, - "MsgProp": 2, - "MsgApp": 3, - "MsgAppResp": 4, - "MsgVote": 5, - "MsgVoteResp": 6, - "MsgSnap": 7, - "MsgHeartbeat": 8, - "MsgHeartbeatResp": 9, - "MsgUnreachable": 10, - "MsgSnapStatus": 11, - "MsgCheckQuorum": 12, - "MsgTransferLeader": 13, - "MsgTimeoutNow": 14, - "MsgReadIndex": 15, - "MsgReadIndexResp": 16, - "MsgPreVote": 17, - "MsgPreVoteResp": 18, + "MsgHup": 0, + "MsgBeat": 1, + "MsgProp": 2, + "MsgApp": 3, + "MsgAppResp": 4, + "MsgVote": 5, + "MsgVoteResp": 6, + "MsgSnap": 7, + "MsgHeartbeat": 8, + "MsgHeartbeatResp": 9, + "MsgUnreachable": 10, + "MsgSnapStatus": 11, + "MsgCheckQuorum": 12, + "MsgTransferLeader": 13, + "MsgTimeoutNow": 14, + "MsgReadIndex": 15, + "MsgReadIndexResp": 16, + "MsgPreVote": 17, + "MsgPreVoteResp": 18, + "MsgStorageAppend": 19, + "MsgStorageAppendResp": 20, + "MsgStorageApply": 21, + "MsgStorageApplyResp": 22, } func (x MessageType) Enum() *MessageType { @@ -386,14 +398,25 @@ type Message struct { From uint64 `protobuf:"varint,3,opt,name=from" json:"from"` Term uint64 `protobuf:"varint,4,opt,name=term" json:"term"` // logTerm is generally used for appending Raft logs to followers. For example, - // (type=MsgApp,index=100,logTerm=5) means leader appends entries starting at - // index=101, and the term of entry at index 100 is 5. + // (type=MsgApp,index=100,logTerm=5) means the leader appends entries starting + // at index=101, and the term of the entry at index 100 is 5. // (type=MsgAppResp,reject=true,index=100,logTerm=5) means follower rejects some // entries from its leader as it already has an entry with term 5 at index 100. + // (type=MsgStorageAppendResp,index=100,logTerm=5) means the local node wrote + // entries up to index=100 in stable storage, and the term of the entry at index + // 100 was 5. This doesn't always mean that the corresponding MsgStorageAppend + // message was the one that carried these entries, just that those entries were + // stable at the time of processing the corresponding MsgStorageAppend. LogTerm uint64 `protobuf:"varint,5,opt,name=logTerm" json:"logTerm"` Index uint64 `protobuf:"varint,6,opt,name=index" json:"index"` Entries []Entry `protobuf:"bytes,7,rep,name=entries" json:"entries"` Commit uint64 `protobuf:"varint,8,opt,name=commit" json:"commit"` + // (type=MsgStorageAppend,vote=5,term=10) means the local node is voting for + // peer 5 in term 10. For MsgStorageAppends, the term, vote, and commit fields + // will either all be set (to facilitate the construction of a HardState) if + // any of the fields have changed or will all be unset if none of the fields + // have changed. + Vote uint64 `protobuf:"varint,13,opt,name=vote" json:"vote"` // snapshot is non-nil and non-empty for MsgSnap messages and nil for all other // message types. However, peer nodes running older binary versions may send a // non-nil, empty value for the snapshot field of non-MsgSnap messages. Code @@ -402,6 +425,10 @@ type Message struct { Reject bool `protobuf:"varint,10,opt,name=reject" json:"reject"` RejectHint uint64 `protobuf:"varint,11,opt,name=rejectHint" json:"rejectHint"` Context []byte `protobuf:"bytes,12,opt,name=context" json:"context,omitempty"` + // responses are populated by a raft node to instruct storage threads on how + // to respond and who to respond to when the work associated with a message + // is complete. Populated for MsgStorageAppend and MsgStorageApply messages. + Responses []Message `protobuf:"bytes,14,rep,name=responses" json:"responses"` } func (m *Message) Reset() { *m = Message{} } @@ -699,72 +726,76 @@ func init() { func init() { proto.RegisterFile("raft.proto", fileDescriptor_b042552c306ae59b) } var fileDescriptor_b042552c306ae59b = []byte{ - // 1028 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcd, 0x6e, 0xdb, 0x46, - 0x17, 0xe5, 0x50, 0xb4, 0x7e, 0xae, 0x64, 0x79, 0x7c, 0xe3, 0x2f, 0x20, 0x0c, 0x43, 0xd1, 0xa7, - 0xa4, 0x88, 0xe0, 0x22, 0x6e, 0xa1, 0x45, 0x51, 0x74, 0xe7, 0x9f, 0x00, 0x76, 0x61, 0xb9, 0xa9, - 0xec, 0x78, 0x51, 0xa0, 0x30, 0xc6, 0xe2, 0x88, 0x66, 0x2b, 0x72, 0x08, 0x72, 0xe4, 0xda, 0x9b, - 0xa2, 0xe8, 0x13, 0x74, 0xd9, 0x4d, 0xb6, 0x7d, 0x80, 0x3e, 0x85, 0x97, 0x06, 0xba, 0xe9, 0x2a, - 0x68, 0xec, 0x17, 0x29, 0x66, 0x38, 0x94, 0x28, 0xd9, 0xc8, 0xa2, 0xbb, 0x99, 0x73, 0xcf, 0xdc, - 0x39, 0xe7, 0xde, 0xcb, 0x21, 0x40, 0xc2, 0x46, 0x72, 0x2b, 0x4e, 0x84, 0x14, 0x58, 0x56, 0xeb, - 0xf8, 0x7c, 0x7d, 0xcd, 0x17, 0xbe, 0xd0, 0xd0, 0x67, 0x6a, 0x95, 0x45, 0x3b, 0x3f, 0xc3, 0xd2, - 0xeb, 0x48, 0x26, 0xd7, 0xe8, 0x82, 0x73, 0xc2, 0x93, 0xd0, 0xb5, 0xdb, 0xa4, 0xeb, 0xec, 0x38, - 0x37, 0xef, 0x9f, 0x59, 0x03, 0x8d, 0xe0, 0x3a, 0x2c, 0x1d, 0x44, 0x1e, 0xbf, 0x72, 0x4b, 0x85, - 0x50, 0x06, 0xe1, 0xa7, 0xe0, 0x9c, 0x5c, 0xc7, 0xdc, 0x25, 0x6d, 0xd2, 0x6d, 0xf6, 0x56, 0xb7, - 0xb2, 0xbb, 0xb6, 0x74, 0x4a, 0x15, 0x98, 0x26, 0xba, 0x8e, 0x39, 0x22, 0x38, 0x7b, 0x4c, 0x32, - 0xd7, 0x69, 0x93, 0x6e, 0x63, 0xa0, 0xd7, 0x9d, 0x5f, 0x08, 0xd0, 0xe3, 0x88, 0xc5, 0xe9, 0x85, - 0x90, 0x7d, 0x2e, 0x99, 0xc7, 0x24, 0xc3, 0x2f, 0x00, 0x86, 0x22, 0x1a, 0x9d, 0xa5, 0x92, 0xc9, - 0x2c, 0x77, 0x7d, 0x96, 0x7b, 0x57, 0x44, 0xa3, 0x63, 0x15, 0x30, 0xb9, 0x6b, 0xc3, 0x1c, 0x50, - 0x4a, 0x03, 0xad, 0xb4, 0x68, 0x22, 0x83, 0x94, 0x3f, 0xa9, 0xfc, 0x15, 0x4d, 0x68, 0xa4, 0xf3, - 0x1d, 0x54, 0x73, 0x05, 0x4a, 0xa2, 0x52, 0xa0, 0xef, 0x6c, 0x0c, 0xf4, 0x1a, 0xbf, 0x82, 0x6a, - 0x68, 0x94, 0xe9, 0xc4, 0xf5, 0x9e, 0x9b, 0x6b, 0x59, 0x54, 0x6e, 0xf2, 0x4e, 0xf9, 0x9d, 0x77, - 0x25, 0xa8, 0xf4, 0x79, 0x9a, 0x32, 0x9f, 0xe3, 0x2b, 0x70, 0xe4, 0xac, 0x56, 0x4f, 0xf2, 0x1c, - 0x26, 0x5c, 0xac, 0x96, 0xa2, 0xe1, 0x1a, 0xd8, 0x52, 0xcc, 0x39, 0xb1, 0xa5, 0x50, 0x36, 0x46, - 0x89, 0x58, 0xb0, 0xa1, 0x90, 0xa9, 0x41, 0x67, 0xd1, 0x20, 0xb6, 0xa0, 0x32, 0x16, 0xbe, 0xee, - 0xee, 0x52, 0x21, 0x98, 0x83, 0xb3, 0xb2, 0x95, 0x1f, 0x96, 0xed, 0x15, 0x54, 0x78, 0x24, 0x93, - 0x80, 0xa7, 0x6e, 0xa5, 0x5d, 0xea, 0xd6, 0x7b, 0xcb, 0x73, 0x3d, 0xce, 0x53, 0x19, 0x0e, 0x6e, - 0x40, 0x79, 0x28, 0xc2, 0x30, 0x90, 0x6e, 0xb5, 0x90, 0xcb, 0x60, 0xd8, 0x83, 0x6a, 0x6a, 0x2a, - 0xe6, 0xd6, 0x74, 0x25, 0xe9, 0x62, 0x25, 0xf5, 0x09, 0x32, 0x98, 0xf2, 0x54, 0xc6, 0x84, 0xff, - 0xc0, 0x87, 0xd2, 0x85, 0x36, 0xe9, 0x56, 0xf3, 0x8c, 0x19, 0x86, 0x2f, 0x00, 0xb2, 0xd5, 0x7e, - 0x10, 0x49, 0xb7, 0x5e, 0xb8, 0xb3, 0x80, 0xa3, 0x0b, 0x95, 0xa1, 0x88, 0x24, 0xbf, 0x92, 0x6e, - 0x43, 0x37, 0x36, 0xdf, 0x76, 0xbe, 0x87, 0xda, 0x3e, 0x4b, 0xbc, 0x6c, 0x7c, 0xf2, 0x0a, 0x92, - 0x07, 0x15, 0x74, 0xc1, 0xb9, 0x14, 0x92, 0xcf, 0x7f, 0x1c, 0x0a, 0x29, 0x18, 0x2e, 0x3d, 0x34, - 0xdc, 0xf9, 0x93, 0x40, 0x6d, 0x3a, 0xaf, 0xf8, 0x14, 0xca, 0xea, 0x4c, 0x92, 0xba, 0xa4, 0x5d, - 0xea, 0x3a, 0x03, 0xb3, 0xc3, 0x75, 0xa8, 0x8e, 0x39, 0x4b, 0x22, 0x15, 0xb1, 0x75, 0x64, 0xba, - 0xc7, 0x97, 0xb0, 0x92, 0xb1, 0xce, 0xc4, 0x44, 0xfa, 0x22, 0x88, 0x7c, 0xb7, 0xa4, 0x29, 0xcd, - 0x0c, 0xfe, 0xc6, 0xa0, 0xf8, 0x1c, 0x96, 0xf3, 0x43, 0x67, 0x91, 0x72, 0xea, 0x68, 0x5a, 0x23, - 0x07, 0x8f, 0xf8, 0x95, 0xc4, 0xe7, 0x00, 0x6c, 0x22, 0xc5, 0xd9, 0x98, 0xb3, 0x4b, 0xae, 0x87, - 0x21, 0x2f, 0x68, 0x4d, 0xe1, 0x87, 0x0a, 0xee, 0xbc, 0x23, 0x00, 0x4a, 0xf4, 0xee, 0x05, 0x8b, - 0x7c, 0x8e, 0x9f, 0x9b, 0xb1, 0xb5, 0xf5, 0xd8, 0x3e, 0x2d, 0x7e, 0x86, 0x19, 0xe3, 0xc1, 0xe4, - 0xbe, 0x84, 0x4a, 0x24, 0x3c, 0x7e, 0x16, 0x78, 0xa6, 0x28, 0x4d, 0x15, 0xbc, 0x7b, 0xff, 0xac, - 0x7c, 0x24, 0x3c, 0x7e, 0xb0, 0x37, 0x28, 0xab, 0xf0, 0x81, 0x57, 0xec, 0x8b, 0x33, 0xd7, 0x17, - 0x5c, 0x07, 0x3b, 0xf0, 0x4c, 0x23, 0xc0, 0x9c, 0xb6, 0x0f, 0xf6, 0x06, 0x76, 0xe0, 0x75, 0x42, - 0xa0, 0xb3, 0xcb, 0x8f, 0x83, 0xc8, 0x1f, 0xcf, 0x44, 0x92, 0xff, 0x22, 0xd2, 0xfe, 0x98, 0xc8, - 0xce, 0x1f, 0x04, 0x1a, 0xb3, 0x3c, 0xa7, 0x3d, 0xdc, 0x01, 0x90, 0x09, 0x8b, 0xd2, 0x40, 0x06, - 0x22, 0x32, 0x37, 0x6e, 0x3c, 0x72, 0xe3, 0x94, 0x93, 0x4f, 0xe4, 0xec, 0x14, 0x7e, 0x09, 0x95, - 0xa1, 0x66, 0x65, 0x1d, 0x2f, 0x3c, 0x29, 0x8b, 0xd6, 0xf2, 0x2f, 0xcc, 0xd0, 0x8b, 0x35, 0x2b, - 0xcd, 0xd5, 0x6c, 0x73, 0x1f, 0x6a, 0xd3, 0x77, 0x17, 0x57, 0xa0, 0xae, 0x37, 0x47, 0x22, 0x09, - 0xd9, 0x98, 0x5a, 0xf8, 0x04, 0x56, 0x34, 0x30, 0xcb, 0x4f, 0x09, 0xfe, 0x0f, 0x56, 0x17, 0xc0, - 0xd3, 0x1e, 0xb5, 0x37, 0xff, 0xb2, 0xa1, 0x5e, 0x78, 0x96, 0x10, 0xa0, 0xdc, 0x4f, 0xfd, 0xfd, - 0x49, 0x4c, 0x2d, 0xac, 0x43, 0xa5, 0x9f, 0xfa, 0x3b, 0x9c, 0x49, 0x4a, 0xcc, 0xe6, 0x4d, 0x22, - 0x62, 0x6a, 0x1b, 0xd6, 0x76, 0x1c, 0xd3, 0x12, 0x36, 0x01, 0xb2, 0xf5, 0x80, 0xa7, 0x31, 0x75, - 0x0c, 0xf1, 0x54, 0x48, 0x4e, 0x97, 0x94, 0x36, 0xb3, 0xd1, 0xd1, 0xb2, 0x89, 0xaa, 0x27, 0x80, - 0x56, 0x90, 0x42, 0x43, 0x5d, 0xc6, 0x59, 0x22, 0xcf, 0xd5, 0x2d, 0x55, 0x5c, 0x03, 0x5a, 0x44, - 0xf4, 0xa1, 0x1a, 0x22, 0x34, 0xfb, 0xa9, 0xff, 0x36, 0x4a, 0x38, 0x1b, 0x5e, 0xb0, 0xf3, 0x31, - 0xa7, 0x80, 0xab, 0xb0, 0x6c, 0x12, 0xa9, 0x2f, 0x6e, 0x92, 0xd2, 0xba, 0xa1, 0xed, 0x5e, 0xf0, - 0xe1, 0x8f, 0xdf, 0x4e, 0x44, 0x32, 0x09, 0x69, 0x43, 0xd9, 0xee, 0xa7, 0xbe, 0x6e, 0xd0, 0x88, - 0x27, 0x87, 0x9c, 0x79, 0x3c, 0xa1, 0xcb, 0xe6, 0xf4, 0x49, 0x10, 0x72, 0x31, 0x91, 0x47, 0xe2, - 0x27, 0xda, 0x34, 0x62, 0x06, 0x9c, 0x79, 0xfa, 0x7f, 0x47, 0x57, 0x8c, 0x98, 0x29, 0xa2, 0xc5, - 0x50, 0xe3, 0xf7, 0x4d, 0xc2, 0xb5, 0xc5, 0x55, 0x73, 0xab, 0xd9, 0x6b, 0x0e, 0x6e, 0xfe, 0x4a, - 0x60, 0xed, 0xb1, 0xf1, 0xc0, 0x0d, 0x70, 0x1f, 0xc3, 0xb7, 0x27, 0x52, 0x50, 0x0b, 0x3f, 0x81, - 0xff, 0x3f, 0x16, 0xfd, 0x5a, 0x04, 0x91, 0x3c, 0x08, 0xe3, 0x71, 0x30, 0x0c, 0x54, 0x2b, 0x3e, - 0x46, 0x7b, 0x7d, 0x65, 0x68, 0xf6, 0xe6, 0x35, 0x34, 0xe7, 0x3f, 0x0a, 0x55, 0x8c, 0x19, 0xb2, - 0xed, 0x79, 0x6a, 0xfc, 0xa9, 0x85, 0x6e, 0x51, 0xec, 0x80, 0x87, 0xe2, 0x92, 0xeb, 0x08, 0x99, - 0x8f, 0xbc, 0x8d, 0x3d, 0x26, 0xb3, 0x88, 0x3d, 0x6f, 0x64, 0xdb, 0xf3, 0x0e, 0xb3, 0xb7, 0x47, - 0x47, 0x4b, 0x3b, 0x2f, 0x6e, 0x3e, 0xb4, 0xac, 0xdb, 0x0f, 0x2d, 0xeb, 0xe6, 0xae, 0x45, 0x6e, - 0xef, 0x5a, 0xe4, 0x9f, 0xbb, 0x16, 0xf9, 0xed, 0xbe, 0x65, 0xfd, 0x7e, 0xdf, 0xb2, 0x6e, 0xef, - 0x5b, 0xd6, 0xdf, 0xf7, 0x2d, 0xeb, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc9, 0x35, 0x94, 0xd2, - 0xbb, 0x08, 0x00, 0x00, + // 1091 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcb, 0x6e, 0x23, 0x45, + 0x14, 0xed, 0x6e, 0x77, 0xfc, 0xb8, 0x76, 0x9c, 0x4a, 0xc5, 0x13, 0x5a, 0x51, 0xe4, 0x31, 0x9e, + 0x41, 0x63, 0x05, 0x4d, 0x40, 0x46, 0x42, 0x88, 0x5d, 0x1e, 0x23, 0x25, 0x28, 0x0e, 0x83, 0x93, + 0xc9, 0x02, 0x09, 0x45, 0x15, 0x77, 0xa5, 0xd3, 0x60, 0x57, 0xb5, 0xaa, 0xcb, 0x21, 0xd9, 0x20, + 0xc4, 0x17, 0xb0, 0x64, 0xc3, 0x96, 0x3d, 0x7c, 0x45, 0x96, 0x59, 0xb2, 0x1a, 0x31, 0xc9, 0x1f, + 0xf0, 0x05, 0xa8, 0xaa, 0xab, 0x1f, 0x76, 0xa2, 0x59, 0xb0, 0xab, 0x3a, 0xf7, 0xd4, 0xbd, 0xe7, + 0x9e, 0xdb, 0x55, 0x0d, 0x20, 0xc8, 0xb9, 0xdc, 0x8c, 0x04, 0x97, 0x1c, 0x97, 0xd5, 0x3a, 0x3a, + 0x5b, 0x6b, 0x05, 0x3c, 0xe0, 0x1a, 0xfa, 0x44, 0xad, 0x92, 0x68, 0xf7, 0x27, 0x58, 0x78, 0xc5, + 0xa4, 0xb8, 0xc6, 0x1e, 0xb8, 0xc7, 0x54, 0x4c, 0x3c, 0xa7, 0x63, 0xf7, 0xdc, 0x6d, 0xf7, 0xe6, + 0xed, 0x53, 0x6b, 0xa8, 0x11, 0xbc, 0x06, 0x0b, 0xfb, 0xcc, 0xa7, 0x57, 0x5e, 0xa9, 0x10, 0x4a, + 0x20, 0xfc, 0x31, 0xb8, 0xc7, 0xd7, 0x11, 0xf5, 0xec, 0x8e, 0xdd, 0x6b, 0xf6, 0x97, 0x37, 0x93, + 0x5a, 0x9b, 0x3a, 0xa5, 0x0a, 0x64, 0x89, 0xae, 0x23, 0x8a, 0x31, 0xb8, 0xbb, 0x44, 0x12, 0xcf, + 0xed, 0xd8, 0xbd, 0xc6, 0x50, 0xaf, 0xbb, 0x3f, 0xdb, 0x80, 0x8e, 0x18, 0x89, 0xe2, 0x0b, 0x2e, + 0x07, 0x54, 0x12, 0x9f, 0x48, 0x82, 0x3f, 0x07, 0x18, 0x71, 0x76, 0x7e, 0x1a, 0x4b, 0x22, 0x93, + 0xdc, 0xf5, 0x3c, 0xf7, 0x0e, 0x67, 0xe7, 0x47, 0x2a, 0x60, 0x72, 0xd7, 0x46, 0x29, 0xa0, 0x94, + 0x86, 0x5a, 0x69, 0xb1, 0x89, 0x04, 0x52, 0xfd, 0x49, 0xd5, 0x5f, 0xb1, 0x09, 0x8d, 0x74, 0xbf, + 0x85, 0x6a, 0xaa, 0x40, 0x49, 0x54, 0x0a, 0x74, 0xcd, 0xc6, 0x50, 0xaf, 0xf1, 0x97, 0x50, 0x9d, + 0x18, 0x65, 0x3a, 0x71, 0xbd, 0xef, 0xa5, 0x5a, 0xe6, 0x95, 0x9b, 0xbc, 0x19, 0xbf, 0xfb, 0x6f, + 0x09, 0x2a, 0x03, 0x1a, 0xc7, 0x24, 0xa0, 0xf8, 0x25, 0xb8, 0x32, 0xf7, 0x6a, 0x25, 0xcd, 0x61, + 0xc2, 0x45, 0xb7, 0x14, 0x0d, 0xb7, 0xc0, 0x91, 0x7c, 0xa6, 0x13, 0x47, 0x72, 0xd5, 0xc6, 0xb9, + 0xe0, 0x73, 0x6d, 0x28, 0x24, 0x6b, 0xd0, 0x9d, 0x6f, 0x10, 0xb7, 0xa1, 0x32, 0xe6, 0x81, 0x9e, + 0xee, 0x42, 0x21, 0x98, 0x82, 0xb9, 0x6d, 0xe5, 0x87, 0xb6, 0xbd, 0x84, 0x0a, 0x65, 0x52, 0x84, + 0x34, 0xf6, 0x2a, 0x9d, 0x52, 0xaf, 0xde, 0x5f, 0x9c, 0x99, 0x71, 0x9a, 0xca, 0x70, 0xf0, 0x3a, + 0x94, 0x47, 0x7c, 0x32, 0x09, 0xa5, 0x57, 0x2d, 0xe4, 0x32, 0x98, 0x92, 0x78, 0xc9, 0x25, 0xf5, + 0x16, 0x8b, 0x12, 0x15, 0x82, 0xfb, 0x50, 0x8d, 0x8d, 0x97, 0x5e, 0x4d, 0x7b, 0x8c, 0xe6, 0x3d, + 0xd6, 0x7c, 0x7b, 0x98, 0xf1, 0x54, 0x2d, 0x41, 0xbf, 0xa7, 0x23, 0xe9, 0x41, 0xc7, 0xee, 0x55, + 0xd3, 0x5a, 0x09, 0x86, 0x9f, 0x03, 0x24, 0xab, 0xbd, 0x90, 0x49, 0xaf, 0x5e, 0xa8, 0x58, 0xc0, + 0x95, 0x35, 0x23, 0xce, 0x24, 0xbd, 0x92, 0x5e, 0x43, 0x8d, 0xdc, 0x14, 0x49, 0x41, 0xfc, 0x19, + 0xd4, 0x04, 0x8d, 0x23, 0xce, 0x62, 0x1a, 0x7b, 0x4d, 0x6d, 0xc0, 0xd2, 0xdc, 0xe0, 0xd2, 0xcf, + 0x30, 0xe3, 0x75, 0xbf, 0x83, 0xda, 0x1e, 0x11, 0x7e, 0xf2, 0x4d, 0xa6, 0x63, 0xb1, 0x1f, 0x8c, + 0x25, 0x75, 0xc3, 0x79, 0xe0, 0x46, 0xee, 0x62, 0xe9, 0xa1, 0x8b, 0xdd, 0xbf, 0x6c, 0xa8, 0x65, + 0x97, 0x00, 0xaf, 0x42, 0x59, 0x9d, 0x11, 0xb1, 0x67, 0x77, 0x4a, 0x3d, 0x77, 0x68, 0x76, 0x78, + 0x0d, 0xaa, 0x63, 0x4a, 0x04, 0x53, 0x11, 0x47, 0x47, 0xb2, 0x3d, 0x7e, 0x01, 0x4b, 0x09, 0xeb, + 0x94, 0x4f, 0x65, 0xc0, 0x43, 0x16, 0x78, 0x25, 0x4d, 0x69, 0x26, 0xf0, 0xd7, 0x06, 0xc5, 0xcf, + 0x60, 0x31, 0x3d, 0x74, 0xca, 0x94, 0x49, 0xae, 0xa6, 0x35, 0x52, 0xf0, 0x50, 0x79, 0xf4, 0x0c, + 0x80, 0x4c, 0x25, 0x3f, 0x1d, 0x53, 0x72, 0x49, 0xf5, 0x17, 0x96, 0xce, 0xa2, 0xa6, 0xf0, 0x03, + 0x05, 0x77, 0x7f, 0xb7, 0x01, 0x94, 0xe8, 0x9d, 0x0b, 0xc2, 0x02, 0x8a, 0x3f, 0x35, 0x77, 0xc1, + 0xd1, 0x77, 0x61, 0xb5, 0x78, 0xb7, 0x13, 0xc6, 0x83, 0xeb, 0xf0, 0x02, 0x2a, 0x8c, 0xfb, 0xf4, + 0x34, 0xf4, 0x8d, 0x29, 0x4d, 0x15, 0xbc, 0x7b, 0xfb, 0xb4, 0x7c, 0xc8, 0x7d, 0xba, 0xbf, 0x3b, + 0x2c, 0xab, 0xf0, 0xbe, 0x8f, 0xbd, 0x7c, 0xa4, 0xc9, 0x43, 0x93, 0x0d, 0x73, 0x0d, 0x9c, 0xd0, + 0x37, 0x83, 0x00, 0x73, 0xda, 0xd9, 0xdf, 0x1d, 0x3a, 0xa1, 0xdf, 0x9d, 0x00, 0xca, 0x8b, 0x1f, + 0x85, 0x2c, 0x18, 0xe7, 0x22, 0xed, 0xff, 0x23, 0xd2, 0x79, 0x9f, 0xc8, 0xee, 0x1f, 0x36, 0x34, + 0xf2, 0x3c, 0x27, 0x7d, 0xbc, 0x0d, 0x20, 0x05, 0x61, 0x71, 0x28, 0x43, 0xce, 0x4c, 0xc5, 0xf5, + 0x47, 0x2a, 0x66, 0x9c, 0xf4, 0x63, 0xce, 0x4f, 0xe1, 0x2f, 0xa0, 0x32, 0xd2, 0xac, 0x64, 0xe2, + 0x85, 0x77, 0x6a, 0xbe, 0xb5, 0xf4, 0xda, 0x1a, 0x7a, 0xd1, 0xb3, 0xd2, 0x8c, 0x67, 0x1b, 0x7b, + 0x50, 0xcb, 0x1e, 0x73, 0xbc, 0x04, 0x75, 0xbd, 0x39, 0xe4, 0x62, 0x42, 0xc6, 0xc8, 0xc2, 0x2b, + 0xb0, 0xa4, 0x81, 0x3c, 0x3f, 0xb2, 0xf1, 0x13, 0x58, 0x9e, 0x03, 0x4f, 0xfa, 0xc8, 0xd9, 0xf8, + 0xb3, 0x04, 0xf5, 0xc2, 0x5b, 0x87, 0x01, 0xca, 0x83, 0x38, 0xd8, 0x9b, 0x46, 0xc8, 0xc2, 0x75, + 0xa8, 0x0c, 0xe2, 0x60, 0x9b, 0x12, 0x89, 0x6c, 0xb3, 0x79, 0x2d, 0x78, 0x84, 0x1c, 0xc3, 0xda, + 0x8a, 0x22, 0x54, 0xc2, 0x4d, 0x80, 0x64, 0x3d, 0xa4, 0x71, 0x84, 0x5c, 0x43, 0x3c, 0xe1, 0x92, + 0xa2, 0x05, 0xa5, 0xcd, 0x6c, 0x74, 0xb4, 0x6c, 0xa2, 0xea, 0xf5, 0x40, 0x15, 0x8c, 0xa0, 0xa1, + 0x8a, 0x51, 0x22, 0xe4, 0x99, 0xaa, 0x52, 0xc5, 0x2d, 0x40, 0x45, 0x44, 0x1f, 0xaa, 0x61, 0x0c, + 0xcd, 0x41, 0x1c, 0xbc, 0x61, 0x82, 0x92, 0xd1, 0x05, 0x39, 0x1b, 0x53, 0x04, 0x78, 0x19, 0x16, + 0x4d, 0x22, 0x75, 0xe3, 0xa6, 0x31, 0xaa, 0x1b, 0xda, 0xce, 0x05, 0x1d, 0xfd, 0xf0, 0xcd, 0x94, + 0x8b, 0xe9, 0x04, 0x35, 0x54, 0xdb, 0x83, 0x38, 0xd0, 0x03, 0x3a, 0xa7, 0xe2, 0x80, 0x12, 0x9f, + 0x0a, 0xb4, 0x68, 0x4e, 0x1f, 0x87, 0x13, 0xca, 0xa7, 0xf2, 0x90, 0xff, 0x88, 0x9a, 0x46, 0xcc, + 0x90, 0x12, 0x5f, 0xff, 0x44, 0xd1, 0x92, 0x11, 0x93, 0x21, 0x5a, 0x0c, 0x32, 0xfd, 0xbe, 0x16, + 0x54, 0xb7, 0xb8, 0x6c, 0xaa, 0x9a, 0xbd, 0xe6, 0x60, 0x73, 0xf2, 0x48, 0x72, 0x41, 0x02, 0xba, + 0x15, 0x45, 0x94, 0xf9, 0x68, 0x05, 0x7b, 0xd0, 0x9a, 0x47, 0x35, 0xbf, 0xa5, 0x26, 0x36, 0x13, + 0x19, 0x5f, 0xa3, 0x27, 0xf8, 0x03, 0x58, 0x99, 0x03, 0x35, 0x7b, 0x75, 0xe3, 0x17, 0x1b, 0x5a, + 0x8f, 0x7d, 0x7c, 0x78, 0x1d, 0xbc, 0xc7, 0xf0, 0xad, 0xa9, 0xe4, 0xc8, 0xc2, 0x1f, 0xc1, 0x87, + 0x8f, 0x45, 0xbf, 0xe2, 0x21, 0x93, 0xfb, 0x93, 0x68, 0x1c, 0x8e, 0x42, 0x35, 0xe8, 0xf7, 0xd1, + 0x5e, 0x5d, 0x19, 0x9a, 0xb3, 0x71, 0x0d, 0xcd, 0xd9, 0x2b, 0xa7, 0xac, 0xce, 0x91, 0x2d, 0xdf, + 0x57, 0x97, 0x0b, 0x59, 0xaa, 0xeb, 0x1c, 0x1e, 0xd2, 0x09, 0xbf, 0xa4, 0x3a, 0x62, 0xcf, 0x46, + 0xde, 0x44, 0x3e, 0x91, 0x49, 0xc4, 0x99, 0x6d, 0x64, 0xcb, 0xf7, 0x0f, 0x92, 0x97, 0x4d, 0x47, + 0x4b, 0xdb, 0xcf, 0x6f, 0xde, 0xb5, 0xad, 0xdb, 0x77, 0x6d, 0xeb, 0xe6, 0xae, 0x6d, 0xdf, 0xde, + 0xb5, 0xed, 0x7f, 0xee, 0xda, 0xf6, 0xaf, 0xf7, 0x6d, 0xeb, 0xb7, 0xfb, 0xb6, 0x75, 0x7b, 0xdf, + 0xb6, 0xfe, 0xbe, 0x6f, 0x5b, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0xe4, 0x8e, 0x4c, 0xc0, 0x6e, + 0x09, 0x00, 0x00, } func (m *Entry) Marshal() (dAtA []byte, err error) { @@ -905,6 +936,23 @@ func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.Responses) > 0 { + for iNdEx := len(m.Responses) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Responses[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRaft(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x72 + } + } + i = encodeVarintRaft(dAtA, i, uint64(m.Vote)) + i-- + dAtA[i] = 0x68 if m.Context != nil { i -= len(m.Context) copy(dAtA[i:], m.Context) @@ -1263,6 +1311,13 @@ func (m *Message) Size() (n int) { l = len(m.Context) n += 1 + l + sovRaft(uint64(l)) } + n += 1 + sovRaft(uint64(m.Vote)) + if len(m.Responses) > 0 { + for _, e := range m.Responses { + l = e.Size() + n += 1 + l + sovRaft(uint64(l)) + } + } return n } @@ -2045,6 +2100,59 @@ func (m *Message) Unmarshal(dAtA []byte) error { m.Context = []byte{} } iNdEx = postIndex + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Vote", wireType) + } + m.Vote = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Vote |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Responses", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaft + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRaft + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRaft + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Responses = append(m.Responses, Message{}) + if err := m.Responses[len(m.Responses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRaft(dAtA[iNdEx:]) diff --git a/raftpb/raft.proto b/raftpb/raft.proto index b3b49e58cc25..a70f736a5992 100644 --- a/raftpb/raft.proto +++ b/raftpb/raft.proto @@ -13,14 +13,12 @@ option (gogoproto.goproto_unrecognized_all) = false; option (gogoproto.goproto_sizecache_all) = false; enum EntryType { - EntryNormal = 0; EntryConfChange = 1; // corresponds to pb.ConfChange EntryConfChangeV2 = 2; // corresponds to pb.ConfChangeV2 } message Entry { - optional uint64 Term = 2 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations optional uint64 Index = 3 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations optional EntryType Type = 1 [(gogoproto.nullable) = false]; @@ -41,25 +39,29 @@ message Snapshot { // For description of different message types, see: // https://pkg.go.dev/go.etcd.io/raft/v3#hdr-MessageType enum MessageType { - MsgHup = 0; - MsgBeat = 1; - MsgProp = 2; - MsgApp = 3; - MsgAppResp = 4; - MsgVote = 5; - MsgVoteResp = 6; - MsgSnap = 7; - MsgHeartbeat = 8; - MsgHeartbeatResp = 9; - MsgUnreachable = 10; - MsgSnapStatus = 11; - MsgCheckQuorum = 12; - MsgTransferLeader = 13; - MsgTimeoutNow = 14; - MsgReadIndex = 15; - MsgReadIndexResp = 16; - MsgPreVote = 17; - MsgPreVoteResp = 18; + MsgHup = 0; + MsgBeat = 1; + MsgProp = 2; + MsgApp = 3; + MsgAppResp = 4; + MsgVote = 5; + MsgVoteResp = 6; + MsgSnap = 7; + MsgHeartbeat = 8; + MsgHeartbeatResp = 9; + MsgUnreachable = 10; + MsgSnapStatus = 11; + MsgCheckQuorum = 12; + MsgTransferLeader = 13; + MsgTimeoutNow = 14; + MsgReadIndex = 15; + MsgReadIndexResp = 16; + MsgPreVote = 17; + MsgPreVoteResp = 18; + MsgStorageAppend = 19; + MsgStorageAppendResp = 20; + MsgStorageApply = 21; + MsgStorageApplyResp = 22; // NOTE: when adding new message types, remember to update the isLocalMsg and // isResponseMsg arrays in raft/util.go and update the corresponding tests in // raft/util_test.go. @@ -71,14 +73,25 @@ message Message { optional uint64 from = 3 [(gogoproto.nullable) = false]; optional uint64 term = 4 [(gogoproto.nullable) = false]; // logTerm is generally used for appending Raft logs to followers. For example, - // (type=MsgApp,index=100,logTerm=5) means leader appends entries starting at - // index=101, and the term of entry at index 100 is 5. + // (type=MsgApp,index=100,logTerm=5) means the leader appends entries starting + // at index=101, and the term of the entry at index 100 is 5. // (type=MsgAppResp,reject=true,index=100,logTerm=5) means follower rejects some // entries from its leader as it already has an entry with term 5 at index 100. + // (type=MsgStorageAppendResp,index=100,logTerm=5) means the local node wrote + // entries up to index=100 in stable storage, and the term of the entry at index + // 100 was 5. This doesn't always mean that the corresponding MsgStorageAppend + // message was the one that carried these entries, just that those entries were + // stable at the time of processing the corresponding MsgStorageAppend. optional uint64 logTerm = 5 [(gogoproto.nullable) = false]; optional uint64 index = 6 [(gogoproto.nullable) = false]; repeated Entry entries = 7 [(gogoproto.nullable) = false]; optional uint64 commit = 8 [(gogoproto.nullable) = false]; + // (type=MsgStorageAppend,vote=5,term=10) means the local node is voting for + // peer 5 in term 10. For MsgStorageAppends, the term, vote, and commit fields + // will either all be set (to facilitate the construction of a HardState) if + // any of the fields have changed or will all be unset if none of the fields + // have changed. + optional uint64 vote = 13 [(gogoproto.nullable) = false]; // snapshot is non-nil and non-empty for MsgSnap messages and nil for all other // message types. However, peer nodes running older binary versions may send a // non-nil, empty value for the snapshot field of non-MsgSnap messages. Code @@ -86,7 +99,11 @@ message Message { optional Snapshot snapshot = 9 [(gogoproto.nullable) = true]; optional bool reject = 10 [(gogoproto.nullable) = false]; optional uint64 rejectHint = 11 [(gogoproto.nullable) = false]; - optional bytes context = 12; + optional bytes context = 12 [(gogoproto.nullable) = true]; + // responses are populated by a raft node to instruct storage threads on how + // to respond and who to respond to when the work associated with a message + // is complete. Populated for MsgStorageAppend and MsgStorageApply messages. + repeated Message responses = 14 [(gogoproto.nullable) = false]; } message HardState { @@ -190,7 +207,6 @@ message ConfChangeSingle { // // [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf message ConfChangeV2 { - optional ConfChangeTransition transition = 1 [(gogoproto.nullable) = false]; repeated ConfChangeSingle changes = 2 [(gogoproto.nullable) = false]; optional bytes context = 3; diff --git a/raftpb/raft_test.go b/raftpb/raft_test.go index 7057446c223d..704ff94f4fc8 100644 --- a/raftpb/raft_test.go +++ b/raftpb/raft_test.go @@ -45,7 +45,7 @@ func TestProtoMemorySizes(t *testing.T) { assert(unsafe.Sizeof(s), if64Bit(144, 80), "Snapshot") var m Message - assert(unsafe.Sizeof(m), if64Bit(128, 92), "Message") + assert(unsafe.Sizeof(m), if64Bit(160, 112), "Message") var hs HardState assert(unsafe.Sizeof(hs), 24, "HardState") diff --git a/rafttest/interaction_env.go b/rafttest/interaction_env.go index 3c245f07fab4..a7dfc0cf6caf 100644 --- a/rafttest/interaction_env.go +++ b/rafttest/interaction_env.go @@ -34,8 +34,10 @@ type Node struct { *raft.RawNode Storage - Config *raft.Config - History []pb.Snapshot + Config *raft.Config + AppendWork []pb.Message // []MsgStorageAppend + ApplyWork []pb.Message // []MsgStorageApply + History []pb.Snapshot } // InteractionEnv facilitates testing of complex interactions between the diff --git a/rafttest/interaction_env_handler.go b/rafttest/interaction_env_handler.go index 73e706feff76..9f95bc123afe 100644 --- a/rafttest/interaction_env_handler.go +++ b/rafttest/interaction_env_handler.go @@ -43,7 +43,7 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string { case "add-nodes": // Example: // - // add-nodes voters=(1 2 3) learners=(4 5) index=2 content=foo + // add-nodes voters=(1 2 3) learners=(4 5) index=2 content=foo async-storage-writes=true err = env.handleAddNodes(t, d) case "campaign": // Example: @@ -67,6 +67,16 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string { // // process-ready 3 err = env.handleProcessReady(t, d) + case "process-append-thread": + // Example: + // + // process-append-thread 3 + err = env.handleProcessAppendThread(t, d) + case "process-apply-thread": + // Example: + // + // process-apply-thread 3 + err = env.handleProcessApplyThread(t, d) case "log-level": // Set the log level. NONE disables all output, including from the test // harness (except errors). diff --git a/rafttest/interaction_env_handler_add_nodes.go b/rafttest/interaction_env_handler_add_nodes.go index f164a6d29e72..f086aed6e7c7 100644 --- a/rafttest/interaction_env_handler_add_nodes.go +++ b/rafttest/interaction_env_handler_add_nodes.go @@ -48,6 +48,8 @@ func (env *InteractionEnv) handleAddNodes(t *testing.T, d datadriven.TestData) e cfg.Applied = snap.Metadata.Index case "content": arg.Scan(t, i, &snap.Data) + case "async-storage-writes": + arg.Scan(t, i, &cfg.AsyncStorageWrites) } } } diff --git a/rafttest/interaction_env_handler_deliver_msgs.go b/rafttest/interaction_env_handler_deliver_msgs.go index 8a4ff10d33aa..671e053948ea 100644 --- a/rafttest/interaction_env_handler_deliver_msgs.go +++ b/rafttest/interaction_env_handler_deliver_msgs.go @@ -73,7 +73,7 @@ func (env *InteractionEnv) DeliverMsgs(rs ...Recipient) int { var n int for _, r := range rs { var msgs []raftpb.Message - msgs, env.Messages = splitMsgs(env.Messages, r.ID) + msgs, env.Messages = splitMsgs(env.Messages, r.ID, r.Drop) n += len(msgs) for _, msg := range msgs { if r.Drop { diff --git a/rafttest/interaction_env_handler_process_append_thread.go b/rafttest/interaction_env_handler_process_append_thread.go new file mode 100644 index 000000000000..38c16f255873 --- /dev/null +++ b/rafttest/interaction_env_handler_process_append_thread.go @@ -0,0 +1,98 @@ +// Copyright 2022 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rafttest + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/datadriven" + + "go.etcd.io/raft/v3" + "go.etcd.io/raft/v3/raftpb" +) + +func (env *InteractionEnv) handleProcessAppendThread(t *testing.T, d datadriven.TestData) error { + idxs := nodeIdxs(t, d) + for _, idx := range idxs { + var err error + if len(idxs) > 1 { + fmt.Fprintf(env.Output, "> %d processing append thread\n", idx+1) + env.withIndent(func() { err = env.ProcessAppendThread(idx) }) + } else { + err = env.ProcessAppendThread(idx) + } + if err != nil { + return err + } + } + return nil +} + +// ProcessAppendThread runs processes a single message on the "append" thread of +// the node with the given index. +func (env *InteractionEnv) ProcessAppendThread(idx int) error { + n := &env.Nodes[idx] + if len(n.AppendWork) == 0 { + env.Output.WriteString("no append work to perform") + return nil + } + m := n.AppendWork[0] + n.AppendWork = n.AppendWork[1:] + + resps := m.Responses + m.Responses = nil + env.Output.WriteString("Processing:\n") + env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") + st := raftpb.HardState{ + Term: m.Term, + Vote: m.Vote, + Commit: m.Commit, + } + var snap raftpb.Snapshot + if m.Snapshot != nil { + snap = *m.Snapshot + } + if err := processAppend(n, st, m.Entries, snap); err != nil { + return err + } + + env.Output.WriteString("Responses:\n") + for _, m := range resps { + env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") + } + env.Messages = append(env.Messages, resps...) + return nil +} + +func processAppend(n *Node, st raftpb.HardState, ents []raftpb.Entry, snap raftpb.Snapshot) error { + // TODO(tbg): the order of operations here is not necessarily safe. See: + // https://github.com/etcd-io/etcd/pull/10861 + s := n.Storage + if !raft.IsEmptyHardState(st) { + if err := s.SetHardState(st); err != nil { + return err + } + } + if err := s.Append(ents); err != nil { + return err + } + if !raft.IsEmptySnap(snap) { + if err := s.ApplySnapshot(snap); err != nil { + return err + } + } + return nil +} diff --git a/rafttest/interaction_env_handler_process_apply_thread.go b/rafttest/interaction_env_handler_process_apply_thread.go new file mode 100644 index 000000000000..d21317e08487 --- /dev/null +++ b/rafttest/interaction_env_handler_process_apply_thread.go @@ -0,0 +1,111 @@ +// Copyright 2022 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rafttest + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/datadriven" + + "go.etcd.io/raft/v3" + "go.etcd.io/raft/v3/raftpb" +) + +func (env *InteractionEnv) handleProcessApplyThread(t *testing.T, d datadriven.TestData) error { + idxs := nodeIdxs(t, d) + for _, idx := range idxs { + var err error + if len(idxs) > 1 { + fmt.Fprintf(env.Output, "> %d processing apply thread\n", idx+1) + env.withIndent(func() { err = env.ProcessApplyThread(idx) }) + } else { + err = env.ProcessApplyThread(idx) + } + if err != nil { + return err + } + } + return nil +} + +// ProcessApplyThread runs processes a single message on the "apply" thread of +// the node with the given index. +func (env *InteractionEnv) ProcessApplyThread(idx int) error { + n := &env.Nodes[idx] + if len(n.ApplyWork) == 0 { + env.Output.WriteString("no apply work to perform") + return nil + } + m := n.ApplyWork[0] + n.ApplyWork = n.ApplyWork[1:] + + resps := m.Responses + m.Responses = nil + env.Output.WriteString("Processing:\n") + env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") + if err := processApply(n, m.Entries); err != nil { + return err + } + + env.Output.WriteString("Responses:\n") + for _, m := range resps { + env.Output.WriteString(raft.DescribeMessage(m, defaultEntryFormatter) + "\n") + } + env.Messages = append(env.Messages, resps...) + return nil +} + +func processApply(n *Node, ents []raftpb.Entry) error { + for _, ent := range ents { + var update []byte + var cs *raftpb.ConfState + switch ent.Type { + case raftpb.EntryConfChange: + var cc raftpb.ConfChange + if err := cc.Unmarshal(ent.Data); err != nil { + return err + } + update = cc.Context + cs = n.RawNode.ApplyConfChange(cc) + case raftpb.EntryConfChangeV2: + var cc raftpb.ConfChangeV2 + if err := cc.Unmarshal(ent.Data); err != nil { + return err + } + cs = n.RawNode.ApplyConfChange(cc) + update = cc.Context + default: + update = ent.Data + } + + // Record the new state by starting with the current state and applying + // the command. + lastSnap := n.History[len(n.History)-1] + var snap raftpb.Snapshot + snap.Data = append(snap.Data, lastSnap.Data...) + // NB: this hard-codes an "appender" state machine. + snap.Data = append(snap.Data, update...) + snap.Metadata.Index = ent.Index + snap.Metadata.Term = ent.Term + if cs == nil { + sl := n.History + cs = &sl[len(sl)-1].Metadata.ConfState + } + snap.Metadata.ConfState = *cs + n.History = append(n.History, snap) + } + return nil +} diff --git a/rafttest/interaction_env_handler_process_ready.go b/rafttest/interaction_env_handler_process_ready.go index 6b2ef18e8e76..e72d3d9dce3a 100644 --- a/rafttest/interaction_env_handler_process_ready.go +++ b/rafttest/interaction_env_handler_process_ready.go @@ -44,65 +44,39 @@ func (env *InteractionEnv) handleProcessReady(t *testing.T, d datadriven.TestDat // ProcessReady runs Ready handling on the node with the given index. func (env *InteractionEnv) ProcessReady(idx int) error { // TODO(tbg): Allow simulating crashes here. - rn, s := env.Nodes[idx].RawNode, env.Nodes[idx].Storage - rd := rn.Ready() + n := &env.Nodes[idx] + rd := n.Ready() env.Output.WriteString(raft.DescribeReady(rd, defaultEntryFormatter)) - // TODO(tbg): the order of operations here is not necessarily safe. See: - // https://github.com/etcd-io/etcd/pull/10861 - if !raft.IsEmptyHardState(rd.HardState) { - if err := s.SetHardState(rd.HardState); err != nil { + + if !n.Config.AsyncStorageWrites { + if err := processAppend(n, rd.HardState, rd.Entries, rd.Snapshot); err != nil { return err } - } - if err := s.Append(rd.Entries); err != nil { - return err - } - if !raft.IsEmptySnap(rd.Snapshot) { - if err := s.ApplySnapshot(rd.Snapshot); err != nil { + if err := processApply(n, rd.CommittedEntries); err != nil { return err } } - for _, ent := range rd.CommittedEntries { - var update []byte - var cs *raftpb.ConfState - switch ent.Type { - case raftpb.EntryConfChange: - var cc raftpb.ConfChange - if err := cc.Unmarshal(ent.Data); err != nil { - return err + + for _, m := range rd.Messages { + if raft.IsLocalMsgTarget(m.To) { + if !n.Config.AsyncStorageWrites { + panic("unexpected local msg target") } - update = cc.Context - cs = rn.ApplyConfChange(cc) - case raftpb.EntryConfChangeV2: - var cc raftpb.ConfChangeV2 - if err := cc.Unmarshal(ent.Data); err != nil { - return err + switch m.Type { + case raftpb.MsgStorageAppend: + n.AppendWork = append(n.AppendWork, m) + case raftpb.MsgStorageApply: + n.ApplyWork = append(n.ApplyWork, m) + default: + panic(fmt.Sprintf("unexpected message type %s", m.Type)) } - cs = rn.ApplyConfChange(cc) - update = cc.Context - default: - update = ent.Data - } - - // Record the new state by starting with the current state and applying - // the command. - lastSnap := env.Nodes[idx].History[len(env.Nodes[idx].History)-1] - var snap raftpb.Snapshot - snap.Data = append(snap.Data, lastSnap.Data...) - // NB: this hard-codes an "appender" state machine. - snap.Data = append(snap.Data, update...) - snap.Metadata.Index = ent.Index - snap.Metadata.Term = ent.Term - if cs == nil { - sl := env.Nodes[idx].History - cs = &sl[len(sl)-1].Metadata.ConfState + } else { + env.Messages = append(env.Messages, m) } - snap.Metadata.ConfState = *cs - env.Nodes[idx].History = append(env.Nodes[idx].History, snap) } - env.Messages = append(env.Messages, rd.Messages...) - - rn.Advance(rd) + if !n.Config.AsyncStorageWrites { + n.Advance(rd) + } return nil } diff --git a/rafttest/interaction_env_handler_stabilize.go b/rafttest/interaction_env_handler_stabilize.go index c4579be3d1b0..d79a9aa4fb28 100644 --- a/rafttest/interaction_env_handler_stabilize.go +++ b/rafttest/interaction_env_handler_stabilize.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/datadriven" + "go.etcd.io/raft/v3" "go.etcd.io/raft/v3/raftpb" ) @@ -31,44 +32,75 @@ func (env *InteractionEnv) handleStabilize(t *testing.T, d datadriven.TestData) // Stabilize repeatedly runs Ready handling on and message delivery to the set // of nodes specified via the idxs slice until reaching a fixed point. func (env *InteractionEnv) Stabilize(idxs ...int) error { - var nodes []Node - for _, idx := range idxs { - nodes = append(nodes, env.Nodes[idx]) - } - if len(nodes) == 0 { - nodes = env.Nodes + var nodes []*Node + if len(idxs) != 0 { + for _, idx := range idxs { + nodes = append(nodes, &env.Nodes[idx]) + } + } else { + for i := range env.Nodes { + nodes = append(nodes, &env.Nodes[i]) + } } for { done := true for _, rn := range nodes { if rn.HasReady() { - done = false idx := int(rn.Status().ID - 1) fmt.Fprintf(env.Output, "> %d handling Ready\n", idx+1) - env.withIndent(func() { env.ProcessReady(idx) }) + var err error + env.withIndent(func() { err = env.ProcessReady(idx) }) + if err != nil { + return err + } + done = false } } for _, rn := range nodes { id := rn.Status().ID // NB: we grab the messages just to see whether to print the header. // DeliverMsgs will do it again. - if msgs, _ := splitMsgs(env.Messages, id); len(msgs) > 0 { + if msgs, _ := splitMsgs(env.Messages, id, false /* drop */); len(msgs) > 0 { fmt.Fprintf(env.Output, "> %d receiving messages\n", id) env.withIndent(func() { env.DeliverMsgs(Recipient{ID: id}) }) done = false } } + for _, rn := range nodes { + idx := int(rn.Status().ID - 1) + if len(rn.AppendWork) > 0 { + fmt.Fprintf(env.Output, "> %d processing append thread\n", idx+1) + for len(rn.AppendWork) > 0 { + var err error + env.withIndent(func() { err = env.ProcessAppendThread(idx) }) + if err != nil { + return err + } + } + done = false + } + } + for _, rn := range nodes { + idx := int(rn.Status().ID - 1) + if len(rn.ApplyWork) > 0 { + fmt.Fprintf(env.Output, "> %d processing apply thread\n", idx+1) + for len(rn.ApplyWork) > 0 { + env.withIndent(func() { env.ProcessApplyThread(idx) }) + } + done = false + } + } if done { return nil } } } -func splitMsgs(msgs []raftpb.Message, to uint64) (toMsgs []raftpb.Message, rmdr []raftpb.Message) { +func splitMsgs(msgs []raftpb.Message, to uint64, drop bool) (toMsgs []raftpb.Message, rmdr []raftpb.Message) { // NB: this method does not reorder messages. for _, msg := range msgs { - if msg.To == to { + if msg.To == to && !(drop && isLocalMsg(msg)) { toMsgs = append(toMsgs, msg) } else { rmdr = append(rmdr, msg) @@ -76,3 +108,8 @@ func splitMsgs(msgs []raftpb.Message, to uint64) (toMsgs []raftpb.Message, rmdr } return toMsgs, rmdr } + +// Don't drop local messages, which require reliable delivery. +func isLocalMsg(msg raftpb.Message) bool { + return msg.From == msg.To || raft.IsLocalMsgTarget(msg.From) || raft.IsLocalMsgTarget(msg.To) +} diff --git a/rawnode.go b/rawnode.go index 66f345f20136..b46182868f8e 100644 --- a/rawnode.go +++ b/rawnode.go @@ -32,9 +32,13 @@ var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found") // The methods of this struct correspond to the methods of Node and are described // more fully there. type RawNode struct { - raft *raft - prevSoftSt *SoftState - prevHardSt pb.HardState + raft *raft + asyncStorageWrites bool + + // Mutable fields. + prevSoftSt *SoftState + prevHardSt pb.HardState + stepsOnAdvance []pb.Message } // NewRawNode instantiates a RawNode from the given configuration. @@ -49,6 +53,7 @@ func NewRawNode(config *Config) (*RawNode, error) { rn := &RawNode{ raft: r, } + rn.asyncStorageWrites = config.AsyncStorageWrites rn.prevSoftSt = r.softState() rn.prevHardSt = r.hardState() return rn, nil @@ -108,14 +113,14 @@ func (rn *RawNode) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState { // Step advances the state machine using the given message. func (rn *RawNode) Step(m pb.Message) error { - // ignore unexpected local messages receiving over network - if IsLocalMsg(m.Type) { + // Ignore unexpected local messages receiving over network. + if IsLocalMsg(m.Type) && !IsLocalMsgTarget(m.From) { return ErrStepLocalMsg } - if pr := rn.raft.prs.Progress[m.From]; pr != nil || !IsResponseMsg(m.Type) { - return rn.raft.Step(m) + if IsResponseMsg(m.Type) && !IsLocalMsgTarget(m.From) && rn.raft.prs.Progress[m.From] == nil { + return ErrStepPeerNotFound } - return ErrStepPeerNotFound + return rn.raft.Step(m) } // Ready returns the outstanding work that the application needs to handle. This @@ -131,7 +136,261 @@ func (rn *RawNode) Ready() Ready { // readyWithoutAccept returns a Ready. This is a read-only operation, i.e. there // is no obligation that the Ready must be handled. func (rn *RawNode) readyWithoutAccept() Ready { - return newReady(rn.raft, rn.prevSoftSt, rn.prevHardSt) + r := rn.raft + + rd := Ready{ + Entries: r.raftLog.nextUnstableEnts(), + CommittedEntries: r.raftLog.nextCommittedEnts(rn.applyUnstableEntries()), + Messages: r.msgs, + } + if softSt := r.softState(); !softSt.equal(rn.prevSoftSt) { + rd.SoftState = softSt + } + if hardSt := r.hardState(); !isHardStateEqual(hardSt, rn.prevHardSt) { + rd.HardState = hardSt + } + if r.raftLog.hasNextUnstableSnapshot() { + rd.Snapshot = *r.raftLog.nextUnstableSnapshot() + } + if len(r.readStates) != 0 { + rd.ReadStates = r.readStates + } + rd.MustSync = MustSync(r.hardState(), rn.prevHardSt, len(rd.Entries)) + + if rn.asyncStorageWrites { + // If async storage writes are enabled, enqueue messages to + // local storage threads, where applicable. + if needStorageAppendMsg(r, rd) { + m := newStorageAppendMsg(r, rd) + rd.Messages = append(rd.Messages, m) + } + if needStorageApplyMsg(rd) { + m := newStorageApplyMsg(r, rd) + rd.Messages = append(rd.Messages, m) + } + } else { + // If async storage writes are disabled, immediately enqueue + // msgsAfterAppend to be sent out. The Ready struct contract + // mandates that Messages cannot be sent until after Entries + // are written to stable storage. + for _, m := range r.msgsAfterAppend { + if m.To != r.id { + rd.Messages = append(rd.Messages, m) + } + } + } + + return rd +} + +// MustSync returns true if the hard state and count of Raft entries indicate +// that a synchronous write to persistent storage is required. +func MustSync(st, prevst pb.HardState, entsnum int) bool { + // Persistent state on all servers: + // (Updated on stable storage before responding to RPCs) + // currentTerm + // votedFor + // log entries[] + return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term +} + +func needStorageAppendMsg(r *raft, rd Ready) bool { + // Return true if log entries, hard state, or a snapshot need to be written + // to stable storage. Also return true if any messages are contingent on all + // prior MsgStorageAppend being processed. + return len(rd.Entries) > 0 || + !IsEmptyHardState(rd.HardState) || + !IsEmptySnap(rd.Snapshot) || + len(r.msgsAfterAppend) > 0 +} + +func needStorageAppendRespMsg(r *raft, rd Ready) bool { + // Return true if raft needs to hear about stabilized entries or an applied + // snapshot. See the comment in newStorageAppendRespMsg, which explains why + // we check hasNextOrInProgressUnstableEnts instead of len(rd.Entries) > 0. + return r.raftLog.hasNextOrInProgressUnstableEnts() || + !IsEmptySnap(rd.Snapshot) +} + +// newStorageAppendMsg creates the message that should be sent to the local +// append thread to instruct it to append log entries, write an updated hard +// state, and apply a snapshot. The message also carries a set of responses +// that should be delivered after the rest of the message is processed. Used +// with AsyncStorageWrites. +func newStorageAppendMsg(r *raft, rd Ready) pb.Message { + m := pb.Message{ + Type: pb.MsgStorageAppend, + To: LocalAppendThread, + From: r.id, + Entries: rd.Entries, + } + if !IsEmptyHardState(rd.HardState) { + // If the Ready includes a HardState update, assign each of its fields + // to the corresponding fields in the Message. This allows clients to + // reconstruct the HardState and save it to stable storage. + // + // If the Ready does not include a HardState update, make sure to not + // assign a value to any of the fields so that a HardState reconstructed + // from them will be empty (return true from raft.IsEmptyHardState). + m.Term = rd.Term + m.Vote = rd.Vote + m.Commit = rd.Commit + } + if !IsEmptySnap(rd.Snapshot) { + snap := rd.Snapshot + m.Snapshot = &snap + } + // Attach all messages in msgsAfterAppend as responses to be delivered after + // the message is processed, along with a self-directed MsgStorageAppendResp + // to acknowledge the entry stability. + // + // NB: it is important for performance that MsgStorageAppendResp message be + // handled after self-directed MsgAppResp messages on the leader (which will + // be contained in msgsAfterAppend). This ordering allows the MsgAppResp + // handling to use a fast-path in r.raftLog.term() before the newly appended + // entries are removed from the unstable log. + m.Responses = r.msgsAfterAppend + if needStorageAppendRespMsg(r, rd) { + m.Responses = append(m.Responses, newStorageAppendRespMsg(r, rd)) + } + return m +} + +// newStorageAppendRespMsg creates the message that should be returned to node +// after the unstable log entries, hard state, and snapshot in the current Ready +// (along with those in all prior Ready structs) have been saved to stable +// storage. +func newStorageAppendRespMsg(r *raft, rd Ready) pb.Message { + m := pb.Message{ + Type: pb.MsgStorageAppendResp, + To: r.id, + From: LocalAppendThread, + // Dropped after term change, see below. + Term: r.Term, + } + if r.raftLog.hasNextOrInProgressUnstableEnts() { + // If the raft log has unstable entries, attach the last index and term of the + // append to the response message. This (index, term) tuple will be handed back + // and consulted when the stability of those log entries is signaled to the + // unstable. If the (index, term) match the unstable log by the time the + // response is received (unstable.stableTo), the unstable log can be truncated. + // + // However, with just this logic, there would be an ABA problem[^1] that could + // lead to the unstable log and the stable log getting out of sync temporarily + // and leading to an inconsistent view. Consider the following example with 5 + // nodes, A B C D E: + // + // 1. A is the leader. + // 2. A proposes some log entries but only B receives these entries. + // 3. B gets the Ready and the entries are appended asynchronously. + // 4. A crashes and C becomes leader after getting a vote from D and E. + // 5. C proposes some log entries and B receives these entries, overwriting the + // previous unstable log entries that are in the process of being appended. + // The entries have a larger term than the previous entries but the same + // indexes. It begins appending these new entries asynchronously. + // 6. C crashes and A restarts and becomes leader again after getting the vote + // from D and E. + // 7. B receives the entries from A which are the same as the ones from step 2, + // overwriting the previous unstable log entries that are in the process of + // being appended from step 5. The entries have the original terms and + // indexes from step 2. Recall that log entries retain their original term + // numbers when a leader replicates entries from previous terms. It begins + // appending these new entries asynchronously. + // 8. The asynchronous log appends from the first Ready complete and stableTo + // is called. + // 9. However, the log entries from the second Ready are still in the + // asynchronous append pipeline and will overwrite (in stable storage) the + // entries from the first Ready at some future point. We can't truncate the + // unstable log yet or a future read from Storage might see the entries from + // step 5 before they have been replaced by the entries from step 7. + // Instead, we must wait until we are sure that the entries are stable and + // that no in-progress appends might overwrite them before removing entries + // from the unstable log. + // + // To prevent these kinds of problems, we also attach the current term to the + // MsgStorageAppendResp (above). If the term has changed by the time the + // MsgStorageAppendResp if returned, the response is ignored and the unstable + // log is not truncated. The unstable log is only truncated when the term has + // remained unchanged from the time that the MsgStorageAppend was sent to the + // time that the MsgStorageAppendResp is received, indicating that no-one else + // is in the process of truncating the stable log. + // + // However, this replaces a correctness problem with a liveness problem. If we + // only attempted to truncate the unstable log when appending new entries but + // also occasionally dropped these responses, then quiescence of new log entries + // could lead to the unstable log never being truncated. + // + // To combat this, we attempt to truncate the log on all MsgStorageAppendResp + // messages where the unstable log is not empty, not just those associated with + // entry appends. This includes MsgStorageAppendResp messages associated with an + // updated HardState, which occur after a term change. + // + // In other words, we set Index and LogTerm in a block that looks like: + // + // if r.raftLog.hasNextOrInProgressUnstableEnts() { ... } + // + // not like: + // + // if len(rd.Entries) > 0 { ... } + // + // To do so, we attach r.raftLog.lastIndex() and r.raftLog.lastTerm(), not the + // (index, term) of the last entry in rd.Entries. If rd.Entries is not empty, + // these will be the same. However, if rd.Entries is empty, we still want to + // attest that this (index, term) is correct at the current term, in case the + // MsgStorageAppend that contained the last entry in the unstable slice carried + // an earlier term and was dropped. + // + // A MsgStorageAppend with a new term is emitted on each term change. This is + // the same condition that causes MsgStorageAppendResp messages with earlier + // terms to be ignored. As a result, we are guaranteed that, assuming a bounded + // number of term changes, there will eventually be a MsgStorageAppendResp + // message that is not ignored. This means that entries in the unstable log + // which have been appended to stable storage will eventually be truncated and + // dropped from memory. + // + // [^1]: https://en.wikipedia.org/wiki/ABA_problem + m.Index = r.raftLog.lastIndex() + m.LogTerm = r.raftLog.lastTerm() + } + if !IsEmptySnap(rd.Snapshot) { + snap := rd.Snapshot + m.Snapshot = &snap + } + return m +} + +func needStorageApplyMsg(rd Ready) bool { return len(rd.CommittedEntries) > 0 } +func needStorageApplyRespMsg(rd Ready) bool { return needStorageApplyMsg(rd) } + +// newStorageApplyMsg creates the message that should be sent to the local +// apply thread to instruct it to apply committed log entries. The message +// also carries a response that should be delivered after the rest of the +// message is processed. Used with AsyncStorageWrites. +func newStorageApplyMsg(r *raft, rd Ready) pb.Message { + ents := rd.CommittedEntries + return pb.Message{ + Type: pb.MsgStorageApply, + To: LocalApplyThread, + From: r.id, + Term: 0, // committed entries don't apply under a specific term + Entries: ents, + Responses: []pb.Message{ + newStorageApplyRespMsg(r, ents), + }, + } +} + +// newStorageApplyRespMsg creates the message that should be returned to node +// after the committed entries in the current Ready (along with those in all +// prior Ready structs) have been applied to the local state machine. +func newStorageApplyRespMsg(r *raft, ents []pb.Entry) pb.Message { + return pb.Message{ + Type: pb.MsgStorageApplyResp, + To: r.id, + From: LocalApplyThread, + Term: 0, // committed entries don't apply under a specific term + Entries: ents, + } } // acceptReady is called when the consumer of the RawNode has decided to go @@ -147,11 +406,44 @@ func (rn *RawNode) acceptReady(rd Ready) { if len(rd.ReadStates) != 0 { rn.raft.readStates = nil } + if !rn.asyncStorageWrites { + if len(rn.stepsOnAdvance) != 0 { + rn.raft.logger.Panicf("two accepted Ready structs without call to Advance") + } + for _, m := range rn.raft.msgsAfterAppend { + if m.To == rn.raft.id { + rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) + } + } + if needStorageAppendRespMsg(rn.raft, rd) { + m := newStorageAppendRespMsg(rn.raft, rd) + rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) + } + if needStorageApplyRespMsg(rd) { + m := newStorageApplyRespMsg(rn.raft, rd.CommittedEntries) + rn.stepsOnAdvance = append(rn.stepsOnAdvance, m) + } + } rn.raft.msgs = nil + rn.raft.msgsAfterAppend = nil + rn.raft.raftLog.acceptUnstable() + if len(rd.CommittedEntries) > 0 { + ents := rd.CommittedEntries + index := ents[len(ents)-1].Index + rn.raft.raftLog.acceptApplying(index, entsSize(ents), rn.applyUnstableEntries()) + } +} + +// applyUnstableEntries returns whether entries are allowed to be applied once +// they are known to be committed but before they have been written locally to +// stable storage. +func (rn *RawNode) applyUnstableEntries() bool { + return !rn.asyncStorageWrites } // HasReady called when RawNode user need to check if any Ready pending. func (rn *RawNode) HasReady() bool { + // TODO(nvanbenschoten): order these cases in terms of cost and frequency. r := rn.raft if !r.softState().equal(rn.prevSoftSt) { return true @@ -159,10 +451,13 @@ func (rn *RawNode) HasReady() bool { if hardSt := r.hardState(); !IsEmptyHardState(hardSt) && !isHardStateEqual(hardSt, rn.prevHardSt) { return true } - if r.raftLog.hasPendingSnapshot() { + if r.raftLog.hasNextUnstableSnapshot() { + return true + } + if len(r.msgs) > 0 || len(r.msgsAfterAppend) > 0 { return true } - if len(r.msgs) > 0 || len(r.raftLog.unstableEntries()) > 0 || r.raftLog.hasNextCommittedEnts() { + if r.raftLog.hasNextUnstableEnts() || r.raftLog.hasNextCommittedEnts(rn.applyUnstableEntries()) { return true } if len(r.readStates) != 0 { @@ -173,8 +468,21 @@ func (rn *RawNode) HasReady() bool { // Advance notifies the RawNode that the application has applied and saved progress in the // last Ready results. -func (rn *RawNode) Advance(rd Ready) { - rn.raft.advance(rd) +// +// NOTE: Advance must not be called when using AsyncStorageWrites. Response messages from +// the local append and apply threads take its place. +func (rn *RawNode) Advance(_ Ready) { + // The actions performed by this function are encoded into stepsOnAdvance in + // acceptReady. In earlier versions of this library, they were computed from + // the provided Ready struct. Retain the unused parameter for compatibility. + if rn.asyncStorageWrites { + rn.raft.logger.Panicf("Advance must not be called when using AsyncStorageWrites") + } + for i, m := range rn.stepsOnAdvance { + _ = rn.raft.Step(m) + rn.stepsOnAdvance[i] = pb.Message{} + } + rn.stepsOnAdvance = rn.stepsOnAdvance[:0] } // Status returns the current status of the given group. This allocates, see diff --git a/rawnode_test.go b/rawnode_test.go index 49f04c1310c5..3e146f8b457c 100644 --- a/rawnode_test.go +++ b/rawnode_test.go @@ -342,6 +342,7 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { if len(rd.Entries) > 0 { t.Fatal("expected no more entries") } + rawNode.Advance(rd) if tc.exp2 == nil { return } @@ -370,6 +371,7 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { if exp := tc.exp2; !reflect.DeepEqual(exp, cs) { t.Fatalf("exp:\n%+v\nact:\n%+v", exp, cs) } + rawNode.Advance(rd) }) } } @@ -490,6 +492,10 @@ func TestRawNodeJointAutoLeave(t *testing.T) { rd = rawNode.Ready() t.Log(DescribeReady(rd, nil)) s.Append(rd.Entries) + rawNode.Advance(rd) + rd = rawNode.Ready() + t.Log(DescribeReady(rd, nil)) + s.Append(rd.Entries) // Check that the right ConfChange comes out. if len(rd.Entries) != 1 || rd.Entries[0].Type != pb.EntryConfChangeV2 { t.Fatalf("expected exactly one more entry, got %+v", rd) @@ -743,11 +749,14 @@ func TestRawNodeStart(t *testing.T) { t.Fatalf("unexpected ready: %+v", rawNode.Ready()) } rawNode.Campaign() + rd := rawNode.Ready() + storage.Append(rd.Entries) + rawNode.Advance(rd) rawNode.Propose([]byte("foo")) if !rawNode.HasReady() { t.Fatal("expected a Ready") } - rd := rawNode.Ready() + rd = rawNode.Ready() if !reflect.DeepEqual(entries, rd.Entries) { t.Fatalf("expected to see entries\n%s, not\n%s", DescribeEntries(entries, nil), DescribeEntries(rd.Entries, nil)) } @@ -861,6 +870,9 @@ func TestRawNodeStatus(t *testing.T) { if err := rn.Campaign(); err != nil { t.Fatal(err) } + rd := rn.Ready() + s.Append(rd.Entries) + rn.Advance(rd) status := rn.Status() if status.Lead != 1 { t.Fatal("not lead") @@ -967,12 +979,12 @@ func TestRawNodeBoundedLogGrowthWithPartition(t *testing.T) { const maxEntries = 16 data := []byte("testdata") testEntry := pb.Entry{Data: data} - maxEntrySize := uint64(maxEntries * PayloadSize(testEntry)) + maxEntrySize := maxEntries * payloadSize(testEntry) t.Log("maxEntrySize", maxEntrySize) s := newTestMemoryStorage(withPeers(1)) cfg := newTestConfig(1, 10, 1, s) - cfg.MaxUncommittedEntriesSize = maxEntrySize + cfg.MaxUncommittedEntriesSize = uint64(maxEntrySize) rawNode, err := NewRawNode(cfg) if err != nil { t.Fatal(err) @@ -998,7 +1010,7 @@ func TestRawNodeBoundedLogGrowthWithPartition(t *testing.T) { // Check the size of leader's uncommitted log tail. It should not exceed the // MaxUncommittedEntriesSize limit. - checkUncommitted := func(exp uint64) { + checkUncommitted := func(exp entryPayloadSize) { t.Helper() if a := rawNode.raft.uncommittedSize; exp != a { t.Fatalf("expected %d uncommitted entry bytes, found %d", exp, a) diff --git a/storage.go b/storage.go index ecccb71b44c6..b781dfa88eb9 100644 --- a/storage.go +++ b/storage.go @@ -131,7 +131,7 @@ func (ms *MemoryStorage) Entries(lo, hi, maxSize uint64) ([]pb.Entry, error) { } ents := ms.ents[lo-offset : hi-offset] - return limitSize(ents, maxSize), nil + return limitSize(ents, entryEncodingSize(maxSize)), nil } // Term implements the Storage interface. diff --git a/testdata/async_storage_writes.txt b/testdata/async_storage_writes.txt new file mode 100644 index 000000000000..52f37092f988 --- /dev/null +++ b/testdata/async_storage_writes.txt @@ -0,0 +1,768 @@ +# Build a pipeline of in-progress entries to append. Let the pipeline grow to a +# depth of 2 before we start allowing work on the append thread to complete. + +add-nodes 3 voters=(1,2,3) index=10 async-storage-writes=true +---- +INFO 1 switched to configuration voters=(1 2 3) +INFO 1 became follower at term 0 +INFO newRaft 1 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] +INFO 2 switched to configuration voters=(1 2 3) +INFO 2 became follower at term 0 +INFO newRaft 2 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] +INFO 3 switched to configuration voters=(1 2 3) +INFO 3 became follower at term 0 +INFO newRaft 3 [peers: [1,2,3], term: 0, commit: 10, applied: 10, lastindex: 10, lastterm: 1] + +campaign 1 +---- +INFO 1 is starting a new election at term 0 +INFO 1 became candidate at term 1 +INFO 1 [logterm: 1, index: 10] sent MsgVote request to 2 at term 1 +INFO 1 [logterm: 1, index: 10] sent MsgVote request to 3 at term 1 + +stabilize +---- +> 1 handling Ready + Ready MustSync=true: + Lead:0 State:StateCandidate + HardState Term:1 Vote:1 Commit:10 + Messages: + 1->2 MsgVote Term:1 Log:1/10 + 1->3 MsgVote Term:1 Log:1/10 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses:[1->1 MsgVoteResp Term:1 Log:0/0] +> 2 receiving messages + 1->2 MsgVote Term:1 Log:1/10 + INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1] + INFO 2 became follower at term 1 + INFO 2 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 +> 3 receiving messages + 1->3 MsgVote Term:1 Log:1/10 + INFO 3 [term: 0] received a MsgVote message with higher term from 1 [term: 1] + INFO 3 became follower at term 1 + INFO 3 [logterm: 1, index: 10, vote: 0] cast MsgVote for 1 [logterm: 1, index: 10] at term 1 +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 + Responses: + 1->1 MsgVoteResp Term:1 Log:0/0 +> 2 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:10 + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses:[2->1 MsgVoteResp Term:1 Log:0/0] +> 3 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:10 + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses:[3->1 MsgVoteResp Term:1 Log:0/0] +> 1 receiving messages + 1->1 MsgVoteResp Term:1 Log:0/0 + INFO 1 received MsgVoteResp from 1 at term 1 + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 + Responses: + 2->1 MsgVoteResp Term:1 Log:0/0 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 + Responses: + 3->1 MsgVoteResp Term:1 Log:0/0 +> 1 receiving messages + 2->1 MsgVoteResp Term:1 Log:0/0 + INFO 1 received MsgVoteResp from 2 at term 1 + INFO 1 has received 2 MsgVoteResp votes and 0 vote rejections + INFO 1 became leader at term 1 + 3->1 MsgVoteResp Term:1 Log:0/0 +> 1 handling Ready + Ready MustSync=true: + Lead:1 State:StateLeader + Entries: + 1/11 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] + 1->3 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[1->1 MsgAppResp Term:1 Log:0/11, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11] +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] +> 3 receiving messages + 1->3 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + 1->1 MsgAppResp Term:1 Log:0/11 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 +> 2 handling Ready + Ready MustSync=true: + Lead:1 State:StateFollower + Entries: + 1/11 EntryNormal "" + Messages: + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[2->1 MsgAppResp Term:1 Log:0/11, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11] +> 3 handling Ready + Ready MustSync=true: + Lead:1 State:StateFollower + Entries: + 1/11 EntryNormal "" + Messages: + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[3->1 MsgAppResp Term:1 Log:0/11, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11] +> 1 receiving messages + 1->1 MsgAppResp Term:1 Log:0/11 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + 2->1 MsgAppResp Term:1 Log:0/11 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + 3->1 MsgAppResp Term:1 Log:0/11 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/11 +> 2 receiving messages + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 +> 3 receiving messages + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:11 + CommittedEntries: + 1/11 EntryNormal "" + Messages: + 1->2 MsgApp Term:1 Log:1/11 Commit:11 + 1->3 MsgApp Term:1 Log:1/11 Commit:11 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""]] +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/11 Commit:11 +> 3 receiving messages + 1->3 MsgApp Term:1 Log:1/11 Commit:11 +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 + Responses: +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:11 + CommittedEntries: + 1/11 EntryNormal "" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses:[2->1 MsgAppResp Term:1 Log:0/11] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""]] +> 3 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:11 + CommittedEntries: + 1/11 EntryNormal "" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 Responses:[3->1 MsgAppResp Term:1 Log:0/11] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""]] +> 1 receiving messages + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 + Responses: + 2->1 MsgAppResp Term:1 Log:0/11 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:11 Vote:1 + Responses: + 3->1 MsgAppResp Term:1 Log:0/11 +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/11 + 3->1 MsgAppResp Term:1 Log:0/11 +> 2 receiving messages + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] +> 3 receiving messages + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + +propose 1 prop_1 +---- +ok + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/12 EntryNormal "prop_1" + Messages: + 1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"] + 1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"] + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[1->1 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"] +1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "prop_1"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/12 EntryNormal "prop_1" + Messages: + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[2->1 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12] +> 3 handling Ready + Ready MustSync=true: + Entries: + 1/12 EntryNormal "prop_1" + Messages: + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[3->1 MsgAppResp Term:1 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12] + +propose 1 prop_2 +---- +ok + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=true: + Entries: + 1/13 EntryNormal "prop_2" + Messages: + 1->2 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "prop_2"] + 1->3 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "prop_2"] + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[1->1 MsgAppResp Term:1 Log:0/13, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "prop_2"] +1->3 MsgApp Term:1 Log:1/12 Commit:11 Entries:[1/13 EntryNormal "prop_2"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/13 EntryNormal "prop_2" + Messages: + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[2->1 MsgAppResp Term:1 Log:0/13, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13] +> 3 handling Ready + Ready MustSync=true: + Entries: + 1/13 EntryNormal "prop_2" + Messages: + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[3->1 MsgAppResp Term:1 Log:0/13, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13] + +process-append-thread 1 2 3 +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + 1->1 MsgAppResp Term:1 Log:0/12 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + 2->1 MsgAppResp Term:1 Log:0/12 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + 3->1 MsgAppResp Term:1 Log:0/12 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12 + +deliver-msgs 1 2 3 +---- +1->1 MsgAppResp Term:1 Log:0/12 +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 +2->1 MsgAppResp Term:1 Log:0/12 +3->1 MsgAppResp Term:1 Log:0/12 +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12 +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/12 + +propose 1 prop_3 +---- +ok + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:12 + Entries: + 1/14 EntryNormal "prop_3" + CommittedEntries: + 1/12 EntryNormal "prop_1" + Messages: + 1->2 MsgApp Term:1 Log:1/13 Commit:12 + 1->3 MsgApp Term:1 Log:1/13 Commit:12 + 1->2 MsgApp Term:1 Log:1/13 Commit:12 Entries:[1/14 EntryNormal "prop_3"] + 1->3 MsgApp Term:1 Log:1/13 Commit:12 Entries:[1/14 EntryNormal "prop_3"] + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses:[1->1 MsgAppResp Term:1 Log:0/14, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/13 Commit:12 +1->2 MsgApp Term:1 Log:1/13 Commit:12 Entries:[1/14 EntryNormal "prop_3"] +1->3 MsgApp Term:1 Log:1/13 Commit:12 +1->3 MsgApp Term:1 Log:1/13 Commit:12 Entries:[1/14 EntryNormal "prop_3"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:12 + Entries: + 1/14 EntryNormal "prop_3" + CommittedEntries: + 1/12 EntryNormal "prop_1" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses:[2->1 MsgAppResp Term:1 Log:0/13, 2->1 MsgAppResp Term:1 Log:0/14, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] +> 3 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:12 + Entries: + 1/14 EntryNormal "prop_3" + CommittedEntries: + 1/12 EntryNormal "prop_1" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] Responses:[3->1 MsgAppResp Term:1 Log:0/13, 3->1 MsgAppResp Term:1 Log:0/14, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"]] + +process-append-thread 1 2 3 +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + 1->1 MsgAppResp Term:1 Log:0/13 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + 2->1 MsgAppResp Term:1 Log:0/13 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + 3->1 MsgAppResp Term:1 Log:0/13 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13 + +deliver-msgs 1 2 3 +---- +1->1 MsgAppResp Term:1 Log:0/13 +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/13 +2->1 MsgAppResp Term:1 Log:0/13 +3->1 MsgAppResp Term:1 Log:0/13 +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/13 +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/13 + +propose 1 prop_4 +---- +ok + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:13 + Entries: + 1/15 EntryNormal "prop_4" + CommittedEntries: + 1/13 EntryNormal "prop_2" + Messages: + 1->2 MsgApp Term:1 Log:1/14 Commit:13 + 1->3 MsgApp Term:1 Log:1/14 Commit:13 + 1->2 MsgApp Term:1 Log:1/14 Commit:13 Entries:[1/15 EntryNormal "prop_4"] + 1->3 MsgApp Term:1 Log:1/14 Commit:13 Entries:[1/15 EntryNormal "prop_4"] + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses:[1->1 MsgAppResp Term:1 Log:0/15, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/14 Commit:13 +1->2 MsgApp Term:1 Log:1/14 Commit:13 Entries:[1/15 EntryNormal "prop_4"] +1->3 MsgApp Term:1 Log:1/14 Commit:13 +1->3 MsgApp Term:1 Log:1/14 Commit:13 Entries:[1/15 EntryNormal "prop_4"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:13 + Entries: + 1/15 EntryNormal "prop_4" + CommittedEntries: + 1/13 EntryNormal "prop_2" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses:[2->1 MsgAppResp Term:1 Log:0/14, 2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] +> 3 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:13 + Entries: + 1/15 EntryNormal "prop_4" + CommittedEntries: + 1/13 EntryNormal "prop_2" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] Responses:[3->1 MsgAppResp Term:1 Log:0/14, 3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"]] + +process-append-thread 1 2 3 +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] + Responses: + 1->1 MsgAppResp Term:1 Log:0/14 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] + Responses: + 2->1 MsgAppResp Term:1 Log:0/13 + 2->1 MsgAppResp Term:1 Log:0/14 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:12 Vote:1 Entries:[1/14 EntryNormal "prop_3"] + Responses: + 3->1 MsgAppResp Term:1 Log:0/13 + 3->1 MsgAppResp Term:1 Log:0/14 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14 + +process-apply-thread 1 2 3 +---- +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + +deliver-msgs 1 2 3 +---- +1->1 MsgAppResp Term:1 Log:0/14 +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/14 +2->1 MsgAppResp Term:1 Log:0/13 +2->1 MsgAppResp Term:1 Log:0/14 +3->1 MsgAppResp Term:1 Log:0/13 +3->1 MsgAppResp Term:1 Log:0/14 +ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/14 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/14 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/12 EntryNormal "prop_1"] + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:14 + CommittedEntries: + 1/14 EntryNormal "prop_3" + Messages: + 1->2 MsgApp Term:1 Log:1/15 Commit:14 + 1->3 MsgApp Term:1 Log:1/15 Commit:14 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses:[AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15] + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/15 Commit:14 +1->3 MsgApp Term:1 Log:1/15 Commit:14 + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:14 + CommittedEntries: + 1/14 EntryNormal "prop_3" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses:[2->1 MsgAppResp Term:1 Log:0/15, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] +> 3 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:14 + CommittedEntries: + 1/14 EntryNormal "prop_3" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 Responses:[3->1 MsgAppResp Term:1 Log:0/15, AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"]] + +process-append-thread 1 2 3 +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] + Responses: + 1->1 MsgAppResp Term:1 Log:0/15 + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] + Responses: + 2->1 MsgAppResp Term:1 Log:0/14 + 2->1 MsgAppResp Term:1 Log:0/15 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:13 Vote:1 Entries:[1/15 EntryNormal "prop_4"] + Responses: + 3->1 MsgAppResp Term:1 Log:0/14 + 3->1 MsgAppResp Term:1 Log:0/15 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 + +process-apply-thread 1 2 3 +---- +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + +deliver-msgs 1 2 3 +---- +1->1 MsgAppResp Term:1 Log:0/15 +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 +2->1 MsgAppResp Term:1 Log:0/14 +2->1 MsgAppResp Term:1 Log:0/15 +3->1 MsgAppResp Term:1 Log:0/14 +3->1 MsgAppResp Term:1 Log:0/15 +ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/13 EntryNormal "prop_2"] + +process-ready 1 2 3 +---- +> 1 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:15 + CommittedEntries: + 1/15 EntryNormal "prop_4" + Messages: + 1->2 MsgApp Term:1 Log:1/15 Commit:15 + 1->3 MsgApp Term:1 Log:1/15 Commit:15 + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"]] +> 2 handling Ready + +> 3 handling Ready + + +deliver-msgs 1 2 3 +---- +1->2 MsgApp Term:1 Log:1/15 Commit:15 +1->3 MsgApp Term:1 Log:1/15 Commit:15 + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:15 + CommittedEntries: + 1/15 EntryNormal "prop_4" + Messages: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses:[2->1 MsgAppResp Term:1 Log:0/15] + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"]] +> 3 handling Ready + Ready MustSync=false: + HardState Term:1 Vote:1 Commit:15 + CommittedEntries: + 1/15 EntryNormal "prop_4" + Messages: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 Responses:[3->1 MsgAppResp Term:1 Log:0/15] + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] Responses:[ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"]] + +process-append-thread 2 3 +---- +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 + Responses: + 2->1 MsgAppResp Term:1 Log:0/15 + AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 + Responses: + 3->1 MsgAppResp Term:1 Log:0/15 + AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 + +process-apply-thread 1 2 3 +---- +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] + +deliver-msgs 1 2 3 +---- +2->1 MsgAppResp Term:1 Log:0/15 +3->1 MsgAppResp Term:1 Log:0/15 +ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] +AppendThread->2 MsgStorageAppendResp Term:1 Log:1/15 +INFO entry at index 15 missing from unstable log; ignoring +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] +AppendThread->3 MsgStorageAppendResp Term:1 Log:1/15 +INFO entry at index 15 missing from unstable log; ignoring +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/14 EntryNormal "prop_3"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + +> 3 handling Ready + + +process-append-thread 2 3 +---- +> 2 processing append thread + Processing: + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 + Responses: + 2->1 MsgAppResp Term:1 Log:0/15 +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 + Responses: + 3->1 MsgAppResp Term:1 Log:0/15 + +process-apply-thread 1 2 3 +---- +> 1 processing apply thread + Processing: + 1->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] + Responses: + ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] +> 2 processing apply thread + Processing: + 2->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] + Responses: + ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] +> 3 processing apply thread + Processing: + 3->ApplyThread MsgStorageApply Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] + Responses: + ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] + +deliver-msgs 1 2 3 +---- +2->1 MsgAppResp Term:1 Log:0/15 +3->1 MsgAppResp Term:1 Log:0/15 +ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] +ApplyThread->2 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] +ApplyThread->3 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/15 EntryNormal "prop_4"] + +process-ready 1 2 3 +---- +> 1 handling Ready + +> 2 handling Ready + +> 3 handling Ready + + +stabilize +---- +> 1 processing append thread + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:14 Vote:1 + Responses: + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 + Processing: + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:15 Vote:1 + Responses: +> 1 receiving messages + AppendThread->1 MsgStorageAppendResp Term:1 Log:1/15 + INFO entry at index 15 missing from unstable log; ignoring diff --git a/testdata/async_storage_writes_append_aba_race.txt b/testdata/async_storage_writes_append_aba_race.txt new file mode 100644 index 000000000000..b1e82f1261fc --- /dev/null +++ b/testdata/async_storage_writes_append_aba_race.txt @@ -0,0 +1,480 @@ +# This test reproduces a scenario similar to the one described in +# newStorageAppendRespMsg, exercising a few interesting interactions +# between asynchronous storage writes, term changes, and log truncation. + +log-level none +---- +ok + +add-nodes 7 voters=(1,2,3,4,5,6,7) index=10 async-storage-writes=true +---- +ok + +# Step 1: node 2 is the leader. + +campaign 2 +---- +ok + +stabilize +---- +ok (quiet) + +log-level info +---- +ok + +# Step 2: node 2 proposes some log entries but only node 1 receives these entries. + +propose 2 init_prop +---- +ok + +process-ready 2 +---- +Ready MustSync=true: +Entries: +1/12 EntryNormal "init_prop" +Messages: +2->1 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->4 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->5 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->6 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->7 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[2->2 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12] + +deliver-msgs 1 drop=(3,4,5,6,7) +---- +2->1 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->4 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->5 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->6 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] +dropped: 2->7 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"] + +# Step 3: node 1 gets the Ready and the entries are appended asynchronously. + +process-ready 1 +---- +Ready MustSync=true: +Entries: +1/12 EntryNormal "init_prop" +Messages: +1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[1->2 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12] + +# Step 4: node 3 becomes the leader after getting a vote from nodes 4, 5, and 6. + +campaign 3 +---- +INFO 3 is starting a new election at term 1 +INFO 3 became candidate at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 1 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 2 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 4 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 5 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 6 at term 2 +INFO 3 [logterm: 1, index: 11] sent MsgVote request to 7 at term 2 + +process-ready 3 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:2 Vote:3 Commit:11 +Messages: +3->1 MsgVote Term:2 Log:1/11 +3->2 MsgVote Term:2 Log:1/11 +3->4 MsgVote Term:2 Log:1/11 +3->5 MsgVote Term:2 Log:1/11 +3->6 MsgVote Term:2 Log:1/11 +3->7 MsgVote Term:2 Log:1/11 +3->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[3->3 MsgVoteResp Term:2 Log:0/0] + +deliver-msgs 4 5 6 +---- +3->4 MsgVote Term:2 Log:1/11 +INFO 4 [term: 1] received a MsgVote message with higher term from 3 [term: 2] +INFO 4 became follower at term 2 +INFO 4 [logterm: 1, index: 11, vote: 0] cast MsgVote for 3 [logterm: 1, index: 11] at term 2 +3->5 MsgVote Term:2 Log:1/11 +INFO 5 [term: 1] received a MsgVote message with higher term from 3 [term: 2] +INFO 5 became follower at term 2 +INFO 5 [logterm: 1, index: 11, vote: 0] cast MsgVote for 3 [logterm: 1, index: 11] at term 2 +3->6 MsgVote Term:2 Log:1/11 +INFO 6 [term: 1] received a MsgVote message with higher term from 3 [term: 2] +INFO 6 became follower at term 2 +INFO 6 [logterm: 1, index: 11, vote: 0] cast MsgVote for 3 [logterm: 1, index: 11] at term 2 + +process-ready 4 5 6 +---- +> 4 handling Ready + Ready MustSync=true: + Lead:0 State:StateFollower + HardState Term:2 Vote:3 Commit:11 + Messages: + 4->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[4->3 MsgVoteResp Term:2 Log:0/0] +> 5 handling Ready + Ready MustSync=true: + Lead:0 State:StateFollower + HardState Term:2 Vote:3 Commit:11 + Messages: + 5->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[5->3 MsgVoteResp Term:2 Log:0/0] +> 6 handling Ready + Ready MustSync=true: + Lead:0 State:StateFollower + HardState Term:2 Vote:3 Commit:11 + Messages: + 6->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[6->3 MsgVoteResp Term:2 Log:0/0] + +process-append-thread 3 4 5 6 +---- +> 3 processing append thread + Processing: + 3->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 + Responses: + 3->3 MsgVoteResp Term:2 Log:0/0 +> 4 processing append thread + Processing: + 4->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 + Responses: + 4->3 MsgVoteResp Term:2 Log:0/0 +> 5 processing append thread + Processing: + 5->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 + Responses: + 5->3 MsgVoteResp Term:2 Log:0/0 +> 6 processing append thread + Processing: + 6->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 + Responses: + 6->3 MsgVoteResp Term:2 Log:0/0 + +deliver-msgs 3 +---- +3->3 MsgVoteResp Term:2 Log:0/0 +INFO 3 received MsgVoteResp from 3 at term 2 +INFO 3 has received 1 MsgVoteResp votes and 0 vote rejections +4->3 MsgVoteResp Term:2 Log:0/0 +INFO 3 received MsgVoteResp from 4 at term 2 +INFO 3 has received 2 MsgVoteResp votes and 0 vote rejections +5->3 MsgVoteResp Term:2 Log:0/0 +INFO 3 received MsgVoteResp from 5 at term 2 +INFO 3 has received 3 MsgVoteResp votes and 0 vote rejections +6->3 MsgVoteResp Term:2 Log:0/0 +INFO 3 received MsgVoteResp from 6 at term 2 +INFO 3 has received 4 MsgVoteResp votes and 0 vote rejections +INFO 3 became leader at term 2 + +# Step 5: node 3 proposes some log entries and node 1 receives these entries, +# overwriting the previous unstable log entries that are in the process of being +# appended. The entries have a larger term than the previous entries but the +# same indexes. It begins appending these new entries asynchronously. + +process-ready 3 +---- +Ready MustSync=true: +Lead:3 State:StateLeader +Entries: +2/12 EntryNormal "" +Messages: +3->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->2 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->4 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->5 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->6 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->7 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[2/12 EntryNormal ""] Responses:[3->3 MsgAppResp Term:2 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:2 Log:2/12] + +deliver-msgs 1 drop=(2,4,5,6,7) +---- +3->1 MsgVote Term:2 Log:1/11 +INFO 1 [term: 1] received a MsgVote message with higher term from 3 [term: 2] +INFO 1 became follower at term 2 +INFO 1 [logterm: 1, index: 12, vote: 0] rejected MsgVote from 3 [logterm: 1, index: 11] at term 2 +3->1 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +INFO found conflict at index 12 [existing term: 1, conflicting term: 2] +INFO replace the unstable entries from index 12 +dropped: 3->2 MsgVote Term:2 Log:1/11 +dropped: 3->2 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +dropped: 3->4 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +dropped: 3->5 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +dropped: 3->6 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] +dropped: 3->7 MsgVote Term:2 Log:1/11 +dropped: 3->7 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] + +process-ready 1 +---- +Ready MustSync=true: +Lead:3 State:StateFollower +HardState Term:2 Commit:11 +Entries: +2/12 EntryNormal "" +Messages: +1->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Entries:[2/12 EntryNormal ""] Responses:[1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0), 1->3 MsgAppResp Term:2 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12] + +# Step 6: node 3 crashes and node 4 becomes leader getting the vote from 5, 6, and 7. + +campaign 4 +---- +INFO 4 is starting a new election at term 2 +INFO 4 became candidate at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 1 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 2 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 3 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 5 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 6 at term 3 +INFO 4 [logterm: 1, index: 11] sent MsgVote request to 7 at term 3 + +process-ready 4 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:3 Vote:4 Commit:11 +Messages: +4->1 MsgVote Term:3 Log:1/11 +4->2 MsgVote Term:3 Log:1/11 +4->3 MsgVote Term:3 Log:1/11 +4->5 MsgVote Term:3 Log:1/11 +4->6 MsgVote Term:3 Log:1/11 +4->7 MsgVote Term:3 Log:1/11 +4->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[4->4 MsgVoteResp Term:3 Log:0/0] + +deliver-msgs 5 6 7 +---- +4->5 MsgVote Term:3 Log:1/11 +INFO 5 [term: 2] received a MsgVote message with higher term from 4 [term: 3] +INFO 5 became follower at term 3 +INFO 5 [logterm: 1, index: 11, vote: 0] cast MsgVote for 4 [logterm: 1, index: 11] at term 3 +4->6 MsgVote Term:3 Log:1/11 +INFO 6 [term: 2] received a MsgVote message with higher term from 4 [term: 3] +INFO 6 became follower at term 3 +INFO 6 [logterm: 1, index: 11, vote: 0] cast MsgVote for 4 [logterm: 1, index: 11] at term 3 +4->7 MsgVote Term:3 Log:1/11 +INFO 7 [term: 1] received a MsgVote message with higher term from 4 [term: 3] +INFO 7 became follower at term 3 +INFO 7 [logterm: 1, index: 11, vote: 0] cast MsgVote for 4 [logterm: 1, index: 11] at term 3 + +process-ready 5 6 7 +---- +> 5 handling Ready + Ready MustSync=true: + HardState Term:3 Vote:4 Commit:11 + Messages: + 5->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[5->4 MsgVoteResp Term:3 Log:0/0] +> 6 handling Ready + Ready MustSync=true: + HardState Term:3 Vote:4 Commit:11 + Messages: + 6->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[6->4 MsgVoteResp Term:3 Log:0/0] +> 7 handling Ready + Ready MustSync=true: + Lead:0 State:StateFollower + HardState Term:3 Vote:4 Commit:11 + Messages: + 7->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[7->4 MsgVoteResp Term:3 Log:0/0] + +process-append-thread 4 5 6 7 +---- +> 4 processing append thread + Processing: + 4->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 + Responses: + 4->4 MsgVoteResp Term:3 Log:0/0 +> 5 processing append thread + Processing: + 5->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 + Responses: + 5->4 MsgVoteResp Term:3 Log:0/0 +> 6 processing append thread + Processing: + 6->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 + Responses: + 6->4 MsgVoteResp Term:3 Log:0/0 +> 7 processing append thread + Processing: + 7->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 + Responses: + 7->4 MsgVoteResp Term:3 Log:0/0 + +deliver-msgs 4 +---- +4->4 MsgVoteResp Term:3 Log:0/0 +INFO 4 received MsgVoteResp from 4 at term 3 +INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections +5->4 MsgVoteResp Term:3 Log:0/0 +INFO 4 received MsgVoteResp from 5 at term 3 +INFO 4 has received 2 MsgVoteResp votes and 0 vote rejections +6->4 MsgVoteResp Term:3 Log:0/0 +INFO 4 received MsgVoteResp from 6 at term 3 +INFO 4 has received 3 MsgVoteResp votes and 0 vote rejections +7->4 MsgVoteResp Term:3 Log:0/0 +INFO 4 received MsgVoteResp from 7 at term 3 +INFO 4 has received 4 MsgVoteResp votes and 0 vote rejections +INFO 4 became leader at term 3 + +process-ready 4 +---- +Ready MustSync=true: +Lead:4 State:StateLeader +Entries: +3/12 EntryNormal "" +Messages: +4->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->2 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->3 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->5 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->6 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->7 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +4->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[4->4 MsgAppResp Term:3 Log:0/12, AppendThread->4 MsgStorageAppendResp Term:3 Log:3/12] + +# Step 7: before the new entries reach node 1, it hears of the term change +# through a heartbeat and persists the new term. Node 1 then receives these +# entries, overwriting the previous unstable log entries that are in the process +# of being appended. The entries have a larger term than the previous entries +# but the same indexes. It begins appending these new entries asynchronously. + +deliver-msgs drop=1 +---- +dropped: 4->1 MsgVote Term:3 Log:1/11 +dropped: 4->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] + +tick-heartbeat 4 +---- +ok + +process-ready 4 +---- +Ready MustSync=false: +Messages: +4->1 MsgHeartbeat Term:3 Log:0/0 +4->2 MsgHeartbeat Term:3 Log:0/0 +4->3 MsgHeartbeat Term:3 Log:0/0 +4->5 MsgHeartbeat Term:3 Log:0/0 +4->6 MsgHeartbeat Term:3 Log:0/0 +4->7 MsgHeartbeat Term:3 Log:0/0 + +deliver-msgs 1 +---- +4->1 MsgHeartbeat Term:3 Log:0/0 +INFO 1 [term: 2] received a MsgHeartbeat message with higher term from 4 [term: 3] +INFO 1 became follower at term 3 + +process-ready 1 +---- +Ready MustSync=true: +Lead:4 State:StateFollower +HardState Term:3 Commit:11 +Messages: +1->4 MsgHeartbeatResp Term:3 Log:0/0 +1->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Responses:[AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12] + +deliver-msgs 4 +---- +1->4 MsgHeartbeatResp Term:3 Log:0/0 + +process-ready 4 +---- +Ready MustSync=false: +Messages: +4->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] + +deliver-msgs 1 +---- +4->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] +INFO found conflict at index 12 [existing term: 2, conflicting term: 3] +INFO replace the unstable entries from index 12 + +process-ready 1 +---- +Ready MustSync=true: +Entries: +3/12 EntryNormal "" +Messages: +1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[1->4 MsgAppResp Term:3 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12] + +# Step 8: The asynchronous log appends from the first Ready complete and the +# MsgStorageAppendResp is returned to the raft node state machine. A decision +# is made about whether to truncate the unstable log. + +raft-log 1 +---- +1/11 EntryNormal "" + +process-append-thread 1 +---- +Processing: +1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] +Responses: +1->2 MsgAppResp Term:1 Log:0/12 +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 + +raft-log 1 +---- +1/11 EntryNormal "" +1/12 EntryNormal "init_prop" + +# Step 9: However, the log entries from the second Ready are still in the +# asynchronous append pipeline and will overwrite (in stable storage) the +# entries from the first Ready at some future point. We can't truncate the +# unstable log yet or a future read from Storage might see the entries from step +# 5 before they have been replaced by the entries from step 7. Instead, we must +# wait until we are sure that the entries are stable and that no in-progress +# appends might overwrite them before removing entries from the unstable log. + +deliver-msgs 1 +---- +AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12 +INFO 1 [term: 3] ignored entry appends from a MsgStorageAppendResp message with lower term [term: 1] + +process-append-thread 1 +---- +Processing: +1->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Entries:[2/12 EntryNormal ""] +Responses: +1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0) +1->3 MsgAppResp Term:2 Log:0/12 +AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12 + +raft-log 1 +---- +1/11 EntryNormal "" +2/12 EntryNormal "" + +deliver-msgs 1 +---- +AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12 +INFO 1 [term: 3] ignored entry appends from a MsgStorageAppendResp message with lower term [term: 2] + +process-append-thread 1 +---- +Processing: +1->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 +Responses: +AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12 + +raft-log 1 +---- +1/11 EntryNormal "" +2/12 EntryNormal "" + +deliver-msgs 1 +---- +AppendThread->1 MsgStorageAppendResp Term:3 Log:2/12 +INFO entry at (index,term)=(12,2) mismatched with entry at (12,3) in unstable log; ignoring + +process-append-thread 1 +---- +Processing: +1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] +Responses: +1->4 MsgAppResp Term:3 Log:0/12 +AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12 + +raft-log 1 +---- +1/11 EntryNormal "" +3/12 EntryNormal "" + +deliver-msgs 1 +---- +AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12 diff --git a/testdata/campaign.txt b/testdata/campaign.txt index c5deb2dc1825..4eefab36d6bc 100644 --- a/testdata/campaign.txt +++ b/testdata/campaign.txt @@ -18,7 +18,6 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 -INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 [logterm: 1, index: 2] sent MsgVote request to 2 at term 1 INFO 1 [logterm: 1, index: 2] sent MsgVote request to 3 at term 1 @@ -31,6 +30,8 @@ stabilize Messages: 1->2 MsgVote Term:1 Log:1/2 1->3 MsgVote Term:1 Log:1/2 + INFO 1 received MsgVoteResp from 1 at term 1 + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections > 2 receiving messages 1->2 MsgVote Term:1 Log:1/2 INFO 2 [term: 0] received a MsgVote message with higher term from 1 [term: 1] diff --git a/testdata/campaign_learner_must_vote.txt b/testdata/campaign_learner_must_vote.txt index 55d42aa436e6..14530b556fe0 100644 --- a/testdata/campaign_learner_must_vote.txt +++ b/testdata/campaign_learner_must_vote.txt @@ -54,7 +54,6 @@ campaign 2 ---- INFO 2 is starting a new election at term 1 INFO 2 became candidate at term 2 -INFO 2 received MsgVoteResp from 2 at term 2 INFO 2 [logterm: 1, index: 4] sent MsgVote request to 1 at term 2 INFO 2 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2 @@ -67,6 +66,8 @@ HardState Term:2 Vote:2 Commit:4 Messages: 2->1 MsgVote Term:2 Log:1/4 2->3 MsgVote Term:2 Log:1/4 +INFO 2 received MsgVoteResp from 2 at term 2 +INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections # n2 is now campaigning while n1 is down (does not respond). The latest config # has n3 as a voter, but n3 doesn't even have the corresponding conf change in diff --git a/testdata/confchange_v1_add_single.txt b/testdata/confchange_v1_add_single.txt index cd07af47944e..0419d28d4fe9 100644 --- a/testdata/confchange_v1_add_single.txt +++ b/testdata/confchange_v1_add_single.txt @@ -11,7 +11,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 # Add v2 (with an auto transition). @@ -35,7 +42,6 @@ stabilize > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader - HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChange v2 @@ -73,7 +79,7 @@ stabilize 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_double_auto.txt b/testdata/confchange_v2_add_double_auto.txt index 0979bdd6abf4..5cf4f24c06f6 100644 --- a/testdata/confchange_v2_add_double_auto.txt +++ b/testdata/confchange_v2_add_double_auto.txt @@ -13,7 +13,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 propose-conf-change 1 transition=auto @@ -36,7 +43,6 @@ process-ready 1 ---- Ready MustSync=true: Lead:1 State:StateLeader -HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 v3 @@ -96,7 +102,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2 3)&&(1) autoleave INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] @@ -170,7 +176,7 @@ stabilize 1 3 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 3 receiving messages 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 5, lastindex: 5, lastterm: 1] restored snapshot [index: 5, term: 1] INFO 3 [commit: 5] restored snapshot [index: 5, term: 1] diff --git a/testdata/confchange_v2_add_double_implicit.txt b/testdata/confchange_v2_add_double_implicit.txt index 45dfc5099b98..81b3f0de1928 100644 --- a/testdata/confchange_v2_add_double_implicit.txt +++ b/testdata/confchange_v2_add_double_implicit.txt @@ -15,7 +15,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 propose-conf-change 1 transition=implicit @@ -38,7 +45,6 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader - HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 @@ -79,7 +85,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2)&&(1) autoleave INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_single_auto.txt b/testdata/confchange_v2_add_single_auto.txt index 7ee3ab6c33c7..fe127bf53e6b 100644 --- a/testdata/confchange_v2_add_single_auto.txt +++ b/testdata/confchange_v2_add_single_auto.txt @@ -13,7 +13,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 # Add v2 (with an auto transition). @@ -36,7 +43,6 @@ stabilize > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader - HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 @@ -74,7 +80,7 @@ stabilize 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_add_single_explicit.txt b/testdata/confchange_v2_add_single_explicit.txt index b4e6e3a83cda..c51900f06b6c 100644 --- a/testdata/confchange_v2_add_single_explicit.txt +++ b/testdata/confchange_v2_add_single_explicit.txt @@ -13,7 +13,14 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 + +process-ready 1 +---- +Ready MustSync=true: +Lead:0 State:StateCandidate +HardState Term:1 Vote:1 Commit:2 INFO 1 received MsgVoteResp from 1 at term 1 +INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 # Add v2 with an explicit transition. @@ -36,7 +43,6 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=true: Lead:1 State:StateLeader - HardState Term:1 Vote:1 Commit:2 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 @@ -74,7 +80,7 @@ stabilize 1 2 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false > 2 receiving messages 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 2 switched to configuration voters=(1 2)&&(1) INFO 2 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] diff --git a/testdata/confchange_v2_replace_leader.txt b/testdata/confchange_v2_replace_leader.txt index be7b573cbaa5..8074188d6cfa 100644 --- a/testdata/confchange_v2_replace_leader.txt +++ b/testdata/confchange_v2_replace_leader.txt @@ -133,7 +133,7 @@ stabilize 1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false > 4 receiving messages 1->4 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 4, term: 1] INFO 4 switched to configuration voters=(2 3 4)&&(1 2 3) INFO 4 [commit: 4, lastindex: 4, lastterm: 1] restored snapshot [index: 4, term: 1] INFO 4 [commit: 4] restored snapshot [index: 4, term: 1] @@ -185,7 +185,6 @@ stabilize INFO 4 [term 1] received MsgTimeoutNow from 1 and starts an election to get leadership. INFO 4 is starting a new election at term 1 INFO 4 became candidate at term 2 - INFO 4 received MsgVoteResp from 4 at term 2 INFO 4 [logterm: 1, index: 4] sent MsgVote request to 1 at term 2 INFO 4 [logterm: 1, index: 4] sent MsgVote request to 2 at term 2 INFO 4 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2 @@ -197,6 +196,8 @@ stabilize 4->1 MsgVote Term:2 Log:1/4 4->2 MsgVote Term:2 Log:1/4 4->3 MsgVote Term:2 Log:1/4 + INFO 4 received MsgVoteResp from 4 at term 2 + INFO 4 has received 1 MsgVoteResp votes and 0 vote rejections > 1 receiving messages 4->1 MsgVote Term:2 Log:1/4 INFO 1 [term: 1] received a MsgVote message with higher term from 4 [term: 2] diff --git a/testdata/probe_and_replicate.txt b/testdata/probe_and_replicate.txt index bebae6ef9c85..8e61b6187072 100644 --- a/testdata/probe_and_replicate.txt +++ b/testdata/probe_and_replicate.txt @@ -357,7 +357,6 @@ campaign 1 ---- INFO 1 is starting a new election at term 7 INFO 1 became candidate at term 8 -INFO 1 received MsgVoteResp from 1 at term 8 INFO 1 [logterm: 6, index: 20] sent MsgVote request to 2 at term 8 INFO 1 [logterm: 6, index: 20] sent MsgVote request to 3 at term 8 INFO 1 [logterm: 6, index: 20] sent MsgVote request to 4 at term 8 @@ -379,6 +378,8 @@ stabilize 1 1->5 MsgVote Term:8 Log:6/20 1->6 MsgVote Term:8 Log:6/20 1->7 MsgVote Term:8 Log:6/20 + INFO 1 received MsgVoteResp from 1 at term 8 + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections stabilize 2 3 4 5 6 7 ---- diff --git a/testdata/replicate_pause.txt b/testdata/replicate_pause.txt index e7333cccb413..67e64a95010a 100644 --- a/testdata/replicate_pause.txt +++ b/testdata/replicate_pause.txt @@ -187,4 +187,4 @@ status 1 ---- 1: StateReplicate match=17 next=18 2: StateReplicate match=17 next=18 -3: StateReplicate match=17 next=18 \ No newline at end of file +3: StateReplicate match=17 next=18 diff --git a/testdata/single_node.txt b/testdata/single_node.txt index 3b6e4f4c1dd9..f6aceb712ff4 100644 --- a/testdata/single_node.txt +++ b/testdata/single_node.txt @@ -12,15 +12,19 @@ campaign 1 ---- INFO 1 is starting a new election at term 0 INFO 1 became candidate at term 1 -INFO 1 received MsgVoteResp from 1 at term 1 -INFO 1 became leader at term 1 stabilize ---- > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + Lead:0 State:StateCandidate HardState Term:1 Vote:1 Commit:3 + INFO 1 received MsgVoteResp from 1 at term 1 + INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections + INFO 1 became leader at term 1 +> 1 handling Ready + Ready MustSync=true: + Lead:1 State:StateLeader Entries: 1/4 EntryNormal "" > 1 handling Ready diff --git a/testdata/snapshot_succeed_via_app_resp.txt b/testdata/snapshot_succeed_via_app_resp.txt index dbbd5ce11d98..a52a683e1555 100644 --- a/testdata/snapshot_succeed_via_app_resp.txt +++ b/testdata/snapshot_succeed_via_app_resp.txt @@ -107,7 +107,7 @@ stabilize 3 ---- > 3 receiving messages 1->3 MsgSnap Term:1 Log:0/0 Snapshot: Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1] + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 11, term: 1] INFO 3 switched to configuration voters=(1 2 3) INFO 3 [commit: 11, lastindex: 11, lastterm: 1] restored snapshot [index: 11, term: 1] INFO 3 [commit: 11] restored snapshot [index: 11, term: 1] diff --git a/util.go b/util.go index d0872182907c..6b68fe2d91cc 100644 --- a/util.go +++ b/util.go @@ -41,20 +41,26 @@ func max(a, b uint64) uint64 { } var isLocalMsg = [...]bool{ - pb.MsgHup: true, - pb.MsgBeat: true, - pb.MsgUnreachable: true, - pb.MsgSnapStatus: true, - pb.MsgCheckQuorum: true, + pb.MsgHup: true, + pb.MsgBeat: true, + pb.MsgUnreachable: true, + pb.MsgSnapStatus: true, + pb.MsgCheckQuorum: true, + pb.MsgStorageAppend: true, + pb.MsgStorageAppendResp: true, + pb.MsgStorageApply: true, + pb.MsgStorageApplyResp: true, } var isResponseMsg = [...]bool{ - pb.MsgAppResp: true, - pb.MsgVoteResp: true, - pb.MsgHeartbeatResp: true, - pb.MsgUnreachable: true, - pb.MsgReadIndexResp: true, - pb.MsgPreVoteResp: true, + pb.MsgAppResp: true, + pb.MsgVoteResp: true, + pb.MsgHeartbeatResp: true, + pb.MsgUnreachable: true, + pb.MsgReadIndexResp: true, + pb.MsgPreVoteResp: true, + pb.MsgStorageAppendResp: true, + pb.MsgStorageApplyResp: true, } func isMsgInArray(msgt pb.MessageType, arr []bool) bool { @@ -70,6 +76,10 @@ func IsResponseMsg(msgt pb.MessageType) bool { return isMsgInArray(msgt, isResponseMsg[:]) } +func IsLocalMsgTarget(id uint64) bool { + return id == LocalAppendThread || id == LocalApplyThread +} + // voteResponseType maps vote and prevote message types to their corresponding responses. func voteRespMsgType(msgt pb.MessageType) pb.MessageType { switch msgt { @@ -153,13 +163,17 @@ type EntryFormatter func([]byte) string // Message for debugging. func DescribeMessage(m pb.Message, f EntryFormatter) string { var buf bytes.Buffer - fmt.Fprintf(&buf, "%x->%x %v Term:%d Log:%d/%d", m.From, m.To, m.Type, m.Term, m.LogTerm, m.Index) + fmt.Fprintf(&buf, "%s->%s %v Term:%d Log:%d/%d", + describeTarget(m.From), describeTarget(m.To), m.Type, m.Term, m.LogTerm, m.Index) if m.Reject { fmt.Fprintf(&buf, " Rejected (Hint: %d)", m.RejectHint) } if m.Commit != 0 { fmt.Fprintf(&buf, " Commit:%d", m.Commit) } + if m.Vote != 0 { + fmt.Fprintf(&buf, " Vote:%d", m.Vote) + } if len(m.Entries) > 0 { fmt.Fprint(&buf, " Entries:[") for i, e := range m.Entries { @@ -173,13 +187,30 @@ func DescribeMessage(m pb.Message, f EntryFormatter) string { if s := m.Snapshot; s != nil && !IsEmptySnap(*s) { fmt.Fprintf(&buf, " Snapshot: %s", DescribeSnapshot(*s)) } + if len(m.Responses) > 0 { + fmt.Fprintf(&buf, " Responses:[") + for i, m := range m.Responses { + if i != 0 { + buf.WriteString(", ") + } + buf.WriteString(DescribeMessage(m, f)) + } + fmt.Fprintf(&buf, "]") + } return buf.String() } -// PayloadSize is the size of the payload of this Entry. Notably, it does not -// depend on its Index or Term. -func PayloadSize(e pb.Entry) int { - return len(e.Data) +func describeTarget(id uint64) string { + switch id { + case None: + return "None" + case LocalAppendThread: + return "AppendThread" + case LocalApplyThread: + return "ApplyThread" + default: + return fmt.Sprintf("%x", id) + } } // DescribeEntry returns a concise human-readable description of an @@ -230,7 +261,19 @@ func DescribeEntries(ents []pb.Entry, f EntryFormatter) string { return buf.String() } -func limitSize(ents []pb.Entry, maxSize uint64) []pb.Entry { +// entryEncodingSize represents the protocol buffer encoding size of one or more +// entries. +type entryEncodingSize uint64 + +func entsSize(ents []pb.Entry) entryEncodingSize { + var size entryEncodingSize + for _, ent := range ents { + size += entryEncodingSize(ent.Size()) + } + return size +} + +func limitSize(ents []pb.Entry, maxSize entryEncodingSize) []pb.Entry { if len(ents) == 0 { return ents } @@ -238,13 +281,33 @@ func limitSize(ents []pb.Entry, maxSize uint64) []pb.Entry { var limit int for limit = 1; limit < len(ents); limit++ { size += ents[limit].Size() - if uint64(size) > maxSize { + if entryEncodingSize(size) > maxSize { break } } return ents[:limit] } +// entryPayloadSize represents the size of one or more entries' payloads. +// Notably, it does not depend on its Index or Term. Entries with empty +// payloads, like those proposed after a leadership change, are considered +// to be zero size. +type entryPayloadSize uint64 + +// payloadSize is the size of the payload of the provided entry. +func payloadSize(e pb.Entry) entryPayloadSize { + return entryPayloadSize(len(e.Data)) +} + +// payloadsSize is the size of the payloads of the provided entries. +func payloadsSize(ents []pb.Entry) entryPayloadSize { + var s entryPayloadSize + for _, e := range ents { + s += payloadSize(e) + } + return s +} + func assertConfStatesEquivalent(l Logger, cs1, cs2 pb.ConfState) { err := cs1.Equivalent(cs2) if err == nil { diff --git a/util_test.go b/util_test.go index d8f69dbb88ff..fc7ddfbd5263 100644 --- a/util_test.go +++ b/util_test.go @@ -60,7 +60,7 @@ func TestLimitSize(t *testing.T) { for _, tt := range tests { t.Run("", func(t *testing.T) { - require.Equal(t, tt.wentries, limitSize(ents, tt.maxsize)) + require.Equal(t, tt.wentries, limitSize(ents, entryEncodingSize(tt.maxsize))) }) } } @@ -89,6 +89,10 @@ func TestIsLocalMsg(t *testing.T) { {pb.MsgReadIndexResp, false}, {pb.MsgPreVote, false}, {pb.MsgPreVoteResp, false}, + {pb.MsgStorageAppend, true}, + {pb.MsgStorageAppendResp, true}, + {pb.MsgStorageApply, true}, + {pb.MsgStorageApplyResp, true}, } for _, tt := range tests { @@ -122,6 +126,10 @@ func TestIsResponseMsg(t *testing.T) { {pb.MsgReadIndexResp, true}, {pb.MsgPreVote, false}, {pb.MsgPreVoteResp, true}, + {pb.MsgStorageAppend, false}, + {pb.MsgStorageAppendResp, true}, + {pb.MsgStorageApply, false}, + {pb.MsgStorageApplyResp, true}, } for i, tt := range tests {