Skip to content

Commit 1b87ee7

Browse files
committed
roachtest: refactor mixedversion tests to use global test monitor
Change the mixedversion tests to use the global test monitor. We want to move all tests to the global test monitor eventually and the mixedversion tests are a good target to ensure our monitor changes targeting specific processes work as they create separate process clusters.
1 parent 22c043b commit 1b87ee7

28 files changed

+49
-106
lines changed

pkg/cmd/roachtest/roachtestutil/mixedversion/helper.go

Lines changed: 0 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -283,20 +283,6 @@ func (h *Helper) GoCommand(cmd string, nodes option.NodeListOption) context.Canc
283283
}, task.Name(desc))
284284
}
285285

286-
// ExpectDeath alerts the testing infrastructure that a node is
287-
// expected to die. Regular restarts as part of the mixedversion
288-
// testing are already taken into account. This function should only
289-
// be used by tests that perform their own node restarts or chaos
290-
// events.
291-
func (h *Helper) ExpectDeath() {
292-
h.ExpectDeaths(1)
293-
}
294-
295-
// ExpectDeaths is the general version of `ExpectDeath()`.
296-
func (h *Helper) ExpectDeaths(n int) {
297-
h.runner.monitor.ExpectDeaths(n)
298-
}
299-
300286
// ClusterVersion returns the currently active cluster version. Avoids
301287
// querying the database if we are not running migrations, since the
302288
// test runner has cached version of the cluster versions.

pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -642,6 +642,10 @@ func NewTest(
642642
crdbNodes option.NodeListOption,
643643
options ...CustomOption,
644644
) *Test {
645+
if !t.Spec().(*registry.TestSpec).Monitor {
646+
t.Fatal("mixedversion tests require enabling the global test monitor in the test spec")
647+
}
648+
645649
opts := defaultTestOptions()
646650
for _, fn := range options {
647651
fn(&opts)
@@ -858,7 +862,7 @@ func (t *Test) Run() {
858862
}
859863

860864
func (t *Test) run(plan *TestPlan) error {
861-
return newTestRunner(t.ctx, t.cancel, plan, t.options.tag, t.logger, t.cluster).run()
865+
return newTestRunner(t.ctx, t.cancel, plan, t.rt, t.options.tag, t.logger, t.cluster).run()
862866
}
863867

864868
func (t *Test) plan() (plan *TestPlan, retErr error) {

pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go

Lines changed: 4 additions & 81 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,6 @@ import (
1414
"path/filepath"
1515
"regexp"
1616
"strings"
17-
"sync"
1817
"sync/atomic"
1918
"text/tabwriter"
2019
"time"
@@ -25,6 +24,7 @@ import (
2524
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
2625
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade"
2726
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/task"
27+
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2828
"github.com/cockroachdb/cockroach/pkg/roachpb"
2929
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
3030
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
@@ -33,23 +33,9 @@ import (
3333
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
3434
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
3535
"github.com/cockroachdb/errors"
36-
"golang.org/x/exp/maps"
3736
)
3837

3938
type (
40-
// crdbMonitor is a thin wrapper around the roachtest monitor API
41-
// (cluster.NewMonitor) that produces error events through a channel
42-
// whenever an unexpected node death happens. It also allows us to
43-
// provide an API for test authors to inform the framework that a
44-
// node death is expected if the test performs its own restarts or
45-
// chaos events.
46-
crdbMonitor struct {
47-
once sync.Once
48-
crdbNodes option.NodeListOption
49-
monitor cluster.Monitor
50-
errCh chan error
51-
}
52-
5339
serviceRuntime struct {
5440
descriptor *ServiceDescriptor
5541
binaryVersions *atomic.Value
@@ -72,7 +58,7 @@ type (
7258
logger *logger.Logger
7359

7460
background task.Manager
75-
monitor *crdbMonitor
61+
monitor test.Monitor
7662

7763
// ranUserHooks keeps track of whether the runner has run any
7864
// user-provided hooks so far.
@@ -111,6 +97,7 @@ func newTestRunner(
11197
ctx context.Context,
11298
cancel context.CancelFunc,
11399
plan *TestPlan,
100+
rt test.Test,
114101
tag string,
115102
l *logger.Logger,
116103
c cluster.Cluster,
@@ -142,7 +129,7 @@ func newTestRunner(
142129
tenantService: tenantService,
143130
cluster: c,
144131
background: task.NewManager(ctx, l),
145-
monitor: newCRDBMonitor(ctx, c, maps.Keys(allCRDBNodes)),
132+
monitor: rt.Monitor(),
146133
ranUserHooks: &ranUserHooks,
147134
}
148135
}
@@ -193,9 +180,6 @@ func (tr *testRunner) run() (retErr error) {
193180
}
194181

195182
return fmt.Errorf("background step `%s` returned error: %w", event.Name, event.Err)
196-
197-
case err := <-tr.monitor.Err():
198-
return tr.testFailure(tr.ctx, err, tr.logger, nil)
199183
}
200184
}
201185
}
@@ -388,11 +372,6 @@ func (tr *testRunner) teardown(stepsChan chan error, testFailed bool) {
388372
tr.logger.Printf("stopping background functions")
389373
tr.background.Terminate(tr.logger)
390374

391-
tr.logger.Printf("stopping node monitor")
392-
if err := tr.monitor.Stop(); err != nil {
393-
tr.logger.Printf("monitor returned error: %v", err)
394-
}
395-
396375
// If the test failed, we wait for any currently running steps to
397376
// return before passing control back to the roachtest
398377
// framework. This achieves a test.log that does not contain any
@@ -614,22 +593,6 @@ func (tr *testRunner) refreshServiceData(ctx context.Context, service *serviceRu
614593
return err
615594
}
616595

617-
// We only want to start the monitor once we know every relevant
618-
// cockroach binary is running. This is due to a limitation on the
619-
// roachprod monitor: it is only able to monitor cockroach processes
620-
// that are running at the time the monitor is created.
621-
//
622-
// For system-only and separate-process deployments, we can
623-
// initialize the monitor right away, since this function is only
624-
// called once the storage cluster is running. For separate-process
625-
// deployments, we start the monitor if this function is called with
626-
// the tenant service. The system is always started first, so when
627-
// this function is called with the tenant service, we know that
628-
// every relevant cockroach binary is running at this point.
629-
if tr.plan.deploymentMode != SeparateProcessDeployment || !isSystem {
630-
tr.monitor.Init()
631-
}
632-
633596
return nil
634597
}
635598

@@ -749,46 +712,6 @@ func (tr *testRunner) addGrafanaAnnotation(
749712
return tr.cluster.AddGrafanaAnnotation(ctx, l, req)
750713
}
751714

752-
func newCRDBMonitor(
753-
ctx context.Context, c cluster.Cluster, crdbNodes option.NodeListOption,
754-
) *crdbMonitor {
755-
return &crdbMonitor{
756-
crdbNodes: crdbNodes,
757-
monitor: c.NewMonitor(ctx, crdbNodes),
758-
errCh: make(chan error),
759-
}
760-
}
761-
762-
// Init must be called once the cluster is initialized and the
763-
// cockroach process is running on the nodes. Init is idempotent.
764-
func (cm *crdbMonitor) Init() {
765-
cm.once.Do(func() {
766-
go func() {
767-
if err := cm.monitor.WaitForNodeDeath(); err != nil {
768-
cm.errCh <- err
769-
}
770-
}()
771-
})
772-
}
773-
774-
// Err returns a channel that will receive errors whenever an
775-
// unexpected node death is observed.
776-
func (cm *crdbMonitor) Err() chan error {
777-
return cm.errCh
778-
}
779-
780-
func (cm *crdbMonitor) ExpectDeaths(n int) {
781-
cm.monitor.ExpectDeaths(int32(n))
782-
}
783-
784-
func (cm *crdbMonitor) Stop() error {
785-
if cm.monitor == nil { // test-only
786-
return nil
787-
}
788-
789-
return cm.monitor.WaitE()
790-
}
791-
792715
// tableWriter is a thin wrapper around the `tabwriter` package used
793716
// by the test runner to display logical and released binary versions
794717
// in a tabular format.

pkg/cmd/roachtest/roachtestutil/mixedversion/runner_test.go

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -119,10 +119,6 @@ func Test_run(t *testing.T) {
119119
startSystemID: 9999,
120120
}
121121

122-
runnerCh := make(chan error)
123-
defer close(runnerCh)
124-
runner.monitor = &crdbMonitor{errCh: runnerCh}
125-
126122
runErr := runner.run()
127123
require.Error(t, runErr)
128124

pkg/cmd/roachtest/roachtestutil/mixedversion/steps.go

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -227,7 +227,6 @@ func (s restartVirtualClusterStep) Run(
227227
l.Printf("restarting node %d (tenant %s) into version %s", s.node, s.virtualCluster, s.version)
228228
node := h.runner.cluster.Node(s.node)
229229

230-
h.ExpectDeath()
231230
stopOpts := option.StopVirtualClusterOpts(s.virtualCluster, node, option.Graceful(maxWait))
232231
if err := h.runner.cluster.StopServiceForVirtualClusterE(ctx, l, stopOpts); err != nil {
233232
return errors.Wrap(err, "failed to stop cockroach process for tenant")
@@ -359,13 +358,13 @@ func (s restartWithNewBinaryStep) Run(
359358
install.TagOption(systemTag),
360359
}, s.settings...)
361360

362-
h.ExpectDeath()
361+
node := h.runner.cluster.Node(s.node)
363362
if err := clusterupgrade.RestartNodesWithNewBinary(
364363
startCtx,
365364
s.rt,
366365
l,
367366
h.runner.cluster,
368-
h.runner.cluster.Node(s.node),
367+
node,
369368
startOpts(customStartOpts...),
370369
s.version,
371370
settings...,
@@ -377,7 +376,7 @@ func (s restartWithNewBinaryStep) Run(
377376
// If we are in shared-process mode and the tenant is already
378377
// running at this point, we wait for the server on the restarted
379378
// node to be up before moving on.
380-
return waitForTenantProcess(ctx, l, h, h.runner.cluster.Node(s.node), s.deploymentMode)
379+
return waitForTenantProcess(ctx, l, h, node, s.deploymentMode)
381380
}
382381

383382
return nil
@@ -815,7 +814,7 @@ func (s panicNodeStep) Run(ctx context.Context, l *logger.Logger, rng *rand.Rand
815814
)
816815
customStartOpts := restartSystemSettings(true, s.initTarget)
817816

818-
h.ExpectDeath()
817+
h.runner.monitor.ExpectProcessDead(s.targetNode)
819818

820819
const stmt = "SELECT crdb_internal.force_panic('expected panic from panicNodeMutator')"
821820
err = h.System.ExecWithGateway(

pkg/cmd/roachtest/tests/acceptance.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ func registerAcceptance(r registry.Registry) {
2929
timeout time.Duration
3030
encryptionSupport registry.EncryptionSupport
3131
defaultLeases bool
32+
monitor bool
3233
randomized bool
3334
workloadNode bool
3435
incompatibleClouds registry.CloudSet
@@ -75,6 +76,7 @@ func registerAcceptance(r registry.Registry) {
7576
fn: runVersionUpgrade,
7677
timeout: 2 * time.Hour, // actually lower in local runs; see `runVersionUpgrade`
7778
defaultLeases: true,
79+
monitor: true,
7880
randomized: true,
7981
suites: []string{registry.MixedVersion},
8082
// Disabled on IBM because s390x is only built on master
@@ -103,6 +105,7 @@ func registerAcceptance(r registry.Registry) {
103105
fn: runValidateSystemSchemaAfterVersionUpgrade,
104106
timeout: 60 * time.Minute,
105107
defaultLeases: true,
108+
monitor: true,
106109
randomized: true,
107110
numNodes: 1,
108111
suites: []string{registry.MixedVersion},
@@ -157,6 +160,7 @@ func registerAcceptance(r registry.Registry) {
157160
Timeout: 10 * time.Minute,
158161
CompatibleClouds: registry.AllClouds.Remove(tc.incompatibleClouds),
159162
Suites: registry.Suites(suites...),
163+
Monitor: tc.monitor,
160164
Randomized: tc.randomized,
161165
}
162166

pkg/cmd/roachtest/tests/admission_control_elastic_mixed_version.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@ func registerElasticWorkloadMixedVersion(r registry.Registry) {
3939
Owner: registry.OwnerKV,
4040
Timeout: 3 * time.Hour,
4141
Benchmark: true,
42+
Monitor: true,
4243
CompatibleClouds: registry.OnlyGCE,
4344
Suites: registry.Suites(registry.MixedVersion, registry.Nightly),
4445
Cluster: r.MakeClusterSpec(4, spec.CPU(8),

pkg/cmd/roachtest/tests/db_console_cypress.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -206,6 +206,7 @@ func registerDbConsoleCypress(r registry.Registry) {
206206
// is impossible to test as of 05/2025.
207207
CompatibleClouds: registry.AllClouds.NoIBM(),
208208
Suites: registry.Suites(registry.MixedVersion, registry.Nightly),
209+
Monitor: true,
209210
Randomized: false,
210211
Run: runDbConsoleCypressMixedVersions,
211212
Timeout: 2 * time.Hour,

pkg/cmd/roachtest/tests/db_console_endpoints.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -100,6 +100,7 @@ func registerDBConsoleEndpointsMixedVersion(r registry.Registry) {
100100
Cluster: r.MakeClusterSpec(5, spec.WorkloadNode()),
101101
CompatibleClouds: registry.AllClouds,
102102
Suites: registry.Suites(registry.MixedVersion, registry.Nightly),
103+
Monitor: true,
103104
Randomized: true,
104105
Run: runDBConsoleMixedVersion,
105106
Timeout: 1 * time.Hour,

pkg/cmd/roachtest/tests/decommission.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -119,6 +119,8 @@ func registerDecommission(r registry.Registry) {
119119
Cluster: r.MakeClusterSpec(numNodes),
120120
CompatibleClouds: registry.AllClouds.NoAWS().NoIBM(),
121121
Suites: registry.Suites(registry.MixedVersion, registry.Nightly),
122+
Monitor: true,
123+
Randomized: true,
122124
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
123125
runDecommissionMixedVersions(ctx, t, c)
124126
},

0 commit comments

Comments
 (0)