raft: return non-nil Inflights in raft status

Recent refactoring to the String() method of `Progress` hit an NPE
because we return nil Inflights as part of the Raft status. Just
fix this at the source and properly populate the Raft status instead
of teaching String() to ignore nil. A real Progress always has a
non-nil Inflights.
release-3.4
Tobias Schottdorf 2019-07-17 12:53:28 +02:00
parent 9fba06ba3b
commit 26a1e60eab
3 changed files with 28 additions and 10 deletions

View File

@ -439,14 +439,26 @@ func TestRawNodeRestartFromSnapshot(t *testing.T) {
// no dependency check between Ready() and Advance()
func TestRawNodeStatus(t *testing.T) {
storage := NewMemoryStorage()
rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage), []Peer{{ID: 1}})
s := NewMemoryStorage()
rn, err := NewRawNode(newTestConfig(1, []uint64{1}, 10, 1, s), nil)
if err != nil {
t.Fatal(err)
}
status := rawNode.Status()
if status == nil {
t.Errorf("expected status struct, got nil")
if status := rn.Status(); status.Progress != nil {
t.Fatalf("expected no Progress because not leader: %+v", status.Progress)
}
if err := rn.Campaign(); err != nil {
t.Fatal(err)
}
status := rn.Status()
if status.Lead != 1 {
t.Fatal("not lead")
}
if status.RaftState != StateLeader {
t.Fatal("not leader")
}
if exp, act := *rn.raft.prs.Progress[1], status.Progress[1]; !reflect.DeepEqual(exp, act) {
t.Fatalf("want: %+v\ngot: %+v", exp, act)
}
}

View File

@ -37,11 +37,9 @@ func getProgressCopy(r *raft) map[uint64]tracker.Progress {
m := make(map[uint64]tracker.Progress)
r.prs.Visit(func(id uint64, pr *tracker.Progress) {
var p tracker.Progress
p, pr = *pr, nil /* avoid accidental reuse below */
// The inflight buffer is tricky to copy and besides, it isn't exposed
// to the client, so pretend it's nil.
p.Inflights = nil
p = *pr
p.Inflights = pr.Inflights.Clone()
pr = nil
m[id] = p
})

View File

@ -40,6 +40,14 @@ func NewInflights(size int) *Inflights {
}
}
// Clone returns an *Inflights that is identical to but shares no memory with
// the receiver.
func (in *Inflights) Clone() *Inflights {
ins := *in
ins.buffer = append([]uint64(nil), in.buffer...)
return &ins
}
// Add notifies the Inflights that a new message with the given index is being
// dispatched. Full() must be called prior to Add() to verify that there is room
// for one more message, and consecutive calls to add Add() must provide a