update the confstate before sending snapshot

When there is a `raftpb.EntryConfChange` after creating the snapshot,
then the confState included in the snapshot is out of date. so We need
to update the confState before sending a snapshot to a follower.
This commit is contained in:
ahrtr 2022-03-07 09:47:37 +08:00
parent 8ac44ffa5f
commit 793218ed2b

View File

@ -456,7 +456,7 @@ func (rc *raftNode) serveChannels() {
rc.publishSnapshot(rd.Snapshot)
}
rc.raftStorage.Append(rd.Entries)
rc.transport.Send(rd.Messages)
rc.transport.Send(rc.processMessages(rd.Messages))
applyDoneC, ok := rc.publishEntries(rc.entriesToApply(rd.CommittedEntries))
if !ok {
rc.stop()
@ -476,6 +476,18 @@ func (rc *raftNode) serveChannels() {
}
}
// When there is a `raftpb.EntryConfChange` after creating the snapshot,
// then the confState included in the snapshot is out of date. so We need
// to update the confState before sending a snapshot to a follower.
func (rc *raftNode) processMessages(ms []raftpb.Message) []raftpb.Message {
for i := 0; i < len(ms); i++ {
if ms[i].Type == raftpb.MsgSnap {
ms[i].Snapshot.Metadata.ConfState = rc.confState
}
}
return ms
}
func (rc *raftNode) serveRaft() {
url, err := url.Parse(rc.peers[rc.id-1])
if err != nil {