Skip to content

Commit

Permalink
Merge pull request #70 from erikgrinaker/prevote-checkquorum-datadriven
Browse files Browse the repository at this point in the history
add data-driven tests for PreVote and CheckQuorum
  • Loading branch information
ahrtr authored Jun 11, 2023
2 parents a042ce3 + 27dd2c2 commit 30e2fa4
Show file tree
Hide file tree
Showing 10 changed files with 919 additions and 1 deletion.
5 changes: 4 additions & 1 deletion interaction_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

"github.com/cockroachdb/datadriven"

"go.etcd.io/raft/v3"
"go.etcd.io/raft/v3/rafttest"
)

Expand All @@ -27,7 +28,9 @@ func TestInteraction(t *testing.T) {
// diff. Only commit the changes if you understand what caused them and if
// they are desired.
datadriven.Walk(t, "testdata", func(t *testing.T, path string) {
env := rafttest.NewInteractionEnv(nil)
env := rafttest.NewInteractionEnv(&rafttest.InteractionOpts{
SetRandomizedElectionTimeout: raft.SetRandomizedElectionTimeout,
})
datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string {
return env.Handle(t, *d)
})
Expand Down
6 changes: 6 additions & 0 deletions raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5000,6 +5000,12 @@ func setRandomizedElectionTimeout(r *raft, v int) {
r.randomizedElectionTimeout = v
}

// SetRandomizedElectionTimeout is like setRandomizedElectionTimeout, but
// exported for use by tests that are not in the raft package, using RawNode.
func SetRandomizedElectionTimeout(r *RawNode, v int) {
setRandomizedElectionTimeout(r.raft, v)
}

func newTestConfig(id uint64, election, heartbeat int, storage Storage) *Config {
return &Config{
ID: id,
Expand Down
4 changes: 4 additions & 0 deletions rafttest/interaction_env.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,10 @@ import (
// InteractionOpts groups the options for an InteractionEnv.
type InteractionOpts struct {
OnConfig func(*raft.Config)

// SetRandomizedElectionTimeout is used to plumb this function down from the
// raft test package.
SetRandomizedElectionTimeout func(node *raft.RawNode, timeout int)
}

// Node is a member of a raft group tested via an InteractionEnv.
Expand Down
16 changes: 16 additions & 0 deletions rafttest/interaction_env_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,14 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string {
// following, etc.). The information for node n is based on
// n's view.
err = env.handleRaftState()
case "set-randomized-election-timeout":
// Set the randomized election timeout for the given node. Will be reset
// again when the node changes state.
//
// Example:
//
// set-randomized-election-timeout 1 timeout=5
err = env.handleSetRandomizedElectionTimeout(t, d)
case "stabilize":
// Deliver messages to and run process-ready on the set of IDs until
// no more work is to be done. If no ids are given, all nodes are used.
Expand All @@ -112,6 +120,14 @@ func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string {
//
// status 5
err = env.handleStatus(t, d)
case "tick-election":
// Tick an election timeout interval for the given node (but beware the
// randomized timeout).
//
// Example:
//
// tick-election 3
err = env.handleTickElection(t, d)
case "tick-heartbeat":
// Tick a heartbeat interval.
//
Expand Down
4 changes: 4 additions & 0 deletions rafttest/interaction_env_handler_add_nodes.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,10 @@ func (env *InteractionEnv) handleAddNodes(t *testing.T, d datadriven.TestData) e
arg.Scan(t, i, &snap.Data)
case "async-storage-writes":
arg.Scan(t, i, &cfg.AsyncStorageWrites)
case "prevote":
arg.Scan(t, i, &cfg.PreVote)
case "checkquorum":
arg.Scan(t, i, &cfg.CheckQuorum)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// Copyright 2023 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package rafttest

import (
"testing"

"github.com/cockroachdb/datadriven"
"github.com/stretchr/testify/require"
)

func (env *InteractionEnv) handleSetRandomizedElectionTimeout(
t *testing.T, d datadriven.TestData,
) error {
idx := firstAsNodeIdx(t, d)
var timeout int
d.ScanArgs(t, "timeout", &timeout)
require.NotZero(t, timeout)

env.Options.SetRandomizedElectionTimeout(env.Nodes[idx].RawNode, timeout)
return nil
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,11 @@ import (
"github.com/cockroachdb/datadriven"
)

func (env *InteractionEnv) handleTickElection(t *testing.T, d datadriven.TestData) error {
idx := firstAsNodeIdx(t, d)
return env.Tick(idx, env.Nodes[idx].Config.ElectionTick)
}

func (env *InteractionEnv) handleTickHeartbeat(t *testing.T, d datadriven.TestData) error {
idx := firstAsNodeIdx(t, d)
return env.Tick(idx, env.Nodes[idx].Config.HeartbeatTick)
Expand Down
236 changes: 236 additions & 0 deletions testdata/checkquorum.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,236 @@
# Tests that CheckQuorum causes a leader to step down if it hasn't heard from a
# quorum of followers in the past election timeout interval.
#
# Also tests that votes are rejected when there is a current leader. In the Raft
# thesis this is part of PreVote, but etcd/raft enables this via CheckQuorum.

log-level none
----
ok

add-nodes 3 voters=(1,2,3) index=10 checkquorum=true
----
ok

campaign 1
----
ok

stabilize
----
ok (quiet)

log-level debug
----
ok

# Campaigning will fail when there is an active leader.
campaign 2
----
INFO 2 is starting a new election at term 1
INFO 2 became candidate at term 2
INFO 2 [logterm: 1, index: 11] sent MsgVote request to 1 at term 2
INFO 2 [logterm: 1, index: 11] sent MsgVote request to 3 at term 2

stabilize
----
> 2 handling Ready
Ready MustSync=true:
Lead:0 State:StateCandidate
HardState Term:2 Vote:2 Commit:11
Messages:
2->1 MsgVote Term:2 Log:1/11
2->3 MsgVote Term:2 Log:1/11
INFO 2 received MsgVoteResp from 2 at term 2
INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections
> 1 receiving messages
2->1 MsgVote Term:2 Log:1/11
INFO 1 [logterm: 1, index: 11, vote: 1] ignored MsgVote from 2 [logterm: 1, index: 11] at term 1: lease is not expired (remaining ticks: 3)
> 3 receiving messages
2->3 MsgVote Term:2 Log:1/11
INFO 3 [logterm: 1, index: 11, vote: 1] ignored MsgVote from 2 [logterm: 1, index: 11] at term 1: lease is not expired (remaining ticks: 3)

# Tick the leader without processing any messages from followers. We have to
# tick 2 election timeouts, since the followers were active in the current
# interval (see messages above).
tick-election 1
----
ok

tick-election 1
----
WARN 1 stepped down to follower since quorum is not active
INFO 1 became follower at term 1

# We'll now send all of the heartbeats that were buffered during the ticks
# above. Conceptually, "the network was slow".
stabilize
----
> 1 handling Ready
Ready MustSync=false:
Lead:0 State:StateFollower
Messages:
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
> 2 receiving messages
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11
> 3 receiving messages
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11
> 2 handling Ready
Ready MustSync=false:
Messages:
2->1 MsgAppResp Term:2 Log:0/0
2->1 MsgAppResp Term:2 Log:0/0
2->1 MsgAppResp Term:2 Log:0/0
2->1 MsgAppResp Term:2 Log:0/0
2->1 MsgAppResp Term:2 Log:0/0
> 3 handling Ready
Ready MustSync=false:
Messages:
3->1 MsgHeartbeatResp Term:1 Log:0/0
3->1 MsgHeartbeatResp Term:1 Log:0/0
3->1 MsgHeartbeatResp Term:1 Log:0/0
3->1 MsgHeartbeatResp Term:1 Log:0/0
3->1 MsgHeartbeatResp Term:1 Log:0/0
> 1 receiving messages
2->1 MsgAppResp Term:2 Log:0/0
INFO 1 [term: 1] received a MsgAppResp message with higher term from 2 [term: 2]
INFO 1 became follower at term 2
2->1 MsgAppResp Term:2 Log:0/0
2->1 MsgAppResp Term:2 Log:0/0
2->1 MsgAppResp Term:2 Log:0/0
2->1 MsgAppResp Term:2 Log:0/0
3->1 MsgHeartbeatResp Term:1 Log:0/0
INFO 1 [term: 2] ignored a MsgHeartbeatResp message with lower term from 3 [term: 1]
3->1 MsgHeartbeatResp Term:1 Log:0/0
INFO 1 [term: 2] ignored a MsgHeartbeatResp message with lower term from 3 [term: 1]
3->1 MsgHeartbeatResp Term:1 Log:0/0
INFO 1 [term: 2] ignored a MsgHeartbeatResp message with lower term from 3 [term: 1]
3->1 MsgHeartbeatResp Term:1 Log:0/0
INFO 1 [term: 2] ignored a MsgHeartbeatResp message with lower term from 3 [term: 1]
3->1 MsgHeartbeatResp Term:1 Log:0/0
INFO 1 [term: 2] ignored a MsgHeartbeatResp message with lower term from 3 [term: 1]
> 1 handling Ready
Ready MustSync=true:
HardState Term:2 Commit:11

# Other nodes can now successfully campaign. Note that we haven't ticked 3, so
# it won't grant votes.
campaign 2
----
INFO 2 is starting a new election at term 2
INFO 2 became candidate at term 3
INFO 2 [logterm: 1, index: 11] sent MsgVote request to 1 at term 3
INFO 2 [logterm: 1, index: 11] sent MsgVote request to 3 at term 3

process-ready 2
----
Ready MustSync=true:
HardState Term:3 Vote:2 Commit:11
Messages:
2->1 MsgVote Term:3 Log:1/11
2->3 MsgVote Term:3 Log:1/11
INFO 2 received MsgVoteResp from 2 at term 3
INFO 2 has received 1 MsgVoteResp votes and 0 vote rejections

deliver-msgs 1
----
2->1 MsgVote Term:3 Log:1/11
INFO 1 [term: 2] received a MsgVote message with higher term from 2 [term: 3]
INFO 1 became follower at term 3
INFO 1 [logterm: 1, index: 11, vote: 0] cast MsgVote for 2 [logterm: 1, index: 11] at term 3

deliver-msgs 3
----
2->3 MsgVote Term:3 Log:1/11
INFO 3 [logterm: 1, index: 11, vote: 1] ignored MsgVote from 2 [logterm: 1, index: 11] at term 1: lease is not expired (remaining ticks: 3)

stabilize
----
> 1 handling Ready
Ready MustSync=true:
HardState Term:3 Vote:2 Commit:11
Messages:
1->2 MsgVoteResp Term:3 Log:0/0
> 2 receiving messages
1->2 MsgVoteResp Term:3 Log:0/0
INFO 2 received MsgVoteResp from 1 at term 3
INFO 2 has received 2 MsgVoteResp votes and 0 vote rejections
INFO 2 became leader at term 3
> 2 handling Ready
Ready MustSync=true:
Lead:2 State:StateLeader
Entries:
3/12 EntryNormal ""
Messages:
2->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""]
2->3 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""]
> 1 receiving messages
2->1 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""]
> 3 receiving messages
2->3 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""]
INFO 3 [term: 1] received a MsgApp message with higher term from 2 [term: 3]
INFO 3 became follower at term 3
> 1 handling Ready
Ready MustSync=true:
Lead:2 State:StateFollower
Entries:
3/12 EntryNormal ""
Messages:
1->2 MsgAppResp Term:3 Log:0/12
> 3 handling Ready
Ready MustSync=true:
Lead:2 State:StateFollower
HardState Term:3 Commit:11
Entries:
3/12 EntryNormal ""
Messages:
3->2 MsgAppResp Term:3 Log:0/12
> 2 receiving messages
1->2 MsgAppResp Term:3 Log:0/12
3->2 MsgAppResp Term:3 Log:0/12
> 2 handling Ready
Ready MustSync=false:
HardState Term:3 Vote:2 Commit:12
CommittedEntries:
3/12 EntryNormal ""
Messages:
2->1 MsgApp Term:3 Log:3/12 Commit:12
2->3 MsgApp Term:3 Log:3/12 Commit:12
> 1 receiving messages
2->1 MsgApp Term:3 Log:3/12 Commit:12
> 3 receiving messages
2->3 MsgApp Term:3 Log:3/12 Commit:12
> 1 handling Ready
Ready MustSync=false:
HardState Term:3 Vote:2 Commit:12
CommittedEntries:
3/12 EntryNormal ""
Messages:
1->2 MsgAppResp Term:3 Log:0/12
> 3 handling Ready
Ready MustSync=false:
HardState Term:3 Commit:12
CommittedEntries:
3/12 EntryNormal ""
Messages:
3->2 MsgAppResp Term:3 Log:0/12
> 2 receiving messages
1->2 MsgAppResp Term:3 Log:0/12
3->2 MsgAppResp Term:3 Log:0/12
Loading

0 comments on commit 30e2fa4

Please sign in to comment.