Browse Source

raft: attach Index to Entry in all tests

Xiang Li 11 years ago
parent
commit
65ad1f6ffd
3 changed files with 48 additions and 48 deletions
  1. 40 40
      raft/log_test.go
  2. 1 1
      raft/raft_paper_test.go
  3. 7 7
      raft/raft_test.go

+ 40 - 40
raft/log_test.go

@@ -24,7 +24,7 @@ import (
 )
 )
 
 
 func TestFindConflict(t *testing.T) {
 func TestFindConflict(t *testing.T) {
-	previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}
+	previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
 	tests := []struct {
 	tests := []struct {
 		from      uint64
 		from      uint64
 		ents      []pb.Entry
 		ents      []pb.Entry
@@ -34,18 +34,18 @@ func TestFindConflict(t *testing.T) {
 		{1, []pb.Entry{}, 0},
 		{1, []pb.Entry{}, 0},
 		{3, []pb.Entry{}, 0},
 		{3, []pb.Entry{}, 0},
 		// no conflict
 		// no conflict
-		{1, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}, 0},
-		{2, []pb.Entry{{Term: 2}, {Term: 3}}, 0},
-		{3, []pb.Entry{{Term: 3}}, 0},
+		{1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0},
+		{2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0},
+		{3, []pb.Entry{{Index: 3, Term: 3}}, 0},
 		// no conflict, but has new entries
 		// no conflict, but has new entries
-		{1, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}, {Term: 4}, {Term: 4}}, 4},
-		{2, []pb.Entry{{Term: 2}, {Term: 3}, {Term: 4}, {Term: 4}}, 4},
-		{3, []pb.Entry{{Term: 3}, {Term: 4}, {Term: 4}}, 4},
-		{4, []pb.Entry{{Term: 4}, {Term: 4}}, 4},
+		{1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
+		{2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
+		{3, []pb.Entry{{Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
+		{4, []pb.Entry{{Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
 		// conflicts with existing entries
 		// conflicts with existing entries
-		{1, []pb.Entry{{Term: 4}, {Term: 4}}, 1},
-		{2, []pb.Entry{{Term: 1}, {Term: 4}, {Term: 4}}, 2},
-		{3, []pb.Entry{{Term: 1}, {Term: 2}, {Term: 4}, {Term: 4}}, 3},
+		{1, []pb.Entry{{Index: 1, Term: 4}, {Index: 2, Term: 4}}, 1},
+		{2, []pb.Entry{{Index: 2, Term: 1}, {Index: 3, Term: 4}, {Index: 4, Term: 4}}, 2},
+		{3, []pb.Entry{{Index: 3, Term: 1}, {Index: 4, Term: 2}, {Index: 5, Term: 4}, {Index: 6, Term: 4}}, 3},
 	}
 	}
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
@@ -60,7 +60,7 @@ func TestFindConflict(t *testing.T) {
 }
 }
 
 
 func TestIsUpToDate(t *testing.T) {
 func TestIsUpToDate(t *testing.T) {
-	previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}
+	previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
 	raftLog := newLog(NewMemoryStorage())
 	raftLog := newLog(NewMemoryStorage())
 	raftLog.append(raftLog.lastIndex(), previousEnts...)
 	raftLog.append(raftLog.lastIndex(), previousEnts...)
 	tests := []struct {
 	tests := []struct {
@@ -91,7 +91,7 @@ func TestIsUpToDate(t *testing.T) {
 }
 }
 
 
 func TestAppend(t *testing.T) {
 func TestAppend(t *testing.T) {
-	previousEnts := []pb.Entry{{Term: 1}, {Term: 2}}
+	previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}
 	tests := []struct {
 	tests := []struct {
 		after     uint64
 		after     uint64
 		ents      []pb.Entry
 		ents      []pb.Entry
@@ -103,30 +103,30 @@ func TestAppend(t *testing.T) {
 			2,
 			2,
 			[]pb.Entry{},
 			[]pb.Entry{},
 			2,
 			2,
-			[]pb.Entry{{Term: 1}, {Term: 2}},
+			[]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}},
 			3,
 			3,
 		},
 		},
 		{
 		{
 			2,
 			2,
-			[]pb.Entry{{Term: 2}},
+			[]pb.Entry{{Index: 3, Term: 2}},
 			3,
 			3,
-			[]pb.Entry{{Term: 1}, {Term: 2}, {Term: 2}},
+			[]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 2}},
 			3,
 			3,
 		},
 		},
 		// conflicts with index 1
 		// conflicts with index 1
 		{
 		{
 			0,
 			0,
-			[]pb.Entry{{Term: 2}},
+			[]pb.Entry{{Index: 1, Term: 2}},
 			1,
 			1,
-			[]pb.Entry{{Term: 2}},
+			[]pb.Entry{{Index: 1, Term: 2}},
 			1,
 			1,
 		},
 		},
 		// conflicts with index 2
 		// conflicts with index 2
 		{
 		{
 			1,
 			1,
-			[]pb.Entry{{Term: 3}, {Term: 3}},
+			[]pb.Entry{{Index: 2, Term: 3}, {Index: 3, Term: 3}},
 			3,
 			3,
-			[]pb.Entry{{Term: 1}, {Term: 3}, {Term: 3}},
+			[]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 3}, {Index: 3, Term: 3}},
 			2,
 			2,
 		},
 		},
 	}
 	}
@@ -158,7 +158,7 @@ func TestAppend(t *testing.T) {
 // If the given (index, term) does not match with the existing log:
 // If the given (index, term) does not match with the existing log:
 // 	return false
 // 	return false
 func TestLogMaybeAppend(t *testing.T) {
 func TestLogMaybeAppend(t *testing.T) {
-	previousEnts := []pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}}
+	previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
 	lastindex := uint64(3)
 	lastindex := uint64(3)
 	lastterm := uint64(3)
 	lastterm := uint64(3)
 	commit := uint64(1)
 	commit := uint64(1)
@@ -176,12 +176,12 @@ func TestLogMaybeAppend(t *testing.T) {
 	}{
 	}{
 		// not match: term is different
 		// not match: term is different
 		{
 		{
-			lastterm - 1, lastindex, lastindex, []pb.Entry{{Term: 4}},
+			lastterm - 1, lastindex, lastindex, []pb.Entry{{Index: lastindex + 1, Term: 4}},
 			0, false, commit, false,
 			0, false, commit, false,
 		},
 		},
 		// not match: index out of bound
 		// not match: index out of bound
 		{
 		{
-			lastterm, lastindex + 1, lastindex, []pb.Entry{{Term: 4}},
+			lastterm, lastindex + 1, lastindex, []pb.Entry{{Index: lastindex + 2, Term: 4}},
 			0, false, commit, false,
 			0, false, commit, false,
 		},
 		},
 		// match with the last existing entry
 		// match with the last existing entry
@@ -206,36 +206,36 @@ func TestLogMaybeAppend(t *testing.T) {
 			0, true, commit, false, // commit do not decrease
 			0, true, commit, false, // commit do not decrease
 		},
 		},
 		{
 		{
-			lastterm, lastindex, lastindex, []pb.Entry{{Term: 4}},
+			lastterm, lastindex, lastindex, []pb.Entry{{Index: lastindex + 1, Term: 4}},
 			lastindex + 1, true, lastindex, false,
 			lastindex + 1, true, lastindex, false,
 		},
 		},
 		{
 		{
-			lastterm, lastindex, lastindex + 1, []pb.Entry{{Term: 4}},
+			lastterm, lastindex, lastindex + 1, []pb.Entry{{Index: lastindex + 1, Term: 4}},
 			lastindex + 1, true, lastindex + 1, false,
 			lastindex + 1, true, lastindex + 1, false,
 		},
 		},
 		{
 		{
-			lastterm, lastindex, lastindex + 2, []pb.Entry{{Term: 4}},
+			lastterm, lastindex, lastindex + 2, []pb.Entry{{Index: lastindex + 1, Term: 4}},
 			lastindex + 1, true, lastindex + 1, false, // do not increase commit higher than lastnewi
 			lastindex + 1, true, lastindex + 1, false, // do not increase commit higher than lastnewi
 		},
 		},
 		{
 		{
-			lastterm, lastindex, lastindex + 2, []pb.Entry{{Term: 4}, {Term: 4}},
+			lastterm, lastindex, lastindex + 2, []pb.Entry{{Index: lastindex + 1, Term: 4}, {Index: lastindex + 2, Term: 4}},
 			lastindex + 2, true, lastindex + 2, false,
 			lastindex + 2, true, lastindex + 2, false,
 		},
 		},
 		// match with the the entry in the middle
 		// match with the the entry in the middle
 		{
 		{
-			lastterm - 1, lastindex - 1, lastindex, []pb.Entry{{Term: 4}},
+			lastterm - 1, lastindex - 1, lastindex, []pb.Entry{{Index: lastindex, Term: 4}},
 			lastindex, true, lastindex, false,
 			lastindex, true, lastindex, false,
 		},
 		},
 		{
 		{
-			lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Term: 4}},
+			lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Index: lastindex - 1, Term: 4}},
 			lastindex - 1, true, lastindex - 1, false,
 			lastindex - 1, true, lastindex - 1, false,
 		},
 		},
 		{
 		{
-			lastterm - 3, lastindex - 3, lastindex, []pb.Entry{{Term: 4}},
+			lastterm - 3, lastindex - 3, lastindex, []pb.Entry{{Index: lastindex - 2, Term: 4}},
 			lastindex - 2, true, lastindex - 2, true, // conflict with existing committed entry
 			lastindex - 2, true, lastindex - 2, true, // conflict with existing committed entry
 		},
 		},
 		{
 		{
-			lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Term: 4}, {Term: 4}},
+			lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Index: lastindex - 1, Term: 4}, {Index: lastindex, Term: 4}},
 			lastindex, true, lastindex, false,
 			lastindex, true, lastindex, false,
 		},
 		},
 	}
 	}
@@ -325,7 +325,7 @@ func TestCompactionSideEffects(t *testing.T) {
 	}
 	}
 
 
 	prev := raftLog.lastIndex()
 	prev := raftLog.lastIndex()
-	raftLog.append(raftLog.lastIndex(), pb.Entry{Term: raftLog.lastIndex() + 1})
+	raftLog.append(raftLog.lastIndex(), pb.Entry{Index: raftLog.lastIndex() + 1, Term: raftLog.lastIndex() + 1})
 	if raftLog.lastIndex() != prev+1 {
 	if raftLog.lastIndex() != prev+1 {
 		t.Errorf("lastIndex = %d, want = %d", raftLog.lastIndex(), prev+1)
 		t.Errorf("lastIndex = %d, want = %d", raftLog.lastIndex(), prev+1)
 	}
 	}
@@ -481,7 +481,7 @@ func TestCompaction(t *testing.T) {
 
 
 			storage := NewMemoryStorage()
 			storage := NewMemoryStorage()
 			for i := uint64(1); i <= tt.lastIndex; i++ {
 			for i := uint64(1); i <= tt.lastIndex; i++ {
-				storage.Append([]pb.Entry{{}})
+				storage.Append([]pb.Entry{{Index: i}})
 			}
 			}
 			raftLog := newLog(storage)
 			raftLog := newLog(storage)
 			raftLog.maybeCommit(tt.lastIndex, 0)
 			raftLog.maybeCommit(tt.lastIndex, 0)
@@ -558,7 +558,7 @@ func TestAt(t *testing.T) {
 	storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
 	storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
 	l := newLog(storage)
 	l := newLog(storage)
 	for i = 1; i < num; i++ {
 	for i = 1; i < num; i++ {
-		l.append(offset+i-1, pb.Entry{Term: i})
+		l.append(offset+i-1, pb.Entry{Index: i, Term: i})
 	}
 	}
 
 
 	tests := []struct {
 	tests := []struct {
@@ -567,8 +567,8 @@ func TestAt(t *testing.T) {
 	}{
 	}{
 		{offset - 1, nil},
 		{offset - 1, nil},
 		{offset, nil},
 		{offset, nil},
-		{offset + num/2, &pb.Entry{Term: num / 2}},
-		{offset + num - 1, &pb.Entry{Term: num - 1}},
+		{offset + num/2, &pb.Entry{Index: num / 2, Term: num / 2}},
+		{offset + num - 1, &pb.Entry{Index: num - 1, Term: num - 1}},
 		{offset + num, nil},
 		{offset + num, nil},
 	}
 	}
 
 
@@ -589,7 +589,7 @@ func TestTerm(t *testing.T) {
 	storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
 	storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
 	l := newLog(storage)
 	l := newLog(storage)
 	for i = 1; i < num; i++ {
 	for i = 1; i < num; i++ {
-		l.append(offset+i-1, pb.Entry{Term: i})
+		l.append(offset+i-1, pb.Entry{Index: i, Term: i})
 	}
 	}
 
 
 	tests := []struct {
 	tests := []struct {
@@ -620,7 +620,7 @@ func TestSlice(t *testing.T) {
 	storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
 	storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
 	l := newLog(storage)
 	l := newLog(storage)
 	for i = 1; i < num; i++ {
 	for i = 1; i < num; i++ {
-		l.append(offset+i-1, pb.Entry{Term: i})
+		l.append(offset+i-1, pb.Entry{Index: i, Term: i})
 	}
 	}
 
 
 	tests := []struct {
 	tests := []struct {
@@ -630,8 +630,8 @@ func TestSlice(t *testing.T) {
 	}{
 	}{
 		{offset - 1, offset + 1, nil},
 		{offset - 1, offset + 1, nil},
 		{offset, offset + 1, nil},
 		{offset, offset + 1, nil},
-		{offset + num/2, offset + num/2 + 1, []pb.Entry{{Term: num / 2}}},
-		{offset + num - 1, offset + num, []pb.Entry{{Term: num - 1}}},
+		{offset + num/2, offset + num/2 + 1, []pb.Entry{{Index: num / 2, Term: num / 2}}},
+		{offset + num - 1, offset + num, []pb.Entry{{Index: num - 1, Term: num - 1}}},
 		{offset + num, offset + num + 1, nil},
 		{offset + num, offset + num + 1, nil},
 
 
 		{offset + num/2, offset + num/2, nil},
 		{offset + num/2, offset + num/2, nil},

+ 1 - 1
raft/raft_paper_test.go

@@ -113,7 +113,7 @@ func TestLeaderBcastBeat(t *testing.T) {
 	r.becomeCandidate()
 	r.becomeCandidate()
 	r.becomeLeader()
 	r.becomeLeader()
 	for i := 0; i < 10; i++ {
 	for i := 0; i < 10; i++ {
-		r.appendEntry(pb.Entry{})
+		r.appendEntry(pb.Entry{Index: uint64(i) + 1})
 	}
 	}
 
 
 	for i := 0; i <= hi; i++ {
 	for i := 0; i <= hi; i++ {

+ 7 - 7
raft/raft_test.go

@@ -715,7 +715,7 @@ func TestHandleHeartbeat(t *testing.T) {
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
 		storage := NewMemoryStorage()
 		storage := NewMemoryStorage()
-		storage.Append([]pb.Entry{{Term: 1}, {Term: 2}, {Term: 3}})
+		storage.Append([]pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}})
 		sm := &raft{
 		sm := &raft{
 			state:     StateFollower,
 			state:     StateFollower,
 			HardState: pb.HardState{Term: 2},
 			HardState: pb.HardState{Term: 2},
@@ -780,7 +780,7 @@ func TestRecvMsgVote(t *testing.T) {
 		}
 		}
 		sm.HardState = pb.HardState{Vote: tt.voteFor}
 		sm.HardState = pb.HardState{Vote: tt.voteFor}
 		sm.raftLog = &raftLog{
 		sm.raftLog = &raftLog{
-			storage:  &MemoryStorage{ents: []pb.Entry{{}, {Term: 2}, {Term: 2}}},
+			storage:  &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 2}, {Index: 2, Term: 2}}},
 			unstable: 3,
 			unstable: 3,
 		}
 		}
 
 
@@ -928,7 +928,7 @@ func TestLeaderAppResp(t *testing.T) {
 		// thus the last log term must be 1 to be committed.
 		// thus the last log term must be 1 to be committed.
 		sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
 		sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
 		sm.raftLog = &raftLog{
 		sm.raftLog = &raftLog{
-			storage:  &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}},
+			storage:  &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}},
 			unstable: 3,
 			unstable: 3,
 		}
 		}
 		sm.becomeCandidate()
 		sm.becomeCandidate()
@@ -980,7 +980,7 @@ func TestBcastBeat(t *testing.T) {
 	sm.becomeCandidate()
 	sm.becomeCandidate()
 	sm.becomeLeader()
 	sm.becomeLeader()
 	for i := 0; i < 10; i++ {
 	for i := 0; i < 10; i++ {
-		sm.appendEntry(pb.Entry{})
+		sm.appendEntry(pb.Entry{Index: uint64(i) + 1})
 	}
 	}
 	// slow follower
 	// slow follower
 	sm.prs[2].match, sm.prs[2].next = 5, 6
 	sm.prs[2].match, sm.prs[2].next = 5, 6
@@ -1034,7 +1034,7 @@ func TestRecvMsgBeat(t *testing.T) {
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
 		sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
 		sm := newRaft(1, []uint64{1, 2, 3}, 10, 1, NewMemoryStorage())
-		sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Term: 0}, {Term: 1}}}}
+		sm.raftLog = &raftLog{storage: &MemoryStorage{ents: []pb.Entry{{}, {Index: 1, Term: 0}, {Index: 2, Term: 1}}}}
 		sm.Term = 1
 		sm.Term = 1
 		sm.state = tt.state
 		sm.state = tt.state
 		switch tt.state {
 		switch tt.state {
@@ -1344,8 +1344,8 @@ func TestRaftNodes(t *testing.T) {
 
 
 func ents(terms ...uint64) *raft {
 func ents(terms ...uint64) *raft {
 	ents := []pb.Entry{{}}
 	ents := []pb.Entry{{}}
-	for _, term := range terms {
-		ents = append(ents, pb.Entry{Term: term})
+	for i, term := range terms {
+		ents = append(ents, pb.Entry{Index: uint64(i), Term: term})
 	}
 	}
 
 
 	sm := &raft{
 	sm := &raft{