From 3152dc81744d3027fdae6841a50ca7184ce42b05 Mon Sep 17 00:00:00 2001 From: David Wyrobnik Date: Mon, 11 Apr 2022 23:34:57 +0000 Subject: [PATCH] contrib/raftexample: Save snapshot and WAL before hard state Update raftexample to save the snapshot file and WAL snapshot entry before hardstate to ensure the snapshot exists during recovery. Otherwise if there is a failure after storing the hard state there may be reference to a non-existent snapshot. This PR introduces the fix from #10219 to the raftexample. --- contrib/raftexample/raft.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/contrib/raftexample/raft.go b/contrib/raftexample/raft.go index 49a18f3ea..b1618e1c1 100644 --- a/contrib/raftexample/raft.go +++ b/contrib/raftexample/raft.go @@ -449,9 +449,13 @@ func (rc *raftNode) serveChannels() { // store raft entries to wal, then publish over commit channel case rd := <-rc.node.Ready(): - rc.wal.Save(rd.HardState, rd.Entries) + // Must save the snapshot file and WAL snapshot entry before saving any other entries + // or hardstate to ensure that recovery after a snapshot restore is possible. if !raft.IsEmptySnap(rd.Snapshot) { rc.saveSnap(rd.Snapshot) + } + rc.wal.Save(rd.HardState, rd.Entries) + if !raft.IsEmptySnap(rd.Snapshot) { rc.raftStorage.ApplySnapshot(rd.Snapshot) rc.publishSnapshot(rd.Snapshot) }