diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go index f82c40430f69..9bb0700e7440 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go @@ -65,7 +65,6 @@ package mixedversion import ( "context" - gosql "database/sql" "fmt" "math/rand" "strings" @@ -93,11 +92,6 @@ const ( // of migration steps before the new cluster version can be // finalized. runWhileMigratingProbability = 0.5 - - // noDBNeeded is an internal sentinel value expected to be returned - // by steps generated by the test plan; it indicates that the step - // requires no connection to a cockroach node. - noDBNeeded = -1 ) var ( @@ -131,7 +125,7 @@ type ( Finalizing bool } - userFunc func(*logger.Logger, *gosql.DB) error + userFunc func(*logger.Logger, *rand.Rand, *Helper) error predicateFunc func(Context) bool // versionUpgradeHook is a hook that can be called at any time @@ -159,17 +153,12 @@ type ( // ID returns a unique ID associated with the step, making it easy // to reference test output with the exact step it relates to ID() int - // DBNode returns the database node that that step connects to - // during its execution. If the step does not require a database - // connection, this function should return the `noDBNeeded` - // constant - DBNode() int // Description is a string representation of the step, intended // for human-consumption. Displayed when pretty-printing the test // plan. Description() string // Run implements the actual functionality of the step. - Run(context.Context, *logger.Logger, cluster.Cluster, func(int) *gosql.DB) error + Run(context.Context, *logger.Logger, cluster.Cluster, *Helper) error } hooks []versionUpgradeHook @@ -218,7 +207,7 @@ func NewTest( } prng, seed := randutil.NewPseudoRand() - testLogger.Printf("random seed: %d", seed) + testLogger.Printf("mixed-version random seed: %d", seed) return &Test{ ctx: ctx, @@ -339,8 +328,7 @@ type startFromCheckpointStep struct { crdbNodes option.NodeListOption } -func (s startFromCheckpointStep) ID() int { return s.id } -func (s startFromCheckpointStep) DBNode() int { return noDBNeeded } +func (s startFromCheckpointStep) ID() int { return s.id } func (s startFromCheckpointStep) Description() string { return fmt.Sprintf("starting cluster from fixtures for version %q", s.version) @@ -350,7 +338,7 @@ func (s startFromCheckpointStep) Description() string { // upload the binary associated with that given version, and finally // start the cockroach binary on these nodes. func (s startFromCheckpointStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { if err := clusterupgrade.InstallFixtures(ctx, l, c, s.crdbNodes, s.version); err != nil { return err @@ -376,8 +364,7 @@ type waitForStableClusterVersionStep struct { nodes option.NodeListOption } -func (s waitForStableClusterVersionStep) ID() int { return s.id } -func (s waitForStableClusterVersionStep) DBNode() int { return noDBNeeded } +func (s waitForStableClusterVersionStep) ID() int { return s.id } func (s waitForStableClusterVersionStep) Description() string { return fmt.Sprintf( @@ -387,38 +374,38 @@ func (s waitForStableClusterVersionStep) Description() string { } func (s waitForStableClusterVersionStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { - return clusterupgrade.WaitForClusterUpgrade(ctx, l, s.nodes, dbFunc) + return clusterupgrade.WaitForClusterUpgrade(ctx, l, s.nodes, helper.Connect) } // preserveDowngradeOptionStep sets the `preserve_downgrade_option` // cluster setting to the binary version running in `node`. type preserveDowngradeOptionStep struct { - id int - node int + id int + crdbNodes option.NodeListOption + prng *rand.Rand } -func (s preserveDowngradeOptionStep) ID() int { return s.id } -func (s preserveDowngradeOptionStep) DBNode() int { return s.node } +func (s preserveDowngradeOptionStep) ID() int { return s.id } func (s preserveDowngradeOptionStep) Description() string { return "preventing auto-upgrades by setting `preserve_downgrade_option`" } func (s preserveDowngradeOptionStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { - db := dbFunc(s.node) - - l.Printf("checking binary version on node %d", s.node) + node, db := helper.RandomDB(s.prng, s.crdbNodes) + l.Printf("checking binary version (via node %d)", node) bv, err := clusterupgrade.BinaryVersion(db) if err != nil { return err } + node, db = helper.RandomDB(s.prng, s.crdbNodes) downgradeOption := bv.String() - l.Printf("setting `preserve_downgrade_option` to %s", downgradeOption) + l.Printf("setting `preserve_downgrade_option` to %s (via node %d)", downgradeOption, node) _, err = db.ExecContext(ctx, "SET CLUSTER SETTING cluster.preserve_downgrade_option = $1", downgradeOption) return err } @@ -434,15 +421,14 @@ type restartWithNewBinaryStep struct { node int } -func (s restartWithNewBinaryStep) ID() int { return s.id } -func (s restartWithNewBinaryStep) DBNode() int { return noDBNeeded } +func (s restartWithNewBinaryStep) ID() int { return s.id } func (s restartWithNewBinaryStep) Description() string { return fmt.Sprintf("restart node %d with binary version %s", s.node, versionMsg(s.version)) } func (s restartWithNewBinaryStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { return clusterupgrade.RestartNodesWithNewBinary( ctx, @@ -459,22 +445,22 @@ func (s restartWithNewBinaryStep) Run( // setting, allowing the upgrade migrations to run and the cluster // version to eventually reach the binary version on the nodes. type finalizeUpgradeStep struct { - id int - node int + id int + crdbNodes option.NodeListOption + prng *rand.Rand } -func (s finalizeUpgradeStep) ID() int { return s.id } -func (s finalizeUpgradeStep) DBNode() int { return s.node } +func (s finalizeUpgradeStep) ID() int { return s.id } func (s finalizeUpgradeStep) Description() string { return "finalize upgrade by resetting `preserve_downgrade_option`" } func (s finalizeUpgradeStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { - db := dbFunc(s.node) - l.Printf("resetting preserve_downgrade_option") + node, db := helper.RandomDB(s.prng, s.crdbNodes) + l.Printf("resetting preserve_downgrade_option (via node %d)", node) _, err := db.ExecContext(ctx, "RESET CLUSTER SETTING cluster.preserve_downgrade_option") return err } @@ -482,22 +468,23 @@ func (s finalizeUpgradeStep) Run( // runHookStep is a step used to run a user-provided hook (i.e., // callbacks passed to `OnStartup`, `InMixedVersion`, or `AfterTest`). type runHookStep struct { - id int - node int - hook versionUpgradeHook + id int + testContext Context + prng *rand.Rand + hook versionUpgradeHook } -func (s runHookStep) ID() int { return s.id } -func (s runHookStep) DBNode() int { return s.node } +func (s runHookStep) ID() int { return s.id } func (s runHookStep) Description() string { return fmt.Sprintf("run %q", s.hook.name) } func (s runHookStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { - return s.hook.fn(l, dbFunc(s.node)) + helper.SetContext(&s.testContext) + return s.hook.fn(l, s.prng, helper) } // sequentialRunStep is a "meta-step" that indicates that a sequence @@ -556,11 +543,6 @@ func prefixedLogger(l *logger.Logger, prefix string) (*logger.Logger, error) { return l.ChildLogger(fileName, logger.LogPrefix(formattedPrefix)) } -func randomNode(rng *rand.Rand, nodes option.NodeListOption) int { - idx := rng.Intn(len(nodes)) - return nodes[idx] -} - func (h hooks) Filter(testContext Context) hooks { var result hooks for _, hook := range h { @@ -577,11 +559,12 @@ func (h hooks) Filter(testContext Context) hooks { // returned. Otherwise, a `concurrentRunStep` is returned, where every // hook is run concurrently. func (h hooks) AsSteps( - label string, idGen func() int, rng *rand.Rand, nodes option.NodeListOption, + label string, idGen func() int, prng *rand.Rand, nodes option.NodeListOption, testContext Context, ) []testStep { steps := make([]testStep, 0, len(h)) for _, hook := range h { - rhs := runHookStep{id: idGen(), node: randomNode(rng, nodes), hook: hook} + hookPrng := rngFromRNG(prng) + rhs := runHookStep{id: idGen(), prng: hookPrng, hook: hook, testContext: testContext} steps = append(steps, rhs) } @@ -589,7 +572,7 @@ func (h hooks) AsSteps( return steps } - return []testStep{newConcurrentRunStep(label, steps, rng)} + return []testStep{newConcurrentRunStep(label, steps, prng)} } func (th *testHooks) AddStartup(hook versionUpgradeHook) { @@ -604,16 +587,16 @@ func (th *testHooks) AddAfterUpgradeFinalized(hook versionUpgradeHook) { th.afterUpgradeFinalized = append(th.afterUpgradeFinalized, hook) } -func (th *testHooks) StartupSteps(idGen func() int) []testStep { - return th.startup.AsSteps(startupLabel, idGen, th.prng, th.crdbNodes) +func (th *testHooks) StartupSteps(idGen func() int, testContext Context) []testStep { + return th.startup.AsSteps(startupLabel, idGen, th.prng, th.crdbNodes, testContext) } func (th *testHooks) MixedVersionSteps(testContext Context, idGen func() int) []testStep { - return th.mixedVersion.Filter(testContext).AsSteps(mixedVersionLabel, idGen, th.prng, th.crdbNodes) + return th.mixedVersion.Filter(testContext).AsSteps(mixedVersionLabel, idGen, th.prng, th.crdbNodes, testContext) } -func (th *testHooks) AfterUpgradeFinalizedSteps(idGen func() int) []testStep { - return th.afterUpgradeFinalized.AsSteps(afterTestLabel, idGen, th.prng, th.crdbNodes) +func (th *testHooks) AfterUpgradeFinalizedSteps(idGen func() int, testContext Context) []testStep { + return th.afterUpgradeFinalized.AsSteps(afterTestLabel, idGen, th.prng, th.crdbNodes, testContext) } func randomDelay(rng *rand.Rand) time.Duration { @@ -621,6 +604,10 @@ func randomDelay(rng *rand.Rand) time.Duration { return time.Duration(possibleDelaysMs[idx]) * time.Millisecond } +func rngFromRNG(rng *rand.Rand) *rand.Rand { + return rand.New(rand.NewSource(rng.Int63())) +} + func versionMsg(version string) string { return clusterupgrade.VersionMsg(version) } diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go index 7a3c58922d62..0fbabe2c39f3 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go @@ -95,17 +95,33 @@ func (p *testPlanner) Plan() *TestPlan { } } +func (p *testPlanner) initialContext() Context { + return Context{ + FromVersion: p.initialVersion, + ToVersion: clusterupgrade.MainVersion, + FromVersionNodes: p.crdbNodes, + } +} + +func (p *testPlanner) finalContext(finalizing bool) Context { + return Context{ + FromVersion: p.initialVersion, + ToVersion: clusterupgrade.MainVersion, + ToVersionNodes: p.crdbNodes, + Finalizing: finalizing, + } +} + // initSteps returns the sequence of steps that should be executed // before we start changing binaries on nodes in the process of // upgrading/downgrading. It will also run any startup hooks the user // may have provided. func (p *testPlanner) initSteps() []testStep { - preserveDowngradeNode := randomNode(p.prng, p.crdbNodes) return append([]testStep{ startFromCheckpointStep{id: p.nextID(), version: p.initialVersion, rt: p.rt, crdbNodes: p.crdbNodes}, waitForStableClusterVersionStep{id: p.nextID(), nodes: p.crdbNodes}, - preserveDowngradeOptionStep{id: p.nextID(), node: preserveDowngradeNode}, - }, p.hooks.StartupSteps(p.nextID)...) + preserveDowngradeOptionStep{id: p.nextID(), prng: p.newRNG(), crdbNodes: p.crdbNodes}, + }, p.hooks.StartupSteps(p.nextID, p.initialContext())...) } // finalSteps are the steps to be run once the nodes have been @@ -115,7 +131,7 @@ func (p *testPlanner) initSteps() []testStep { func (p *testPlanner) finalSteps() []testStep { return append([]testStep{ waitForStableClusterVersionStep{id: p.nextID(), nodes: p.crdbNodes}, - }, p.hooks.AfterUpgradeFinalizedSteps(p.nextID)...) + }, p.hooks.AfterUpgradeFinalizedSteps(p.nextID, p.finalContext(false /* finalizing */))...) } func (p *testPlanner) upgradeSteps(from, to string) []testStep { @@ -165,11 +181,9 @@ func (p *testPlanner) changeVersionSteps(from, to, label string) []testStep { // `preserve_downgrade_option` and potentially running mixed-version // hooks while the cluster version is changing. func (p *testPlanner) finalizeUpgradeSteps() []testStep { - testContext := Context{Finalizing: true} - finalizeNode := randomNode(p.prng, p.crdbNodes) return append([]testStep{ - finalizeUpgradeStep{id: p.nextID(), node: finalizeNode}, - }, p.hooks.MixedVersionSteps(testContext, p.nextID)...) + finalizeUpgradeStep{id: p.nextID(), prng: p.newRNG(), crdbNodes: p.crdbNodes}, + }, p.hooks.MixedVersionSteps(p.finalContext(true /* finalizing */), p.nextID)...) } func (p *testPlanner) nextID() int { @@ -177,6 +191,10 @@ func (p *testPlanner) nextID() int { return p.stepCount } +func (p *testPlanner) newRNG() *rand.Rand { + return rngFromRNG(p.prng) +} + // PrettyPrint displays a tree-like view of the mixed-version test // plan, useful when debugging mixed-version test failures. Each step // is assigned an ID, making it easy to correlate the step that @@ -211,10 +229,6 @@ func (plan *TestPlan) prettyPrintStep(out *strings.Builder, step testStep, prefi // concurrent execution), and what database node the step is // connecting to. writeSingle := func(rs singleStep, extraContext ...string) { - if node := rs.DBNode(); node != noDBNeeded { - dbinfo := fmt.Sprintf("with connection to node %d", node) - extraContext = append([]string{dbinfo}, extraContext...) - } var extras string if contextStr := strings.Join(extraContext, ", "); contextStr != "" { extras = ", " + contextStr diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go index b7d560b8c2ba..c078aaf62f74 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go @@ -12,7 +12,6 @@ package mixedversion import ( "context" - gosql "database/sql" "fmt" "io" "math/rand" @@ -81,31 +80,31 @@ func TestTestPlanner(t *testing.T) { mixed-version test plan for upgrading from %[1]s to : ├── starting cluster from fixtures for version "%[1]s" (1) ├── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (2) -├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+`, with connection to node 4 (3) +├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+` (3) ├── upgrade nodes :1-4 from "%[1]s" to "" │ ├── restart node 2 with binary version (4) │ ├── restart node 1 with binary version (5) -│ ├── run "mixed-version 1", with connection to node 3 (6) +│ ├── run "mixed-version 1" (6) │ ├── restart node 4 with binary version (7) │ ├── restart node 3 with binary version (8) -│ └── run "mixed-version 2", with connection to node 3 (9) +│ └── run "mixed-version 2" (9) ├── downgrade nodes :1-4 from "" to "%[1]s" │ ├── restart node 3 with binary version %[1]s (10) │ ├── restart node 4 with binary version %[1]s (11) │ ├── run mixed-version hooks concurrently -│ │ ├── run "mixed-version 1", with connection to node 1, after 200ms delay (12) -│ │ └── run "mixed-version 2", with connection to node 1, after 200ms delay (13) +│ │ ├── run "mixed-version 1", after 200ms delay (12) +│ │ └── run "mixed-version 2", after 200ms delay (13) │ ├── restart node 2 with binary version %[1]s (14) │ └── restart node 1 with binary version %[1]s (15) ├── upgrade nodes :1-4 from "%[1]s" to "" │ ├── restart node 3 with binary version (16) -│ ├── run "mixed-version 1", with connection to node 1 (17) +│ ├── run "mixed-version 1" (17) │ ├── restart node 4 with binary version (18) │ ├── restart node 1 with binary version (19) │ ├── restart node 2 with binary version (20) -│ └── run "mixed-version 2", with connection to node 2 (21) -├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+`, with connection to node 3 (22) -├── run "mixed-version 2", with connection to node 1 (23) +│ └── run "mixed-version 2" (21) +├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+` (22) +├── run "mixed-version 2" (23) └── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (24) `, previousVersion, ) @@ -184,6 +183,6 @@ func requireConcurrentHooks(t *testing.T, step testStep, names ...string) { } } -func dummyHook(*logger.Logger, *gosql.DB) error { +func dummyHook(*logger.Logger, *rand.Rand, *Helper) error { return nil } diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go index 07422d12195c..7f9826f6fe09 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go @@ -14,6 +14,7 @@ import ( "context" gosql "database/sql" "fmt" + "math/rand" "os" "path/filepath" "regexp" @@ -31,6 +32,17 @@ import ( ) type ( + // Helper is the struct passed to user-functions providing helper + // functions that mixed-version tests can use. + Helper struct { + ctx context.Context + testContext *Context + conns []*gosql.DB + crdbNodes option.NodeListOption + + stepLogger *logger.Logger + } + testRunner struct { ctx context.Context plan *TestPlan @@ -86,22 +98,28 @@ func (tr *testRunner) run() error { // runStep contains the logic of running a single test step, called // recursively in the case of sequentialRunStep and concurrentRunStep. func (tr *testRunner) runStep(step testStep) error { - if ss, ok := step.(singleStep); ok && ss.ID() > 1 { - // if we are running a singleStep that is *not* the first step, - // we can initialize the database connections. This represents the - // assumption that the first step in the test plan is the one that - // sets up binaries and make the `cockroach` process available on - // the nodes. - // TODO(renato): consider a way to make the test runner crash if - // the assumption does not hold - if err := tr.maybeInitConnections(); err != nil { - return err - } - if err := tr.refreshBinaryVersions(); err != nil { - return err - } - if err := tr.refreshClusterVersions(); err != nil { - return err + if ss, ok := step.(singleStep); ok { + if ss.ID() == 1 { + // if this is the first singleStep of the plan, ensure it is an + // "initialization step" (i.e., cockroach nodes are ready after + // it executes). This is an assumption of the test runner and + // makes for clear error messages if that assumption is broken. + if err := tr.ensureInitializationStep(ss); err != nil { + return err + } + } else { + // update the runner's view of the cluster's binary and cluster + // versions before every non-initialization `singleStep` is + // executed + if err := tr.maybeInitConnections(); err != nil { + return err + } + if err := tr.refreshBinaryVersions(); err != nil { + return err + } + if err := tr.refreshClusterVersions(); err != nil { + return err + } } } @@ -136,12 +154,13 @@ func (tr *testRunner) runStep(step testStep) error { } tr.logStep("STARTING", ss, stepLogger) + tr.logVersions(stepLogger) start := timeutil.Now() defer func() { prefix := fmt.Sprintf("FINISHED [%s]", timeutil.Since(start)) tr.logStep(prefix, ss, stepLogger) }() - if err := ss.Run(tr.ctx, stepLogger, tr.cluster, tr.conn); err != nil { + if err := ss.Run(tr.ctx, stepLogger, tr.cluster, tr.newHelper(stepLogger)); err != nil { return tr.reportError(err, ss, stepLogger) } @@ -188,6 +207,18 @@ func (tr *testRunner) logStep(prefix string, step singleStep, l *logger.Logger) l.Printf("%[1]s %s (%d): %s %[1]s", dashes, prefix, step.ID(), step.Description()) } +// logVersions writes the current cached versions of the binary and +// cluster versions on each node. The cached versions should exist for +// all steps but the first one (when we start the cluster itself). +func (tr *testRunner) logVersions(l *logger.Logger) { + if tr.binaryVersions == nil || tr.clusterVersions == nil { + return + } + + l.Printf("binary versions: %s", formatVersions(tr.binaryVersions)) + l.Printf("cluster versions: %s", formatVersions(tr.clusterVersions)) +} + // loggerFor creates a logger instance to be used by a test step. Logs // will be available under `mixed-version-test/{ID}.log`, making it // easy to go from the IDs displayed in the test plan to the @@ -231,6 +262,15 @@ func (tr *testRunner) refreshClusterVersions() error { return nil } +func (tr *testRunner) ensureInitializationStep(ss singleStep) error { + _, isInit := ss.(startFromCheckpointStep) + if !isInit { + return fmt.Errorf("unexpected initialization type in mixed-version test: %T", ss) + } + + return nil +} + // maybeInitConnections initialize connections if the connection cache // is empty. func (tr *testRunner) maybeInitConnections() error { @@ -251,6 +291,15 @@ func (tr *testRunner) maybeInitConnections() error { return nil } +func (tr *testRunner) newHelper(l *logger.Logger) *Helper { + return &Helper{ + ctx: tr.ctx, + conns: tr.connCache, + crdbNodes: tr.crdbNodes, + stepLogger: l, + } +} + // conn returns a database connection to the given node. Assumes the // connection cache has been previously initialized. func (tr *testRunner) conn(node int) *gosql.DB { @@ -265,6 +314,48 @@ func (tr *testRunner) closeConnections() { } } +// RandomDB returns a (nodeID, connection) tuple for a randomly picked +// cockroach node according to the parameters passed. +func (h *Helper) RandomDB(prng *rand.Rand, nodes option.NodeListOption) (int, *gosql.DB) { + node := nodes[prng.Intn(len(nodes))] + return node, h.Connect(node) +} + +// QueryRow performs `db.QueryRowContext` on a randomly picked +// database node. The query and the node picked are logged in the logs +// of the step that calls this function. +func (h *Helper) QueryRow(rng *rand.Rand, query string, args ...interface{}) *gosql.Row { + node, db := h.RandomDB(rng, h.crdbNodes) + h.stepLogger.Printf("running SQL statement:\n%s\nArgs: %v\nNode: %d", query, args, node) + return db.QueryRowContext(h.ctx, query, args...) +} + +// Exec performs `db.ExecContext` on a randomly picked database node. +// The query and the node picked are logged in the logs of the step +// that calls this function. +func (h *Helper) Exec(rng *rand.Rand, query string, args ...interface{}) error { + node, db := h.RandomDB(rng, h.crdbNodes) + h.stepLogger.Printf("running SQL statement:\n%s\nArgs: %v\nNode: %d", query, args, node) + _, err := db.ExecContext(h.ctx, query, args...) + return err +} + +func (h *Helper) Connect(node int) *gosql.DB { + return h.conns[node-1] +} + +// SetContext should be called by steps that need access to the test +// context, as that is only visible to them. +func (h *Helper) SetContext(c *Context) { + h.testContext = c +} + +// Context returns the test context associated with a certain step. It +// is made available for user-functions (see runHookStep). +func (h *Helper) Context() *Context { + return h.testContext +} + func renameFailedLogger(l *logger.Logger) error { currentFileName := l.File.Name() newLogName := strings.TrimSuffix(currentFileName, filepath.Ext(currentFileName)) diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go index baf111b90966..cfc4faf8265f 100644 --- a/pkg/cmd/roachtest/tests/versionupgrade.go +++ b/pkg/cmd/roachtest/tests/versionupgrade.go @@ -14,6 +14,7 @@ import ( "context" gosql "database/sql" "fmt" + "math/rand" "path/filepath" "runtime" "time" @@ -103,20 +104,17 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { } mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All()) - mvt.InMixedVersion("run backup", func(l *logger.Logger, db *gosql.DB) error { + mvt.InMixedVersion("run backup", func(l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { // Verify that backups can be created in various configurations. This is // important to test because changes in system tables might cause backups to // fail in mixed-version clusters. dest := fmt.Sprintf("nodelocal://0/%d", timeutil.Now().UnixNano()) - l.Printf("writing backup to %s", dest) - _, err := db.ExecContext(ctx, `BACKUP TO $1`, dest) - return err + return h.Exec(rng, `BACKUP TO $1`, dest) }) - mvt.InMixedVersion("test features", func(l *logger.Logger, db *gosql.DB) error { + mvt.InMixedVersion("test features", func(l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { for _, featureTest := range versionUpgradeTestFeatures { l.Printf("running feature test %q", featureTest.name) - _, err := db.ExecContext(ctx, featureTest.statement) - if err != nil { + if err := h.Exec(rng, featureTest.statement); err != nil { l.Printf("%q: ERROR (%s)", featureTest.name, err) return err } @@ -125,7 +123,7 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { return nil }) - mvt.AfterUpgradeFinalized("check if GC TTL is pinned", func(l *logger.Logger, db *gosql.DB) error { + mvt.AfterUpgradeFinalized("check if GC TTL is pinned", func(l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { // TODO(irfansharif): This can be removed when the predecessor version // in this test is v23.1, where the default is 4h. This test was only to // make sure that existing clusters that upgrade to 23.1 retained their @@ -139,7 +137,7 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { WHERE target = 'RANGE default' LIMIT 1 ` - if err := db.QueryRowContext(ctx, query).Scan(&ttlSeconds); err != nil { + if err := h.QueryRow(rng, query).Scan(&ttlSeconds); err != nil { return fmt.Errorf("error querying GC TTL: %w", err) } expectedTTL := 24 * 60 * 60 // NB: 24h is what's used in the fixture