Skip to content

Commit 9295740

Browse files
craig[bot]hakuuww
andcommitted
Merge #143710
143710: raft: update TestLeaderSyncFollowerLog r=pav-kv a=hakuuww This TestLeaderSyncFollowerLog test doesn't serve its purpose any more because of how the test framework is setup. (see #143708) Specifically https://github.com/cockroachdb/cockroach/blob/91b58a15c1453f2c6f5f559cfb92029e8af84227/pkg/raft/storage.go#L301-L317 You can see that entries are not in fact appended when trying to create a new node with a init raftLog. This PR aims to fix the issue for this particular test with newly written test cases. **There should be a lot of other places where a similar fix is needed for legacy raft tests.** Epic: none Release note: none Co-authored-by: Anthony Xu <[email protected]>
2 parents d074347 + d5103aa commit 9295740

File tree

1 file changed

+10
-10
lines changed

1 file changed

+10
-10
lines changed

pkg/raft/raft_paper_test.go

Lines changed: 10 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -662,22 +662,22 @@ func TestFollowerAppendEntries(t *testing.T) {
662662
// into consistency with its own.
663663
// Reference: section 5.3, figure 7
664664
func TestLeaderSyncFollowerLog(t *testing.T) {
665-
ents := index(0).terms(0, 1, 1, 1, 4, 4, 5, 5, 6, 6, 6)
666-
term := uint64(8)
665+
ents := index(1).terms(1, 1, 1, 4, 4, 5, 5, 6, 6, 6)
666+
term := uint64(9)
667667
for i, tt := range [][]pb.Entry{
668-
index(0).terms(0, 1, 1, 1, 4, 4, 5, 5, 6, 6),
669-
index(0).terms(0, 1, 1, 1, 4, 4),
670-
index(0).terms(0, 1, 1, 1, 4, 4, 5, 5, 6, 6, 6, 6),
671-
index(0).terms(0, 1, 1, 1, 4, 4, 5, 5, 6, 6, 6, 7, 7),
672-
index(0).terms(0, 1, 1, 1, 4, 4, 4, 4),
673-
index(0).terms(0, 1, 1, 1, 2, 2, 2, 3, 3, 3, 3, 3),
668+
index(1).terms(1, 1, 1, 4, 4, 5, 5, 6, 6),
669+
index(1).terms(1, 1, 1, 4, 4),
670+
index(1).terms(1, 1, 1, 4, 4, 5, 5, 6, 6, 6, 6),
671+
index(1).terms(1, 1, 1, 4, 4, 5, 5, 6, 6, 6, 7, 7),
672+
index(1).terms(1, 1, 1, 4, 4, 4, 4),
673+
index(1).terms(1, 1, 1, 2, 2, 2, 3, 3, 3, 3, 3),
674674
} {
675675
leadStorage := newTestMemoryStorage(withPeers(1, 2, 3))
676-
leadStorage.Append(ents)
676+
require.NoError(t, leadStorage.Append(ents))
677677
lead := newTestRaft(1, 10, 1, leadStorage)
678678
lead.loadState(pb.HardState{Commit: lead.raftLog.lastIndex(), Term: term})
679679
followerStorage := newTestMemoryStorage(withPeers(1, 2, 3))
680-
followerStorage.Append(tt)
680+
require.NoError(t, followerStorage.Append(tt))
681681
follower := newTestRaft(2, 10, 1, followerStorage)
682682
follower.loadState(pb.HardState{Term: term - 1})
683683
// It is necessary to have a three-node cluster.

0 commit comments

Comments
 (0)