Skip to content

Commit

Permalink
raft: enable store liveness in raft unit tests phase 1
Browse files Browse the repository at this point in the history
This commit modifies the following tests to allow them to run in
both store liveness enabled/disabled configurations:

1) TestCommitWithoutNewTermEntry
2) TestConfChangeCheckBeforeCampaign
3) TestConfChangeV2CheckBeforeCampaign
4) TestLeaderCycle
5) TestLeaderCyclePreVote
6) TestLeaderTransferReceiveHigherTermVote
7) TestLogReplication
8) TestOldMessages

References: #132241

Release note: None
  • Loading branch information
iskettaneh committed Nov 1, 2024
1 parent a39ae7d commit 79de58f
Showing 1 changed file with 168 additions and 21 deletions.
189 changes: 168 additions & 21 deletions pkg/raft/raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -413,26 +413,61 @@ func TestLearnerCanVote(t *testing.T) {
}

func TestLeaderCycle(t *testing.T) {
testLeaderCycle(t, false)
testutils.RunTrueAndFalse(t, "store-liveness-enabled",
func(t *testing.T, storeLivenessEnabled bool) {
testLeaderCycle(t, false, storeLivenessEnabled)
})
}

func TestLeaderCyclePreVote(t *testing.T) {
testLeaderCycle(t, true)
testutils.RunTrueAndFalse(t, "store-liveness-enabled",
func(t *testing.T, storeLivenessEnabled bool) {
testLeaderCycle(t, true, storeLivenessEnabled)
})
}

// testLeaderCycle verifies that each node in a cluster can campaign
// and be elected in turn. This ensures that elections (including
// pre-vote) work when not starting from a clean slate (as they do in
// TestLeaderElection)
func testLeaderCycle(t *testing.T, preVote bool) {
cfg := fortificationDisabledConfig
func testLeaderCycle(t *testing.T, preVote bool, storeLivenessEnabled bool) {
var cfg func(c *Config) = nil

if preVote {
cfg = preVoteConfigWithFortificationDisabled
}

if preVote && storeLivenessEnabled {
cfg = preVoteConfig
} else if preVote && !storeLivenessEnabled {
cfg = preVoteConfigWithFortificationDisabled
} else if !preVote && storeLivenessEnabled {
// The default configuration satisfies this condition.
} else if !preVote && !storeLivenessEnabled {
cfg = fortificationDisabledConfig
}

n := newNetworkWithConfig(cfg, nil, nil, nil)
n.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
curLeader := pb.PeerID(1)

for campaignerID := pb.PeerID(1); campaignerID <= 3; campaignerID++ {
if storeLivenessEnabled {
// We need to withdraw support of the current leader to allow the new peer
// to campaign and get elected.
n.livenessFabric.WithdrawSupportForPeerFromAllPeers(curLeader)
}

n.send(pb.Message{From: campaignerID, To: campaignerID, Type: pb.MsgHup})

if storeLivenessEnabled {
// Restore the support state.
n.livenessFabric.GrantSupportForPeerFromAllPeers(curLeader)
}

// Update the current leader to prep for the next iteration.
curLeader = campaignerID

for _, peer := range n.peers {
sm := peer.(*raft)
if sm.id == campaignerID {
Expand Down Expand Up @@ -568,21 +603,34 @@ func testVoteFromAnyState(t *testing.T, vt pb.MessageType) {
}
}

// TestLogReplication tests that the normal replication flow works.
func TestLogReplication(t *testing.T) {
testutils.RunTrueAndFalse(t, "store-liveness-enabled",
func(t *testing.T, storeLivenessEnabled bool) {
testLogReplication(t, storeLivenessEnabled)
})
}

func testLogReplication(t *testing.T, storeLivenessEnabled bool) {
var cfg func(c *Config) = nil
if !storeLivenessEnabled {
cfg = fortificationDisabledConfig
}

tests := []struct {
*network
msgs []pb.Message
wcommitted uint64
}{
{
newNetworkWithConfig(fortificationDisabledConfig, nil, nil, nil),
newNetworkWithConfig(cfg, nil, nil, nil),
[]pb.Message{
{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}},
},
2,
},
{
newNetworkWithConfig(fortificationDisabledConfig, nil, nil, nil),
newNetworkWithConfig(cfg, nil, nil, nil),
[]pb.Message{
{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}},
{From: 1, To: 2, Type: pb.MsgHup},
Expand All @@ -596,7 +644,18 @@ func TestLogReplication(t *testing.T) {
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})

for _, m := range tt.msgs {
if m.Type == pb.MsgHup && storeLivenessEnabled {
// We need to withdraw support of the current leader to allow the new peer
// to campaign and get elected.
tt.livenessFabric.WithdrawSupportForPeerFromAllPeers(1)
}

tt.send(m)

if m.Type == pb.MsgHup && storeLivenessEnabled {
// Restore the support state.
tt.livenessFabric.GrantSupportForPeerFromAllPeers(1)
}
}

for j, x := range tt.network.peers {
Expand Down Expand Up @@ -709,7 +768,8 @@ func testCannotCommitWithoutNewTermEntry(t *testing.T, storeLivenessEnabled bool

// Elect 2 as the new leader with term 2.
if storeLivenessEnabled {
// We need to withdraw support of the current leader.
// We need to withdraw support of the current leader to allow the new peer
// to campaign and get elected.
tt.livenessFabric.WithdrawSupportForPeerFromAllPeers(1)
}

Expand All @@ -733,10 +793,22 @@ func testCannotCommitWithoutNewTermEntry(t *testing.T, storeLivenessEnabled bool
assert.Equal(t, uint64(5), sm.raftLog.committed)
}

// TestCommitWithoutNewTermEntry tests the entries could be committed
// when leader changes, no new proposal comes in.
// TestCommitWithoutNewTermEntry tests the entries could be committed when
// leader changes, no new proposal comes in.
func TestCommitWithoutNewTermEntry(t *testing.T) {
tt := newNetworkWithConfig(fortificationDisabledConfig, nil, nil, nil, nil, nil)
testutils.RunTrueAndFalse(t, "store-liveness-enabled",
func(t *testing.T, storeLivenessEnabled bool) {
testCommitWithoutNewTermEntry(t, storeLivenessEnabled)
})
}

func testCommitWithoutNewTermEntry(t *testing.T, storeLivenessEnabled bool) {
var cfg func(c *Config) = nil
if !storeLivenessEnabled {
cfg = fortificationDisabledConfig
}

tt := newNetworkWithConfig(cfg, nil, nil, nil, nil, nil)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})

// 0 cannot reach 3,4,5
Expand All @@ -756,8 +828,20 @@ func TestCommitWithoutNewTermEntry(t *testing.T) {
// elect 2 as the new leader with term 2
// after append a ChangeTerm entry from the current term, all entries
// should be committed

if storeLivenessEnabled {
// We need to withdraw support of the current leader to allow the new peer
// to campaign and get elected.
tt.livenessFabric.WithdrawSupportForPeerFromAllPeers(1)
}

tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})

if storeLivenessEnabled {
// Restore the support state.
tt.livenessFabric.GrantSupportForPeerFromAllPeers(1)
}

assert.Equal(t, uint64(4), sm.raftLog.committed)
}

Expand Down Expand Up @@ -920,11 +1004,37 @@ func TestSingleNodePreCandidate(t *testing.T) {
}

func TestOldMessages(t *testing.T) {
tt := newNetworkWithConfig(fortificationDisabledConfig, nil, nil, nil)
testutils.RunTrueAndFalse(t, "store-liveness-enabled",
func(t *testing.T, storeLivenessEnabled bool) {
testOldMessages(t, storeLivenessEnabled)
})
}

func testOldMessages(t *testing.T, storeLivenessEnabled bool) {
var cfg func(c *Config) = nil
if !storeLivenessEnabled {
cfg = fortificationDisabledConfig
}

tt := newNetworkWithConfig(cfg, nil, nil, nil)
// make 0 leader @ term 3
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})

if storeLivenessEnabled {
// We need to withdraw support of the current leader to allow the new peer
// to campaign and get elected.
tt.livenessFabric.WithdrawSupportForPeerFromAllPeers(1)
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
tt.livenessFabric.GrantSupportForPeerFromAllPeers(1)

tt.livenessFabric.WithdrawSupportForPeerFromAllPeers(2)
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
tt.livenessFabric.GrantSupportForPeerFromAllPeers(2)
} else {
tt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
tt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
}

// pretend we're an old leader trying to make progress; this entry is expected to be ignored.
tt.send(pb.Message{From: 2, To: 1, Type: pb.MsgApp, Term: 2, Entries: index(3).terms(2)})
// commit a new entry
Expand Down Expand Up @@ -3281,7 +3391,19 @@ func TestLeaderTransferIgnoreProposal(t *testing.T) {
}

func TestLeaderTransferReceiveHigherTermVote(t *testing.T) {
nt := newNetworkWithConfig(fortificationDisabledConfig, nil, nil, nil)
testutils.RunTrueAndFalse(t, "store-liveness-enabled",
func(t *testing.T, storeLivenessEnabled bool) {
testLeaderTransferReceiveHigherTermVote(t, storeLivenessEnabled)
})
}

func testLeaderTransferReceiveHigherTermVote(t *testing.T, storeLivenessEnabled bool) {
var cfg func(c *Config) = nil
if !storeLivenessEnabled {
cfg = fortificationDisabledConfig
}

nt := newNetworkWithConfig(cfg, nil, nil, nil)
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})

// Isolate node 3 and propose an entry on 1. This will cause node 3 to fall
Expand All @@ -3297,6 +3419,12 @@ func TestLeaderTransferReceiveHigherTermVote(t *testing.T) {
nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
require.Equal(t, pb.PeerID(3), lead.leadTransferee)

if storeLivenessEnabled {
// We need to withdraw support of the current leader to allow the new peer
// to campaign and get elected.
nt.livenessFabric.WithdrawSupportForPeerFromAllPeers(1)
}

nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup, Index: 1, Term: 2})

checkLeaderTransferState(t, lead, pb.StateFollower, 2)
Expand Down Expand Up @@ -3860,8 +3988,13 @@ func TestPreVoteMigrationWithFreeStuckPreCandidate(t *testing.T) {

}

func testConfChangeCheckBeforeCampaign(t *testing.T, v2 bool) {
nt := newNetworkWithConfig(fortificationDisabledConfig, nil, nil, nil)
func testConfChangeCheckBeforeCampaign(t *testing.T, v2 bool, storeLivenessEnabled bool) {
var cfg func(c *Config) = nil
if !storeLivenessEnabled {
cfg = fortificationDisabledConfig
}

nt := newNetworkWithConfig(cfg, nil, nil, nil)
n1 := nt.peers[1].(*raft)
n2 := nt.peers[2].(*raft)
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
Expand Down Expand Up @@ -3893,6 +4026,12 @@ func testConfChangeCheckBeforeCampaign(t *testing.T, v2 bool) {
},
})

if storeLivenessEnabled {
// We need to withdraw support of the current leader to allow the new peer
// to campaign and get elected.
nt.livenessFabric.WithdrawSupportForPeerFromAllPeers(1)
}

// Trigger campaign in node 2
for i := int64(0); i < n2.randomizedElectionTimeout; i++ {
n2.tick()
Expand All @@ -3908,9 +4047,10 @@ func testConfChangeCheckBeforeCampaign(t *testing.T, v2 bool) {
// conf change is not applied.
assert.Equal(t, pb.StateFollower, n2.state)

// TODO(arul): a leader that steps down will currently never campaign due to
// the fortification promise that it made to itself. We'll need to fix this.
n1.deFortify(n1.lead, n1.Term)
if storeLivenessEnabled {
// Restore the support state.
nt.livenessFabric.GrantSupportForPeerFromAllPeers(1)
}

// Advance apply on node 1 and re-establish leadership.
nextEnts(n1, nt.storage[1])
Expand All @@ -3925,6 +4065,7 @@ func testConfChangeCheckBeforeCampaign(t *testing.T, v2 bool) {

// Transfer leadership to peer 2 again.
nt.send(pb.Message{From: 2, To: 1, Type: pb.MsgTransferLeader})

// The outgoing leader steps down immediately.
assert.Equal(t, pb.StateFollower, n1.state)
// The transfer target campaigns immediately now that the committed conf
Expand All @@ -3934,12 +4075,18 @@ func testConfChangeCheckBeforeCampaign(t *testing.T, v2 bool) {

// TestConfChangeCheckBeforeCampaign tests if unapplied ConfChange is checked before campaign.
func TestConfChangeCheckBeforeCampaign(t *testing.T) {
testConfChangeCheckBeforeCampaign(t, false)
testutils.RunTrueAndFalse(t, "store-liveness-enabled",
func(t *testing.T, storeLivenessEnabled bool) {
testConfChangeCheckBeforeCampaign(t, false, storeLivenessEnabled)
})
}

// TestConfChangeV2CheckBeforeCampaign tests if unapplied ConfChangeV2 is checked before campaign.
func TestConfChangeV2CheckBeforeCampaign(t *testing.T) {
testConfChangeCheckBeforeCampaign(t, true)
testutils.RunTrueAndFalse(t, "store-liveness-enabled",
func(t *testing.T, storeLivenessEnabled bool) {
testConfChangeCheckBeforeCampaign(t, true, storeLivenessEnabled)
})
}

func TestFastLogRejection(t *testing.T) {
Expand Down

0 comments on commit 79de58f

Please sign in to comment.