From fa247d09cc800485339f298890afc4be8f2932e4 Mon Sep 17 00:00:00 2001 From: Ben Darnell Date: Wed, 10 Dec 2014 17:34:40 -0500 Subject: [PATCH] raft: remove panic when we see a proposal with no leader. This panic can never be reached when using raft.Node, because we only read from propc when there is a leader. However, it is possible to see this error when using raft the raft object directly (as in MultiNode), and in this case it is better to simply drop the proposal (as if we had sent it to a leader that immediately vanished). Add an error return to MemoryStorage.Append for consistency. --- raft/raft.go | 6 ++++-- raft/storage.go | 7 ++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/raft/raft.go b/raft/raft.go index c62858369..d511458fe 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -479,7 +479,8 @@ func stepLeader(r *raft, m pb.Message) { func stepCandidate(r *raft, m pb.Message) { switch m.Type { case pb.MsgProp: - panic("no leader") + log.Printf("raft: no leader; dropping proposal") + return case pb.MsgApp: r.becomeFollower(r.Term, m.From) r.handleAppendEntries(m) @@ -510,7 +511,8 @@ func stepFollower(r *raft, m pb.Message) { switch m.Type { case pb.MsgProp: if r.lead == None { - panic("no leader") + log.Printf("raft: no leader; dropping proposal") + return } m.To = r.lead r.send(m) diff --git a/raft/storage.go b/raft/storage.go index 9ffc06a70..63f444d61 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -180,18 +180,18 @@ func (ms *MemoryStorage) Compact(i uint64, cs *pb.ConfState, data []byte) error } // Append the new entries to storage. -func (ms *MemoryStorage) Append(entries []pb.Entry) { +func (ms *MemoryStorage) Append(entries []pb.Entry) error { ms.Lock() defer ms.Unlock() if len(entries) == 0 { - return + return nil } first := ms.snapshot.Metadata.Index + 1 last := entries[0].Index + uint64(len(entries)) - 1 // shortcut if there is no new entry. if last < first { - return + return nil } // truncate old entries if first > entries[0].Index { @@ -209,4 +209,5 @@ func (ms *MemoryStorage) Append(entries []pb.Entry) { log.Panicf("missing log entry [last: %d, append at: %d]", ms.snapshot.Metadata.Index+uint64(len(ms.ents)), entries[0].Index) } + return nil }