diff --git a/raft/node_test.go b/raft/node_test.go index 224a46414..abb6a42b3 100644 --- a/raft/node_test.go +++ b/raft/node_test.go @@ -600,9 +600,6 @@ func TestReadyContainUpdates(t *testing.T) { // start with correct configuration change entries, and can accept and commit // proposals. func TestNodeStart(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cc := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1} ccdata, err := cc.Marshal() if err != nil { @@ -620,11 +617,17 @@ func TestNodeStart(t *testing.T) { MustSync: true, }, { - HardState: raftpb.HardState{Term: 2, Commit: 3, Vote: 1}, + HardState: raftpb.HardState{Term: 2, Commit: 2, Vote: 1}, Entries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}}, - CommittedEntries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}}, + CommittedEntries: []raftpb.Entry{{Term: 2, Index: 2, Data: nil}}, MustSync: true, }, + { + HardState: raftpb.HardState{Term: 2, Commit: 3, Vote: 1}, + Entries: nil, + CommittedEntries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}}, + MustSync: false, + }, } storage := NewMemoryStorage() c := &Config{ @@ -636,27 +639,44 @@ func TestNodeStart(t *testing.T) { MaxInflightMsgs: 256, } n := StartNode(c, []Peer{{ID: 1}}) - defer n.Stop() - g := <-n.Ready() - if !reflect.DeepEqual(g, wants[0]) { - t.Fatalf("#%d: g = %+v,\n w %+v", 1, g, wants[0]) - } else { - storage.Append(g.Entries) + ctx, cancel, n := newNodeTestHarness(t, context.Background(), c, Peer{ID: 1}) + defer cancel() + + { + rd := <-n.Ready() + if !reflect.DeepEqual(rd, wants[0]) { + t.Fatalf("#1: rd = %+v,\n w %+v", rd, wants[0]) + } + storage.Append(rd.Entries) n.Advance() } if err := n.Campaign(ctx); err != nil { t.Fatal(err) } - rd := <-n.Ready() - storage.Append(rd.Entries) - n.Advance() + + { + rd := <-n.Ready() + storage.Append(rd.Entries) + n.Advance() + } n.Propose(ctx, []byte("foo")) - if g2 := <-n.Ready(); !reflect.DeepEqual(g2, wants[1]) { - t.Errorf("#%d: g = %+v,\n w %+v", 2, g2, wants[1]) - } else { - storage.Append(g2.Entries) + { + rd := <-n.Ready() + if !reflect.DeepEqual(rd, wants[1]) { + t.Errorf("#2: rd = %+v,\n w %+v", rd, wants[1]) + } + storage.Append(rd.Entries) + n.Advance() + } + + { + rd := <-n.Ready() + if !reflect.DeepEqual(rd, wants[2]) { + t.Errorf("#3: rd = %+v,\n w %+v", rd, wants[2]) + } + storage.Append(rd.Entries) n.Advance() }