Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
132860: roachtest: deflake Test_updateSpecForSelectiveTests r=nameisbhaskar,herkolategan a=DarrylWong

This change reduces the amount of iterations the randomized test takes from 1000 to 100, as each iteration spins up a goroutine that can lead to OOM issues. On race builds, it further decreases it to 10.

It also switches the tests to log using t.Log instead of logging to stdout.

Fixes: #132745
Release note: none
Epic: none

133490: raft: don't ignore MsgTimeoutNow as candidate r=nvanbenschoten a=nvanbenschoten

This commit updates `stepCandidate` to not ignore `MsgTimeoutNow` messages from the leader. These messages not only teach the candidate who the leader is, but they also allow the candidate to call a new election at the _next_ term. Furthermore, they allow the candidate to perform a "force" election. So we don't want to drop these messages if at all possible.

This is a pretty rare case because a leader will only transfer leadership to a peer that it knows has an up-to-date log, but it is possible, as demonstrated by the test.

Epic: None
Release note: None

Co-authored-by: DarrylWong <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
3 people committed Oct 28, 2024
3 parents c184e6d + 0390330 + b44df2f commit 7a9a01b
Show file tree
Hide file tree
Showing 5 changed files with 58 additions and 11 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,7 @@ go_test(
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/echotest",
"//pkg/util",
"//pkg/util/randutil",
"//pkg/util/stop",
"//pkg/util/syncutil",
Expand Down
14 changes: 9 additions & 5 deletions pkg/cmd/roachtest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,9 @@ func testsToRun(
// the test categorization must be complete in 30 seconds
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
updateSpecForSelectiveTests(ctx, specs)
updateSpecForSelectiveTests(ctx, specs, func(format string, args ...interface{}) {
fmt.Fprintf(os.Stdout, format, args...)
})
}

var notSkipped []registry.TestSpec
Expand Down Expand Up @@ -308,12 +310,14 @@ func testsToRun(
// the successful tests.
// 4. The tests that meet the 35% criteria, are marked as "selected=true"
// 5. All tests that are marked "selected=true" are considered for the test run.
func updateSpecForSelectiveTests(ctx context.Context, specs []registry.TestSpec) {
func updateSpecForSelectiveTests(
ctx context.Context, specs []registry.TestSpec, logFunc func(format string, args ...interface{}),
) {
selectedTestsCount := 0
allTests, err := testselector.CategoriseTests(ctx,
testselector.NewDefaultSelectTestsReq(roachtestflags.Cloud, roachtestflags.Suite))
if err != nil {
fmt.Printf("running all tests! error selecting tests: %v\n", err)
logFunc("running all tests! error selecting tests: %v\n", err)
return
}

Expand Down Expand Up @@ -350,7 +354,7 @@ func updateSpecForSelectiveTests(ctx context.Context, specs []registry.TestSpec)
for i := 0; i < numberOfTestsToSelect; i++ {
successfulTests[i].Selected = true
}
fmt.Printf("%d selected out of %d successful tests.\n", numberOfTestsToSelect, len(successfulTests))
logFunc("%d selected out of %d successful tests.\n", numberOfTestsToSelect, len(successfulTests))
for i := range specs {
if testShouldBeSkipped(allTestsMap, specs[i], roachtestflags.Suite) {
if specs[i].Skip == "" {
Expand All @@ -366,7 +370,7 @@ func updateSpecForSelectiveTests(ctx context.Context, specs []registry.TestSpec)
specs[i].SetStats(td.AvgDurationInMillis, td.LastFailureIsPreempt)
}
}
fmt.Printf("%d out of %d tests selected for the run!\n", selectedTestsCount, len(specs))
logFunc("%d out of %d tests selected for the run!\n", selectedTestsCount, len(specs))
}

// testShouldBeSkipped decides whether a test should be skipped based on test details and suite
Expand Down
23 changes: 18 additions & 5 deletions pkg/cmd/roachtest/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/testselector"
"github.com/cockroachdb/cockroach/pkg/internal/team"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/stretchr/testify/assert"
Expand Down Expand Up @@ -124,7 +125,9 @@ func Test_updateSpecForSelectiveTests(t *testing.T) {
require.Nil(t, err)
specs, _ := getTestSelectionMockData()
mock.ExpectPrepare(regexp.QuoteMeta(testselector.PreparedQuery)).WillReturnError(fmt.Errorf("failed to prepare"))
updateSpecForSelectiveTests(ctx, specs)
updateSpecForSelectiveTests(ctx, specs, func(format string, args ...interface{}) {
t.Logf(format, args...)
})
for _, s := range specs {
if !strings.Contains(s.Name, "skipped") {
require.Empty(t, s.Skip)
Expand All @@ -150,7 +153,9 @@ func Test_updateSpecForSelectiveTests(t *testing.T) {
mock.ExpectPrepare(regexp.QuoteMeta(testselector.PreparedQuery))
mock.ExpectQuery(regexp.QuoteMeta(testselector.PreparedQuery)).WillReturnRows(rows)
specsLengthBefore := len(specs)
updateSpecForSelectiveTests(ctx, specs)
updateSpecForSelectiveTests(ctx, specs, func(format string, args ...interface{}) {
t.Logf(format, args...)
})
require.Equal(t, specsLengthBefore, len(specs))
for _, s := range specs {
if strings.Contains(s.Name, "success_skip_selector") {
Expand All @@ -169,10 +174,16 @@ func Test_updateSpecForSelectiveTests(t *testing.T) {
}
}
})
// We run the randomized tests 1000 times for 100 randomly generated tests
// We run the randomized tests 100 times for 100 randomly generated tests
t.Run("run with randomised data", func(t *testing.T) {
iteration := 0
totalIterations := 1000
totalIterations := 100

// Too many goroutines causes this test to OOM.
if util.RaceEnabled {
t.Log("race enabled, reducing totalIterations to 10")
totalIterations = 10
}
totalTestCount := 100
oldSuite := roachtestflags.Suite
roachtestflags.Suite = registry.Nightly
Expand Down Expand Up @@ -212,7 +223,9 @@ func Test_updateSpecForSelectiveTests(t *testing.T) {
defer mu.Unlock()
return dbs, err
}
updateSpecForSelectiveTests(ctx, specs)
updateSpecForSelectiveTests(ctx, specs, func(format string, args ...interface{}) {
t.Logf(format, args...)
})
require.Equal(t, specsLengthBefore, len(specs))
// dataMap is used of assertions
dataMap := make(map[string][]string)
Expand Down
6 changes: 5 additions & 1 deletion pkg/raft/raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -2060,7 +2060,11 @@ func stepCandidate(r *raft, m pb.Message) error {
r.becomeFollower(r.Term, r.lead)
}
case pb.MsgTimeoutNow:
r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From)
r.becomeFollower(m.Term, m.From) // always m.Term == r.Term
// TODO(nvanbenschoten): this is temporarily duplicating logic from
// stepFollower. Unify.
r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership", r.id, r.Term, m.From)
r.hup(campaignTransfer)
}
return nil
}
Expand Down
25 changes: 25 additions & 0 deletions pkg/raft/raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3099,6 +3099,31 @@ func TestLeaderTransferToSlowFollower(t *testing.T) {
checkLeaderTransferState(t, lead, pb.StateFollower, 3)
}

func TestLeaderTransferToCandidate(t *testing.T) {
nt := newNetworkWithConfig(preVoteConfigWithFortificationDisabled, nil, nil, nil)
n3 := nt.peers[3].(*raft)

// Elect node 1 as the leader of term 1.
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
require.Equal(t, uint64(1), n3.Term)

// Isolate node 3 so that it decides to become a pre-candidate.
nt.isolate(3)
for i := 0; i < n3.randomizedElectionTimeout; i++ {
nt.tick(n3)
}
require.Equal(t, pb.StatePreCandidate, n3.state)
require.Equal(t, uint64(1), n3.Term)

// Reconnect node 3 and initiate a transfer of leadership from node 1 to node
// 3, all before node 3 steps back to a follower. This will instruct node 3 to
// call an election at the next term, which it can and does win.
nt.recover()
nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader})
require.Equal(t, pb.StateLeader, n3.state)
require.Equal(t, uint64(2), n3.Term)
}

func TestLeaderTransferAfterSnapshot(t *testing.T) {
nt := newNetwork(nil, nil, nil)
nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
Expand Down

0 comments on commit 7a9a01b

Please sign in to comment.