mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
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:
parent
8ac44ffa5f
commit
793218ed2b
@ -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 {
|
||||
|
Loading…
x
Reference in New Issue
Block a user