raft: use RawNode for node's event loop

It has always bugged me that any new feature essentially needed to be
tested twice due to the two ways in which apps can use raft (`*node` and
`*RawNode`). Due to upcoming testing work for joint consensus, now is a
good time to rectify this somewhat.

This commit removes most logic from `(*node).run` and uses `*RawNode`
internally. This simplifies the logic and also lead (via debugging) to
some insight on how the semantics of the approaches differ, which is now
documented in the comments.
This commit is contained in:
Tobias Schottdorf
2019-07-15 17:18:56 +02:00
parent 233be58056
commit c62b7048b5
5 changed files with 173 additions and 167 deletions

View File

@@ -132,9 +132,12 @@ func TestNodePropose(t *testing.T) {
n := newNode()
s := NewMemoryStorage()
r := newTestRaft(1, []uint64{1}, 10, 1, s)
go n.run(r)
n.Campaign(context.TODO())
rn := newTestRawNode(1, []uint64{1}, 10, 1, s)
r := rn.raft
go n.run(rn)
if err := n.Campaign(context.TODO()); err != nil {
t.Fatal(err)
}
for {
rd := <-n.Ready()
s.Append(rd.Entries)
@@ -172,10 +175,11 @@ func TestNodeReadIndex(t *testing.T) {
n := newNode()
s := NewMemoryStorage()
r := newTestRaft(1, []uint64{1}, 10, 1, s)
rn := newTestRawNode(1, []uint64{1}, 10, 1, s)
r := rn.raft
r.readStates = wrs
go n.run(r)
go n.run(rn)
n.Campaign(context.TODO())
for {
rd := <-n.Ready()
@@ -309,8 +313,9 @@ func TestNodeProposeConfig(t *testing.T) {
n := newNode()
s := NewMemoryStorage()
r := newTestRaft(1, []uint64{1}, 10, 1, s)
go n.run(r)
rn := newTestRawNode(1, []uint64{1}, 10, 1, s)
r := rn.raft
go n.run(rn)
n.Campaign(context.TODO())
for {
rd := <-n.Ready()
@@ -347,8 +352,8 @@ func TestNodeProposeConfig(t *testing.T) {
func TestNodeProposeAddDuplicateNode(t *testing.T) {
n := newNode()
s := NewMemoryStorage()
r := newTestRaft(1, []uint64{1}, 10, 1, s)
go n.run(r)
rn := newTestRawNode(1, []uint64{1}, 10, 1, s)
go n.run(rn)
n.Campaign(context.TODO())
rdyEntries := make([]raftpb.Entry, 0)
ticker := time.NewTicker(time.Millisecond * 100)
@@ -422,8 +427,8 @@ func TestNodeProposeAddDuplicateNode(t *testing.T) {
// who is the current leader.
func TestBlockProposal(t *testing.T) {
n := newNode()
r := newTestRaft(1, []uint64{1}, 10, 1, NewMemoryStorage())
go n.run(r)
rn := newTestRawNode(1, []uint64{1}, 10, 1, NewMemoryStorage())
go n.run(rn)
defer n.Stop()
errc := make(chan error, 1)
@@ -463,8 +468,9 @@ func TestNodeProposeWaitDropped(t *testing.T) {
n := newNode()
s := NewMemoryStorage()
r := newTestRaft(1, []uint64{1}, 10, 1, s)
go n.run(r)
rn := newTestRawNode(1, []uint64{1}, 10, 1, s)
r := rn.raft
go n.run(rn)
n.Campaign(context.TODO())
for {
rd := <-n.Ready()
@@ -497,8 +503,9 @@ func TestNodeProposeWaitDropped(t *testing.T) {
func TestNodeTick(t *testing.T) {
n := newNode()
s := NewMemoryStorage()
r := newTestRaft(1, []uint64{1}, 10, 1, s)
go n.run(r)
rn := newTestRawNode(1, []uint64{1}, 10, 1, s)
r := rn.raft
go n.run(rn)
elapsed := r.electionElapsed
n.Tick()
@@ -517,11 +524,11 @@ func TestNodeTick(t *testing.T) {
func TestNodeStop(t *testing.T) {
n := newNode()
s := NewMemoryStorage()
r := newTestRaft(1, []uint64{1}, 10, 1, s)
rn := newTestRawNode(1, []uint64{1}, 10, 1, s)
donec := make(chan struct{})
go func() {
n.run(r)
n.run(rn)
close(donec)
}()
@@ -618,7 +625,9 @@ func TestNodeStart(t *testing.T) {
n.Advance()
}
n.Campaign(ctx)
if err := n.Campaign(ctx); err != nil {
t.Fatal(err)
}
rd := <-n.Ready()
storage.Append(rd.Entries)
n.Advance()
@@ -646,10 +655,12 @@ func TestNodeRestart(t *testing.T) {
st := raftpb.HardState{Term: 1, Commit: 1}
want := Ready{
HardState: st,
// No HardState is emitted because there was no change.
HardState: raftpb.HardState{},
// commit up to index commit index in st
CommittedEntries: entries[:st.Commit],
MustSync: true,
// MustSync is false because no HardState or new entries are provided.
MustSync: false,
}
storage := NewMemoryStorage()
@@ -691,10 +702,14 @@ func TestNodeRestartFromSnapshot(t *testing.T) {
st := raftpb.HardState{Term: 1, Commit: 3}
want := Ready{
HardState: st,
// No HardState is emitted because nothing changed relative to what is
// already persisted.
HardState: raftpb.HardState{},
// commit up to index commit index in st
CommittedEntries: entries,
MustSync: true,
// MustSync is only true when there is a new HardState or new entries;
// neither is the case here.
MustSync: false,
}
s := NewMemoryStorage()
@@ -800,8 +815,8 @@ func TestNodeProposeAddLearnerNode(t *testing.T) {
defer ticker.Stop()
n := newNode()
s := NewMemoryStorage()
r := newTestRaft(1, []uint64{1}, 10, 1, s)
go n.run(r)
rn := newTestRawNode(1, []uint64{1}, 10, 1, s)
go n.run(rn)
n.Campaign(context.TODO())
stop := make(chan struct{})
done := make(chan struct{})
@@ -895,9 +910,12 @@ func TestCommitPagination(t *testing.T) {
s := NewMemoryStorage()
cfg := newTestConfig(1, []uint64{1}, 10, 1, s)
cfg.MaxCommittedSizePerReady = 2048
r := newRaft(cfg)
rn, err := NewRawNode(cfg, nil)
if err != nil {
t.Fatal(err)
}
n := newNode()
go n.run(r)
go n.run(rn)
n.Campaign(context.TODO())
rd := readyWithTimeout(&n)
@@ -984,9 +1002,12 @@ func TestNodeCommitPaginationAfterRestart(t *testing.T) {
// this and *will* return it (which is how the Commit index ended up being 10 initially).
cfg.MaxSizePerMsg = size - uint64(s.ents[len(s.ents)-1].Size()) - 1
r := newRaft(cfg)
rn, err := NewRawNode(cfg, nil)
if err != nil {
t.Fatal(err)
}
n := newNode()
go n.run(r)
go n.run(rn)
defer n.Stop()
rd := readyWithTimeout(&n)
@@ -1011,9 +1032,12 @@ func TestNodeBoundedLogGrowthWithPartition(t *testing.T) {
s := NewMemoryStorage()
cfg := newTestConfig(1, []uint64{1}, 10, 1, s)
cfg.MaxUncommittedEntriesSize = maxEntrySize
r := newRaft(cfg)
rn, err := NewRawNode(cfg, nil)
if err != nil {
t.Fatal(err)
}
n := newNode()
go n.run(r)
go n.run(rn)
defer n.Stop()
n.Campaign(context.TODO())
@@ -1028,14 +1052,14 @@ func TestNodeBoundedLogGrowthWithPartition(t *testing.T) {
// committing anything. These proposals should not cause the leader's
// log to grow indefinitely.
for i := 0; i < 1024; i++ {
n.Propose(context.TODO(), data)
_ = n.Propose(context.TODO(), data)
}
// Check the size of leader's uncommitted log tail. It should not exceed the
// MaxUncommittedEntriesSize limit.
checkUncommitted := func(exp uint64) {
t.Helper()
if a := r.uncommittedSize; exp != a {
if a := rn.raft.uncommittedSize; exp != a {
t.Fatalf("expected %d uncommitted entry bytes, found %d", exp, a)
}
}