Skip to content

Commit

Permalink
Merge #103946
Browse files Browse the repository at this point in the history
103946: roachtest: categorise post-test assertions separately r=smg260 a=smg260

roachtest: categorise post-test assertion errors separately

1. Simple change to mark failures that occur in post-test validation (teardown), so that we can more clearly report a test that has experience a failure after its execution has ended.

2. Every test, unless opted-out, runs various post validations such as node liveness and consistency checks. These checks have been moved to their own function, and failures that occur here are now logged to a separate file, and indicated as such in github when an issue is created.\
\
Previously, these checks were done during teardown and logged to the teardown.log.

Epic: none
Fixes: #98366

Release note: None

Co-authored-by: Miral Gadani <miral@cockroachlabs.com>
  • Loading branch information
craig[bot] and Miral Gadani committed Jun 12, 2023
2 parents 65b478a + 432909c commit 9c2f650
Show file tree
Hide file tree
Showing 6 changed files with 286 additions and 234 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ go_test(
"//pkg/cmd/roachtest/spec",
"//pkg/cmd/roachtest/test",
"//pkg/internal/team",
"//pkg/roachprod/errors",
"//pkg/roachprod/logger",
"//pkg/roachprod/vm",
"//pkg/testutils",
Expand Down
22 changes: 10 additions & 12 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -1608,33 +1608,31 @@ func (c *clusterImpl) HealthStatus(
return results, nil
}

// FailOnInvalidDescriptors fails the test if there exists any descriptors in
// assertValidDescriptors fails the test if there exists any descriptors in
// the crdb_internal.invalid_objects virtual table.
func (c *clusterImpl) FailOnInvalidDescriptors(ctx context.Context, db *gosql.DB, t *testImpl) {
func (c *clusterImpl) assertValidDescriptors(ctx context.Context, db *gosql.DB, t *testImpl) error {
t.L().Printf("checking for invalid descriptors")
if err := timeutil.RunWithTimeout(
return timeutil.RunWithTimeout(
ctx, "invalid descriptors check", 1*time.Minute,
func(ctx context.Context) error {
return roachtestutil.CheckInvalidDescriptors(ctx, db)
},
); err != nil {
t.Errorf("invalid descriptors check failed: %v", err)
}
)
}

// FailOnReplicaDivergence fails the test if
// assertConsistentReplicas fails the test if
// crdb_internal.check_consistency(true, ”, ”) indicates that any ranges'
// replicas are inconsistent with each other.
func (c *clusterImpl) FailOnReplicaDivergence(ctx context.Context, db *gosql.DB, t *testImpl) {
func (c *clusterImpl) assertConsistentReplicas(
ctx context.Context, db *gosql.DB, t *testImpl,
) error {
t.L().Printf("checking for replica divergence")
if err := timeutil.RunWithTimeout(
return timeutil.RunWithTimeout(
ctx, "consistency check", 5*time.Minute,
func(ctx context.Context) error {
return roachtestutil.CheckReplicaDivergenceOnDB(ctx, t.L(), db)
},
); err != nil {
t.Errorf("consistency check failed: %v", err)
}
)
}

// FetchDmesg grabs the dmesg logs if possible. This requires being able to run
Expand Down
34 changes: 9 additions & 25 deletions pkg/cmd/roachtest/github.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,14 +32,6 @@ type githubIssues struct {
teamLoader func() (team.Map, error)
}

type issueCategory int

const (
otherErr issueCategory = iota
clusterCreationErr
sshErr
)

func newGithubIssues(disable bool, c *clusterImpl, vmCreateOpts *vm.CreateOpts) *githubIssues {
return &githubIssues{
disable: disable,
Expand Down Expand Up @@ -106,7 +98,7 @@ func (g *githubIssues) shouldPost(t test.Test) (bool, string) {
}

func (g *githubIssues) createPostRequest(
t test.Test, cat issueCategory, message string,
t test.Test, firstFailure failure, message string,
) issues.PostRequest {
var mention []string
var projColID int
Expand All @@ -118,21 +110,23 @@ func (g *githubIssues) createPostRequest(
messagePrefix := ""
var infraFlake bool
// Overrides to shield eng teams from potential flakes
if cat == clusterCreationErr {
switch {
case failureContainsError(firstFailure, errClusterProvisioningFailed):
issueOwner = registry.OwnerDevInf
issueName = "cluster_creation"
messagePrefix = fmt.Sprintf("test %s was skipped due to ", t.Name())
infraFlake = true
} else if cat == sshErr {
case failureContainsError(firstFailure, rperrors.ErrSSH255):
issueOwner = registry.OwnerTestEng
issueName = "ssh_problem"
messagePrefix = fmt.Sprintf("test %s failed due to ", t.Name())
infraFlake = true
case failureContainsError(firstFailure, errDuringPostAssertions):
messagePrefix = fmt.Sprintf("test %s failed during post test assertions (see test-post-assertions.log) due to ", t.Name())
}

// Issues posted from roachtest are identifiable as such and
// they are also release blockers (this label may be removed
// by a human upon closer investigation).
// Issues posted from roachtest are identifiable as such, and they are also release blockers
// (this label may be removed by a human upon closer investigation).
labels := []string{"O-roachtest"}
if !spec.NonReleaseBlocker && !infraFlake {
labels = append(labels, "release-blocker")
Expand Down Expand Up @@ -218,20 +212,10 @@ func (g *githubIssues) MaybePost(t *testImpl, l *logger.Logger, message string)
return nil
}

cat := otherErr

// Overrides to shield eng teams from potential flakes
firstFailure := t.firstFailure()
if failureContainsError(firstFailure, errClusterProvisioningFailed) {
cat = clusterCreationErr
} else if failureContainsError(firstFailure, rperrors.ErrSSH255) {
cat = sshErr
}

return g.issuePoster(
context.Background(),
l,
issues.UnitTestFormatter,
g.createPostRequest(t, cat, message),
g.createPostRequest(t, t.firstFailure(), message),
)
}
154 changes: 84 additions & 70 deletions pkg/cmd/roachtest/github_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@ package main

import (
"context"
"errors"
"fmt"
"strings"
"testing"

Expand All @@ -21,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/internal/team"
rperrors "github.com/cockroachdb/cockroach/pkg/roachprod/errors"
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -101,18 +104,22 @@ func TestShouldPost(t *testing.T) {
}

func TestCreatePostRequest(t *testing.T) {
createFailure := func(ref error) failure {
return failure{squashedErr: ref}
}

testCases := []struct {
nonReleaseBlocker bool
clusterCreationFailed bool
loadTeamsFailed bool
localSSD bool
arch vm.CPUArch
category issueCategory
failure failure
expectedPost bool
expectedReleaseBlocker bool
expectedParams map[string]string
}{
{true, false, false, false, "", otherErr, true, false,
{true, false, false, false, "", createFailure(errors.New("other")), true, false,
prefixAll(map[string]string{
"cloud": "gce",
"encrypted": "false",
Expand All @@ -123,7 +130,7 @@ func TestCreatePostRequest(t *testing.T) {
"localSSD": "false",
}),
},
{true, false, false, true, vm.ArchARM64, clusterCreationErr, true, false,
{true, false, false, true, vm.ArchARM64, createFailure(errClusterProvisioningFailed), true, false,
prefixAll(map[string]string{
"cloud": "gce",
"encrypted": "false",
Expand All @@ -138,92 +145,99 @@ func TestCreatePostRequest(t *testing.T) {
// !nonReleaseBlocker and issue is an SSH flake. Also ensure that
// in the event of a failed cluster creation, nil `vmOptions` and
// `clusterImpl` are not dereferenced
{false, true, false, false, "", sshErr, true, false,
{false, true, false, false, "", createFailure(rperrors.ErrSSH255), true, false,
prefixAll(map[string]string{
"cloud": "gce",
"ssd": "0",
"cpu": "4",
}),
},
//Simulate failure loading TEAMS.yaml
{true, false, true, false, "", otherErr, false, false, nil},
{true, false, true, false, "", createFailure(errors.New("other")), false, false, nil},
//Error during post test assertions
{true, false, false, false, "", createFailure(errDuringPostAssertions), false, false, nil},
}

reg := makeTestRegistry(spec.GCE, "", "", false, false)
for _, c := range testCases {
clusterSpec := reg.MakeClusterSpec(1, spec.Arch(c.arch))

testSpec := &registry.TestSpec{
Name: "github_test",
Owner: OwnerUnitTest,
Cluster: clusterSpec,
NonReleaseBlocker: c.nonReleaseBlocker,
}

ti := &testImpl{
spec: testSpec,
l: nilLogger(),
}

testClusterImpl := &clusterImpl{spec: clusterSpec, arch: vm.ArchAMD64}
vo := vm.DefaultCreateOpts()
vmOpts := &vo

if c.clusterCreationFailed {
testClusterImpl = nil
vmOpts = nil
} else if !c.localSSD {
// The default is true set in `vm.DefaultCreateOpts`
vmOpts.SSDOpts.UseLocalSSD = false
}
for idx, c := range testCases {
t.Run(fmt.Sprintf("%d", idx+1), func(t *testing.T) {
clusterSpec := reg.MakeClusterSpec(1, spec.Arch(c.arch))

testSpec := &registry.TestSpec{
Name: "github_test",
Owner: OwnerUnitTest,
Cluster: clusterSpec,
NonReleaseBlocker: c.nonReleaseBlocker,
}

teamLoadFn := validTeamsFn
ti := &testImpl{
spec: testSpec,
l: nilLogger(),
}

if c.loadTeamsFailed {
teamLoadFn = invalidTeamsFn
}
testClusterImpl := &clusterImpl{spec: clusterSpec, arch: vm.ArchAMD64}
vo := vm.DefaultCreateOpts()
vmOpts := &vo

github := &githubIssues{
vmCreateOpts: vmOpts,
cluster: testClusterImpl,
teamLoader: teamLoadFn,
}
if c.clusterCreationFailed {
testClusterImpl = nil
vmOpts = nil
} else if !c.localSSD {
// The default is true set in `vm.DefaultCreateOpts`
vmOpts.SSDOpts.UseLocalSSD = false
}

if c.loadTeamsFailed {
// Assert that if TEAMS.yaml cannot be loaded then function panics.
assert.Panics(t, func() { github.createPostRequest(ti, c.category, "message") })
} else {
req := github.createPostRequest(ti, c.category, "message")
teamLoadFn := validTeamsFn

if c.expectedParams != nil {
require.Equal(t, c.expectedParams, req.ExtraParams)
if c.loadTeamsFailed {
teamLoadFn = invalidTeamsFn
}

require.True(t, contains(req.ExtraLabels, nil, "O-roachtest"))
require.Equal(t, c.expectedReleaseBlocker, contains(req.ExtraLabels, nil, "release-blocker"))

expectedTeam := "@cockroachdb/unowned"
expectedName := "github_test"
expectedLabel := ""
expectedMessagePrefix := ""

if c.category == clusterCreationErr {
expectedTeam = "@cockroachdb/dev-inf"
expectedName = "cluster_creation"
expectedMessagePrefix = "test github_test was skipped due to "
} else if c.category == sshErr {
expectedTeam = "@cockroachdb/test-eng"
expectedLabel = "T-testeng"
expectedName = "ssh_problem"
expectedMessagePrefix = "test github_test failed due to "
github := &githubIssues{
vmCreateOpts: vmOpts,
cluster: testClusterImpl,
teamLoader: teamLoadFn,
}

require.Contains(t, req.MentionOnCreate, expectedTeam)
require.Equal(t, expectedName, req.TestName)
require.True(t, strings.HasPrefix(req.Message, expectedMessagePrefix), req.Message)
if expectedLabel != "" {
require.Contains(t, req.ExtraLabels, expectedLabel)
if c.loadTeamsFailed {
// Assert that if TEAMS.yaml cannot be loaded then function panics.
assert.Panics(t, func() { github.createPostRequest(ti, c.failure, "message") })
} else {
req := github.createPostRequest(ti, c.failure, "message")

if c.expectedParams != nil {
require.Equal(t, c.expectedParams, req.ExtraParams)
}

require.True(t, contains(req.ExtraLabels, nil, "O-roachtest"))
require.Equal(t, c.expectedReleaseBlocker, contains(req.ExtraLabels, nil, "release-blocker"))

expectedTeam := "@cockroachdb/unowned"
expectedName := "github_test"
expectedLabel := ""
expectedMessagePrefix := ""

if errors.Is(c.failure.squashedErr, errClusterProvisioningFailed) {
expectedTeam = "@cockroachdb/dev-inf"
expectedName = "cluster_creation"
expectedMessagePrefix = "test github_test was skipped due to "
} else if errors.Is(c.failure.squashedErr, rperrors.ErrSSH255) {
expectedTeam = "@cockroachdb/test-eng"
expectedLabel = "T-testeng"
expectedName = "ssh_problem"
expectedMessagePrefix = "test github_test failed due to "
} else if errors.Is(c.failure.squashedErr, errDuringPostAssertions) {
expectedMessagePrefix = "test github_test failed during post test assertions (see test-post-assertions.log) due to "
}

require.Contains(t, req.MentionOnCreate, expectedTeam)
require.Equal(t, expectedName, req.TestName)
require.True(t, strings.HasPrefix(req.Message, expectedMessagePrefix), req.Message)
if expectedLabel != "" {
require.Contains(t, req.ExtraLabels, expectedLabel)
}
}
}
})

}
}
5 changes: 2 additions & 3 deletions pkg/cmd/roachtest/test_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -370,8 +370,7 @@ func (t *testImpl) addFailure(depth int, format string, args ...interface{}) {
t.mu.output = append(t.mu.output, '\n')
}

// We take the first error from each failure which is the
// "squashed" error that contains all information of a failure
// We take the "squashed" error that contains information of all the errors for each failure.
func formatFailure(b *strings.Builder, reportFailures ...failure) {
for i, failure := range reportFailures {
if i > 0 {
Expand Down Expand Up @@ -402,7 +401,7 @@ func (t *testImpl) failedRLocked() bool {
func (t *testImpl) firstFailure() failure {
t.mu.RLock()
defer t.mu.RUnlock()
if len(t.mu.failures) <= 0 {
if len(t.mu.failures) == 0 {
return failure{}
}
return t.mu.failures[0]
Expand Down
Loading

0 comments on commit 9c2f650

Please sign in to comment.