Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

raft: do not attach term to MsgReadIndex #6749

Merged
merged 1 commit into from
Oct 29, 2016
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
58 changes: 58 additions & 0 deletions raft/node_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,64 @@ func TestNodeReadIndex(t *testing.T) {
}
}

// 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) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

probably all we want to test here is:

  • elect 1 as leader
  • send readindex request to 2
  • verify 2 forwards this message to 1 with term not set
  • send readindex request to 3
  • verify 3 forwards this message to 1 with term not set as well.
  • now elect 3 as leader
  • let 1 steps the two messages previously we got from 2, 3
  • verify 1 forwards these messages again to the new leader 3

r1 := newTestRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
r2 := newTestRaft(2, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
r3 := newTestRaft(3, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())

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 readindex request to r2(follower)
r2.Step(raftpb.Message{From: 2, To: 2, Type: raftpb.MsgReadIndex, Entries: testEntries})

// verify r2(follower) forwards this message to r1(leader) with term not set
if len(r2.msgs) != 1 {
t.Fatalf("len(r2.msgs) expected 1, got %d", len(r2.msgs))
}
readIndxMsg1 := raftpb.Message{From: 2, To: 1, Type: raftpb.MsgReadIndex, Entries: testEntries}
if !reflect.DeepEqual(r2.msgs[0], readIndxMsg1) {
t.Fatalf("r2.msgs[0] expected %+v, got %+v", readIndxMsg1, r2.msgs[0])
}

// send readindex request to r3(follower)
r3.Step(raftpb.Message{From: 3, To: 3, Type: raftpb.MsgReadIndex, Entries: testEntries})

// verify r3(follower) forwards this message to r1(leader) with term not set as well.
if len(r3.msgs) != 1 {
t.Fatalf("len(r3.msgs) expected 1, got %d", len(r3.msgs))
}
readIndxMsg2 := raftpb.Message{From: 3, To: 1, Type: raftpb.MsgReadIndex, Entries: testEntries}
if !reflect.DeepEqual(r3.msgs[0], readIndxMsg2) {
t.Fatalf("r3.msgs[0] expected %+v, got %+v", readIndxMsg2, r3.msgs[0])
}

// now elect r3 as leader
nt.send(raftpb.Message{From: 3, To: 3, Type: raftpb.MsgHup})

// let r1 steps the two messages previously we got from r2, r3
r1.Step(readIndxMsg1)
r1.Step(readIndxMsg2)

// verify r1(follower) forwards these messages again to r3(new leader)
if len(r1.msgs) != 2 {
t.Fatalf("len(r1.msgs) expected 1, got %d", len(r1.msgs))
}
readIndxMsg3 := raftpb.Message{From: 1, To: 3, Type: raftpb.MsgReadIndex, Entries: testEntries}
if !reflect.DeepEqual(r1.msgs[0], readIndxMsg3) {
t.Fatalf("r1.msgs[0] expected %+v, got %+v", readIndxMsg3, r1.msgs[0])
}
if !reflect.DeepEqual(r1.msgs[1], readIndxMsg3) {
t.Fatalf("r1.msgs[1] expected %+v, got %+v", readIndxMsg3, r1.msgs[1])
}
}

// TestNodeProposeConfig ensures that node.ProposeConfChange sends the given configuration proposal
// to the underlying raft.
func TestNodeProposeConfig(t *testing.T) {
Expand Down
5 changes: 3 additions & 2 deletions raft/raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -353,10 +353,11 @@ func (r *raft) send(m pb.Message) {
if m.Term != 0 {
panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term))
}
// do not attach term to MsgProp
// do not attach term to MsgProp, MsgReadIndex
// proposals are a way to forward to the leader and
// should be treated as local message.
if m.Type != pb.MsgProp {
// MsgReadIndex is also forwarded to leader.
if m.Type != pb.MsgProp && m.Type != pb.MsgReadIndex {
m.Term = r.Term
}
}
Expand Down