Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
119535: roachtest: fail tests if monitor encounters an error r=srosenberg a=renatolabs

This commit updates the roachprod and roachtest monitors to 1) send an
event when the monitor is abruptly terminated (i.e., reader stream
sees an EOF when the associated context is *not* canceled); and 2)
return any errors encountered by the roachprod monitor in roachtest,
causing the currently running test to fail. The error has TestEng
ownership so that teams are not be pinged on these kinds of flakes.

The main purpose of this change is for the monitor to fail in
situations where the monitored node is preempted by the cloud
provider. Previously, these events would be ignored, leading to a test
timeout, wasting resources and leading to confusing test failures
being reported on GitHub.

Fixes: #118563.

Release note: None

119725: changefeedccl: disable TestChangefeedOnlyInitialScanCSV with pulsar r=rharding6373 a=jayshrivastava

Informs: #119289
Release note: None
Epic: None

119732: master: Update pkg/testutils/release/cockroach_releases.yaml r=rail a=github-actions[bot]

Update pkg/testutils/release/cockroach_releases.yaml with recent values.

Epic: None
Release note: None
Release justification: test-only updates

Co-authored-by: Renato Costa <renato@cockroachlabs.com>
Co-authored-by: Jayant Shrivastava <jayants@cockroachlabs.com>
Co-authored-by: CRL Release bot <teamcity@cockroachlabs.com>
  • Loading branch information
4 people committed Feb 28, 2024
4 parents eab85b4 + 29892b5 + 1a90bfb + 5b274c8 commit 0ce84d4
Show file tree
Hide file tree
Showing 10 changed files with 86 additions and 33 deletions.
3 changes: 2 additions & 1 deletion pkg/ccl/changefeedccl/changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7526,7 +7526,8 @@ func TestChangefeedOnlyInitialScanCSV(t *testing.T) {
}
}

cdcTest(t, testFn, feedTestEnterpriseSinks)
// TODO(#119289): re-enable pulsar
cdcTest(t, testFn, feedTestEnterpriseSinks, feedTestOmitSinks("pulsar"))
}

func TestChangefeedOnlyInitialScanCSVSinkless(t *testing.T) {
Expand Down
7 changes: 3 additions & 4 deletions pkg/cmd/roachtest/github.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,10 +171,7 @@ func (g *githubIssues) createPostRequest(
issueClusterName := ""
errWithOwnership := failuresSpecifyOwner(failures)
switch {
case errWithOwnership != nil:
handleErrorWithOwnership(*errWithOwnership)

// The following error come from various entrypoints in roachprod,
// The following errors come from various entrypoints in roachprod,
// but we know that they should be handled by TestEng whenever they
// happen during a test.
case failuresContainsError(failures, rperrors.ErrSSH255):
Expand All @@ -189,6 +186,8 @@ func (g *githubIssues) createPostRequest(
registry.WithTitleOverride("dns_problem"),
registry.InfraFlake,
))
case errWithOwnership != nil:
handleErrorWithOwnership(*errWithOwnership)
}

// Issues posted from roachtest are identifiable as such, and they are also release blockers
Expand Down
28 changes: 16 additions & 12 deletions pkg/cmd/roachtest/github_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -297,38 +297,42 @@ func TestCreatePostRequest(t *testing.T) {
"coverageBuild": "true",
}),
},
// 9. Verify preemption failure are routed to test-eng and marked as infra-flake,
// even if the first failure is another handled error.
// 9. Verify preemption failure are routed to test-eng and marked as infra-flake, when the
// first failure is a non-handled error.
{
nonReleaseBlocker: true,
failures: []failure{createFailure(gce.ErrDNSOperation), createFailure(vmPreemptionError("my_VM"))},
failures: []failure{createFailure(errors.New("random")), createFailure(vmPreemptionError("my_VM"))},
expectedPost: true,
expectedName: "vm_preemption",
expectedTeam: "@cockroachdb/test-eng",
expectedName: "vm_preemption",
expectedMessagePrefix: testName + " failed",
expectedLabels: []string{"T-testeng", "X-infra-flake"},
},
// 10. Verify preemption failure are routed to test-eng and marked as infra-flake, when the
// first failure is a non-handled error.
// 10. Verify preemption failure are routed to test-eng and marked as infra-flake, when the only error is
// preemption failure
{
nonReleaseBlocker: true,
failures: []failure{createFailure(errors.New("random")), createFailure(vmPreemptionError("my_VM"))},
nonReleaseBlocker: true,
failures: []failure{
{errors: []error{vmPreemptionError("my_VM")}},
},
expectedPost: true,
expectedTeam: "@cockroachdb/test-eng",
expectedName: "vm_preemption",
expectedMessagePrefix: testName + " failed",
expectedLabels: []string{"T-testeng", "X-infra-flake"},
},
// 11. Verify preemption failure are routed to test-eng and marked as infra-flake, when the only error is
// preemption failure
// 11. Errors with ownership that happen as a result of roachprod
// errors are ignored -- roachprod errors are routed directly to
// test-eng.
{
nonReleaseBlocker: true,
failures: []failure{
{errors: []error{vmPreemptionError("my_VM")}},
createFailure(gce.ErrDNSOperation),
createFailure(registry.ErrorWithOwner(registry.OwnerSQLFoundations, errors.New("oops"))),
},
expectedPost: true,
expectedTeam: "@cockroachdb/test-eng",
expectedName: "vm_preemption",
expectedName: "dns_problem",
expectedMessagePrefix: testName + " failed",
expectedLabels: []string{"T-testeng", "X-infra-flake"},
},
Expand Down
44 changes: 37 additions & 7 deletions pkg/cmd/roachtest/monitor.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/roachprod"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
Expand Down Expand Up @@ -168,16 +169,45 @@ func (m *monitorImpl) startNodeMonitor() {
}

for info := range eventsCh {
_, isDeath := info.Event.(install.MonitorProcessDead)
isExpectedDeath := isDeath && atomic.AddInt32(&m.expDeaths, -1) >= 0
var expectedDeathStr string
if isExpectedDeath {
expectedDeathStr = ": expected"
var retErr error

switch e := info.Event.(type) {
case install.MonitorError:
if !errors.Is(e.Err, install.MonitorNoCockroachProcessesError) {
// Monitor errors should only occur when something went
// wrong in the monitor logic itself or test infrastructure
// (SSH flakes, VM preemption, etc). These should be sent
// directly to TestEng.
//
// NOTE: we ignore `MonitorNoCockroachProcessesError` as a
// lot of monitors uses in current tests would fail with
// this error if we returned it and current uses are
// harmless. In the future, the monitor should be able to
// detect when new cockroach processes start running on a
// node instead of assuming that the processes are running
// when the monitor starts.
retErr = registry.ErrorWithOwner(
registry.OwnerTestEng,
e.Err,
registry.WithTitleOverride("monitor_failure"),
registry.InfraFlake,
)
}
case install.MonitorProcessDead:
isExpectedDeath := atomic.AddInt32(&m.expDeaths, -1) >= 0
if isExpectedDeath {
expectedDeathStr = ": expected"
}

if !isExpectedDeath {
retErr = fmt.Errorf("unexpected node event: %s", info)
}
}
m.l.Printf("Monitor event: %s%s", info, expectedDeathStr)

if isDeath && !isExpectedDeath {
return fmt.Errorf("unexpected node event: %s", info)
m.l.Printf("Monitor event: %s%s", info, expectedDeathStr)
if retErr != nil {
return retErr
}
}

Expand Down
6 changes: 6 additions & 0 deletions pkg/cmd/roachtest/test_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -445,6 +445,12 @@ func (t *testImpl) suppressFailures() {
t.mu.failuresSuppressed = true
}

func (t *testImpl) resetFailures() {
t.mu.Lock()
defer t.mu.Unlock()
t.mu.failures = nil
}

// 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 {
Expand Down
8 changes: 8 additions & 0 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -1016,6 +1016,14 @@ func (r *testRunner) runTest(
preemptedVMNames := getPreemptedVMNames(ctx, c, l)
if preemptedVMNames != "" {
failureMsg = fmt.Sprintf("VMs preempted during the test run : %s\n\n**Other Failure**\n%s", preemptedVMNames, failureMsg)
// Reset failures in the test so that the VM preemption
// error is the one that is taken into account when
// reporting the failure. Note any other failures that
// happened during the test will be present in the
// `failureMsg` used when reporting the issue. In addition,
// `failure_N.log` files should also already exist at this
// point.
t.resetFailures()
t.Error(vmPreemptionError(preemptedVMNames))
}
output := fmt.Sprintf("%s\ntest artifacts and logs in: %s", failureMsg, t.ArtifactsDir())
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -443,7 +443,7 @@ func runLoadSplits(ctx context.Context, t test.Test, c cluster.Cluster, params s
)
c.Start(ctx, t.L(), startOpts, settings, crdbNodes)

m := c.NewMonitor(ctx, c.All())
m := c.NewMonitor(ctx, crdbNodes)
m.Go(func(ctx context.Context) error {
db := c.Conn(ctx, t.L(), 1)
defer db.Close()
Expand Down
9 changes: 7 additions & 2 deletions pkg/roachprod/install/cluster_synced.go
Original file line number Diff line number Diff line change
Expand Up @@ -656,6 +656,11 @@ type MonitorError struct {
Err error
}

// MonitorNoCockroachProcessesError is the error returned when the
// monitor is called on a node that is not running a `cockroach`
// process by the time the monitor runs.
var MonitorNoCockroachProcessesError = errors.New("no cockroach processes running")

// NodeMonitorInfo is a message describing a cockroach process' status.
type NodeMonitorInfo struct {
// The index of the node (in a SyncedCluster) at which the message originated.
Expand Down Expand Up @@ -792,8 +797,7 @@ func (c *SyncedCluster) Monitor(
vcs := map[virtualClusterInfo]struct{}{}
vcLines := strings.TrimSuffix(result.CombinedOut, "\n")
if vcLines == "" {
err := errors.New("no cockroach processes running")
sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}})
sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{MonitorNoCockroachProcessesError}})
return
}
for _, label := range strings.Split(vcLines, "\n") {
Expand Down Expand Up @@ -955,6 +959,7 @@ wait
for {
line, _, err := r.ReadLine()
if err == io.EOF {
sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}})
return
}
if err != nil {
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/logictest/REPOSITORIES.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -13,11 +13,11 @@ _CONFIGS = [
(CONFIG_LINUX_AMD64, "2e817e489ce5ab91706d1008dc023ee8ff5c79051f1af45e6012f44edc736c0a"),
(CONFIG_LINUX_ARM64, "68d9d5e270cfaaa1b07fcf40051385d7f014de6527030a58e300e2b22c3db49b"),
]),
("23.2.1", [
(CONFIG_DARWIN_AMD64, "83b4326e81e2168965d22a9a9ea534f7a81f1aed0b39860b8ef715497f178c2a"),
(CONFIG_DARWIN_ARM64, "7e37ae53668f64869b941fc3a343298ef0f1ecdc24e2d40d23d2b9050bc3821d"),
(CONFIG_LINUX_AMD64, "258fb9ba5d7f8e6fac6c972f3bf3010f8a5c39e93aea89ae195e4ed82ffd600a"),
(CONFIG_LINUX_ARM64, "851a889f870e22536e961048810ad5becff128bf34df0187ef00c62e1ca8592d"),
("23.2.2", [
(CONFIG_DARWIN_AMD64, "2da531bc329683d4926c5f6ad36f6eaffb1fcb93e306db10e187522f44edb102"),
(CONFIG_DARWIN_ARM64, "f0e7eccf86d6d9dd415493a3ec02327ff481e97c2dcfd45b9ab1cdf6222d787f"),
(CONFIG_LINUX_AMD64, "bf58a5bccca741d455209a01c530f26148de303ce3e388c59285bca9665a95b5"),
(CONFIG_LINUX_ARM64, "baff9a60473d7b0ea9385e34b29cadb65c2fca6d6919c0f4450308eaf8b44290"),
]),
]

Expand Down
2 changes: 1 addition & 1 deletion pkg/testutils/release/cockroach_releases.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
- 23.1.0
predecessor: "22.2"
"23.2":
latest: 23.2.1
latest: 23.2.2
predecessor: "23.1"
"24.1":
predecessor: "23.2"

0 comments on commit 0ce84d4

Please sign in to comment.