raft: add DisableProposalForwarding option

this allows users to disable followers from forwarding proposals to the
leader.
This commit is contained in:
smetro
2017-05-24 21:42:18 -07:00
parent 8b09309c81
commit e461017ac5
2 changed files with 59 additions and 12 deletions

View File

@@ -190,6 +190,38 @@ func TestNodeReadIndex(t *testing.T) {
}
}
// TestDisableProposalForwarding ensures that proposals are not forwarded to
// the leader when DisableProposalForwarding is true.
func TestDisableProposalForwarding(t *testing.T) {
r1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
r2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
cfg3 := newTestConfig(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
cfg3.DisableProposalForwarding = true
r3 := newRaft(cfg3)
nt := newNetwork(r1, r2, r3)
// elect r1 as leader
nt.send(raftpb.Message{From: 1, To: 1, Type: raftpb.MsgHup})
var testEntries = []raftpb.Entry{{Data: []byte("testdata")}}
// send proposal to r2(follower) where DisableProposalForwarding is false
r2.Step(raftpb.Message{From: 2, To: 2, Type: raftpb.MsgProp, Entries: testEntries})
// verify r2(follower) does forward the proposal when DisableProposalForwarding is false
if len(r2.msgs) != 1 {
t.Fatalf("len(r2.msgs) expected 1, got %d", len(r2.msgs))
}
// send proposal to r3(follower) where DisableProposalForwarding is true
r3.Step(raftpb.Message{From: 3, To: 3, Type: raftpb.MsgProp, Entries: testEntries})
// verify r3(follower) does not forward the proposal when DisableProposalForwarding is true
if len(r3.msgs) != 0 {
t.Fatalf("len(r3.msgs) expected 0, got %d", len(r3.msgs))
}
}
// TestNodeReadIndexToOldLeader ensures that raftpb.MsgReadIndex to old leader
// gets forwarded to the new leader and 'send' method does not attach its term.
func TestNodeReadIndexToOldLeader(t *testing.T) {