From 2c3e868abdb2b5e221207e4404f55bcdf50b5e89 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Thu, 22 Jun 2023 16:32:55 -0400 Subject: [PATCH] roachtest: validate cluster when using fixtures and provide opt-out This commit updates the `mixedversion` API with two main changes: * cluster validation: previously, mixed-version tests would always start from checked-in fixtures. Those fixtures have an implied cluster topology and require the cluster to have 4 cockroach nodes -- if that's not the case, the cluster fails to start up with non-obvious error messages. To make this situation clearer, we now validate that the test is being run with the correct number of nodes. * to allow for larger-scale mixed-version tests (more than 4 nodes), we now expose a `DisableFixtures` option that can be passed to `NewTest`. With this option, nodes start with an empty store directory, eliminating any restrictions on cluster size. Epic: CRDB-19321 Release note: None --- go.mod | 2 +- .../clusterupgrade/clusterupgrade.go | 10 +- .../roachtestutil/mixedversion/BUILD.bazel | 2 + .../mixedversion/mixedversion.go | 168 +++++++++++++----- .../mixedversion/mixedversion_test.go | 50 ++++++ .../roachtestutil/mixedversion/planner.go | 27 ++- .../mixedversion/planner_test.go | 113 +++++++----- .../roachtestutil/mixedversion/runner.go | 19 +- pkg/cmd/roachtest/tests/versionupgrade.go | 2 +- 9 files changed, 276 insertions(+), 117 deletions(-) create mode 100644 pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go diff --git a/go.mod b/go.mod index a50951646de2..a8fe268bd4e3 100644 --- a/go.mod +++ b/go.mod @@ -196,6 +196,7 @@ require ( github.com/pierrre/geohash v1.0.0 github.com/pires/go-proxyproto v0.7.0 github.com/pkg/browser v0.0.0-20210115035449-ce105d075bb4 + github.com/pkg/errors v0.9.1 github.com/pmezard/go-difflib v1.0.0 github.com/pressly/goose/v3 v3.5.3 github.com/prometheus/client_golang v1.12.1 @@ -361,7 +362,6 @@ require ( github.com/opencontainers/go-digest v1.0.0 // indirect github.com/openzipkin/zipkin-go v0.2.5 // indirect github.com/pelletier/go-toml v1.9.3 // indirect - github.com/pkg/errors v0.9.1 // indirect github.com/pkg/profile v1.6.0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/pquerna/cachecontrol v0.0.0-20200921180117-858c6e7e6b7e // indirect diff --git a/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go b/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go index f3c0ecff1aff..e603fc0e6f1c 100644 --- a/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go +++ b/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go @@ -117,7 +117,10 @@ func UploadVersion( func InstallFixtures( ctx context.Context, l *logger.Logger, c cluster.Cluster, nodes option.NodeListOption, v string, ) error { - c.Run(ctx, nodes, "mkdir -p {store-dir}") + if err := c.RunE(ctx, nodes, "mkdir -p {store-dir}"); err != nil { + return fmt.Errorf("creating store-dir: %w", err) + } + vv := version.MustParse("v" + v) // The fixtures use cluster version (major.minor) but the input might be // a patch release. @@ -133,7 +136,10 @@ func InstallFixtures( } } // Extract fixture. Fail if there's already an LSM in the store dir. - c.Run(ctx, nodes, "ls {store-dir}/marker.* 1> /dev/null 2>&1 && exit 1 || (cd {store-dir} && tar -xf fixture.tgz)") + if err := c.RunE(ctx, nodes, "ls {store-dir}/marker.* 1> /dev/null 2>&1 && exit 1 || (cd {store-dir} && tar -xf fixture.tgz)"); err != nil { + return fmt.Errorf("extracting fixtures: %w", err) + } + return nil } diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel b/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel index 7fd3c00f33a1..3d1aa17b676e 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel @@ -23,12 +23,14 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/version", + "@com_github_pkg_errors//:errors", ], ) go_test( name = "mixedversion_test", srcs = [ + "mixedversion_test.go", "planner_test.go", "runner_test.go", ], diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go index 8f0754f82d8f..8500a4e5342e 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go @@ -35,22 +35,26 @@ // // Typical usage: // -// mvt, err := mixedversion.NewTest(...) -// mvt.InMixedVersion("test my feature", func(l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { -// l.Printf("testing feature X") -// node, db := h.RandomDB(rng, c.All()) -// l.Printf("running query on node %d", node) -// _, err := db.ExecContext(ctx, "SELECT * FROM test") -// return err -// }) -// mvt.InMixedVersion("test another feature", func(l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { -// l.Printf("testing feature Y") -// node, db := h.RandomDB(rng, c.All()) -// l.Printf("running query on node %d", node) -// _, err := db.ExecContext(ctx, "SELECT * FROM test2") -// return err -// }) -// mvt.Run() +// mvt, err := mixedversion.NewTest(...) +// mvt.InMixedVersion("test my feature", func( +// ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper, +// ) error { +// l.Printf("testing feature X") +// node, db := h.RandomDB(rng, c.All()) +// l.Printf("running query on node %d", node) +// _, err := db.ExecContext(ctx, "SELECT * FROM test") +// return err +// }) +// mvt.InMixedVersion("test another feature", func( +// ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper, +// ) error { +// l.Printf("testing feature Y") +// node, db := h.RandomDB(rng, c.All()) +// l.Printf("running query on node %d", node) +// _, err := db.ExecContext(ctx, "SELECT * FROM test2") +// return err +// }) +// mvt.Run() // // Functions passed to `InMixedVersion` will be called at arbitrary // points during an upgrade/downgrade process. They may also be called @@ -84,6 +88,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/version" + "github.com/pkg/errors" ) const ( @@ -101,6 +106,11 @@ const ( // finalized. runWhileMigratingProbability = 0.5 + // numNodesInFixtures is the number of nodes expected to exist in a + // cluster that can use the test fixtures in + // `pkg/cmd/roachtest/fixtures`. + numNodesInFixtures = 4 + // CurrentCockroachPath is the path to the binary where the current // version of cockroach being tested is located. This file is // uploaded before any user functions are run. The primary use case @@ -122,6 +132,12 @@ var ( defaultClusterSettings = []install.ClusterSettingOption{ install.SecureOption(true), } + + defaultTestOptions = testOptions{ + // We use fixtures more often than not as they are more likely to + // detect bugs, especially in migrations. + useFixturesProbability: 0.7, + } ) type ( @@ -214,6 +230,14 @@ type ( crdbNodes option.NodeListOption } + // testOptions contains some options that can be changed by the user + // that expose some control over the generated test plan. + testOptions struct { + useFixturesProbability float64 + } + + customOption func(*testOptions) + // Test is the main struct callers of this package interact with. Test struct { ctx context.Context @@ -222,6 +246,8 @@ type ( logger *logger.Logger crdbNodes option.NodeListOption + options testOptions + rt test.Test prng *rand.Rand seed int64 @@ -252,6 +278,22 @@ type ( StopFunc func() ) +// NeverUseFixtures is an option that can be passed to `NewTest` to +// disable the use of fixtures in the test. Necessary if the test +// wants to use a number of cockroach nodes other than 4. +func NeverUseFixtures(opts *testOptions) { + opts.useFixturesProbability = 0 +} + +// AlwaysUseFixtures is an option that can be passed to `NewTest` to +// force the test to always start the cluster from the fixtures in +// `pkg/cmd/roachtest/fixtures`. Necessary if the test makes +// assertions that rely on the existence of data present in the +// fixtures. +func AlwaysUseFixtures(opts *testOptions) { + opts.useFixturesProbability = 1 +} + // NewTest creates a Test struct that users can use to create and run // a mixed-version roachtest. func NewTest( @@ -260,27 +302,37 @@ func NewTest( l *logger.Logger, c cluster.Cluster, crdbNodes option.NodeListOption, + options ...customOption, ) *Test { testLogger, err := prefixedLogger(l, logPrefix) if err != nil { t.Fatal(err) } + opts := defaultTestOptions + for _, fn := range options { + fn(&opts) + } + prng, seed := randutil.NewLockedPseudoRand() testLogger.Printf("mixed-version random seed: %d", seed) testCtx, cancel := context.WithCancel(ctx) - return &Test{ + test := &Test{ ctx: testCtx, cancel: cancel, cluster: c, logger: testLogger, crdbNodes: crdbNodes, + options: opts, rt: t, prng: prng, seed: seed, hooks: &testHooks{prng: prng, crdbNodes: crdbNodes}, } + + assertValidTest(test, t.Fatal) + return test } // RNG returns the underlying random number generator used by the @@ -433,6 +485,7 @@ func (t *Test) plan() (*TestPlan, error) { planner := testPlanner{ initialVersion: previousRelease, + options: t.options, rt: t.rt, crdbNodes: t.crdbNodes, hooks: t.hooks, @@ -458,32 +511,47 @@ func (t *Test) runCommandFunc(nodes option.NodeListOption, cmd string) userFunc } } -// startFromCheckpointStep is the step that starts the cluster from a -// specific `version`, using checked-in fixtures. -type startFromCheckpointStep struct { +// installFixturesStep is the step that copies the fixtures from +// `pkg/cmd/roachtest/fixtures` for a specific version into the nodes' +// store dir. +type installFixturesStep struct { id int - rt test.Test version string crdbNodes option.NodeListOption } -func (s startFromCheckpointStep) ID() int { return s.id } -func (s startFromCheckpointStep) Background() shouldStop { return nil } +func (s installFixturesStep) ID() int { return s.id } +func (s installFixturesStep) Background() shouldStop { return nil } -func (s startFromCheckpointStep) Description() string { - return fmt.Sprintf("starting cluster from fixtures for version %q", s.version) +func (s installFixturesStep) Description() string { + return fmt.Sprintf("installing fixtures for version %q", s.version) } -// Run will copy the fixtures to all database nodes in the cluster, -// 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, helper *Helper, +func (s installFixturesStep) Run( + ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper, ) error { - if err := clusterupgrade.InstallFixtures(ctx, l, c, s.crdbNodes, s.version); err != nil { - return err - } + return clusterupgrade.InstallFixtures(ctx, l, c, s.crdbNodes, s.version) +} +// startStep is the step that starts the cluster from a specific +// `version`. +type startStep struct { + id int + rt test.Test + version string + crdbNodes option.NodeListOption +} + +func (s startStep) ID() int { return s.id } +func (s startStep) Background() shouldStop { return nil } + +func (s startStep) Description() string { + return fmt.Sprintf("starting cluster at version %q", s.version) +} + +// Run uploads the binary associated with the given version and starts +// the cockroach binary on the nodes. +func (s startStep) Run(ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper) error { binaryPath, err := clusterupgrade.UploadVersion(ctx, s.rt, l, c, s.crdbNodes, s.version) if err != nil { return err @@ -517,7 +585,7 @@ func (s uploadCurrentVersionStep) Description() string { } func (s uploadCurrentVersionStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, + ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper, ) error { _, err := clusterupgrade.UploadVersion(ctx, s.rt, l, c, s.crdbNodes, clusterupgrade.MainVersion) if err != nil { @@ -547,9 +615,9 @@ func (s waitForStableClusterVersionStep) Description() string { } func (s waitForStableClusterVersionStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, + ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper, ) error { - return clusterupgrade.WaitForClusterUpgrade(ctx, l, s.nodes, helper.Connect) + return clusterupgrade.WaitForClusterUpgrade(ctx, l, s.nodes, h.Connect) } // preserveDowngradeOptionStep sets the `preserve_downgrade_option` @@ -568,16 +636,16 @@ func (s preserveDowngradeOptionStep) Description() string { } func (s preserveDowngradeOptionStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, + ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper, ) error { - node, db := helper.RandomDB(s.prng, s.crdbNodes) + node, db := h.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) + node, db = h.RandomDB(s.prng, s.crdbNodes) downgradeOption := bv.String() 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) @@ -603,7 +671,7 @@ func (s restartWithNewBinaryStep) Description() string { } func (s restartWithNewBinaryStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, + ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper, ) error { return clusterupgrade.RestartNodesWithNewBinary( ctx, @@ -639,9 +707,9 @@ func (s finalizeUpgradeStep) Description() string { } func (s finalizeUpgradeStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, + ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper, ) error { - node, db := helper.RandomDB(s.prng, s.crdbNodes) + node, db := h.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 @@ -665,10 +733,10 @@ func (s runHookStep) Description() string { } func (s runHookStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, + ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper, ) error { - helper.SetContext(&s.testContext) - return s.hook.fn(ctx, l, s.prng, helper) + h.SetContext(&s.testContext) + return s.hook.fn(ctx, l, s.prng, h) } // sequentialRunStep is a "meta-step" that indicates that a sequence @@ -821,3 +889,13 @@ func rngFromRNG(rng *rand.Rand) *rand.Rand { func versionMsg(version string) string { return clusterupgrade.VersionMsg(version) } + +func assertValidTest(test *Test, fatalFunc func(...interface{})) { + if test.options.useFixturesProbability > 0 && len(test.crdbNodes) != numNodesInFixtures { + err := fmt.Errorf( + "invalid cluster: use of fixtures requires %d cockroach nodes, got %d (%v)", + numNodesInFixtures, len(test.crdbNodes), test.crdbNodes, + ) + fatalFunc(errors.Wrap(err, "mixedversion.NewTest")) + } +} diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go new file mode 100644 index 000000000000..87e5d37bb820 --- /dev/null +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go @@ -0,0 +1,50 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package mixedversion + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" + "github.com/stretchr/testify/require" +) + +func Test_assertValidTest(t *testing.T) { + var fatalErr error + fatalFunc := func() func(...interface{}) { + fatalErr = nil + return func(args ...interface{}) { + require.Len(t, args, 1) + err, isErr := args[0].(error) + require.True(t, isErr) + + fatalErr = err + } + } + + notEnoughNodes := option.NodeListOption{1, 2, 3} + tooManyNodes := option.NodeListOption{1, 2, 3, 5, 6} + + for _, crdbNodes := range []option.NodeListOption{notEnoughNodes, tooManyNodes} { + mvt := newTest() + mvt.crdbNodes = crdbNodes + + assertValidTest(mvt, fatalFunc()) + require.Error(t, fatalErr) + require.Contains(t, fatalErr.Error(), "mixedversion.NewTest: invalid cluster: use of fixtures requires 4 cockroach nodes") + + mvt = newTest(NeverUseFixtures) + mvt.crdbNodes = crdbNodes + + assertValidTest(mvt, fatalFunc()) + require.NoError(t, fatalErr) + } +} diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go index b3cb6673e984..f4594c06781a 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go @@ -28,6 +28,7 @@ type ( TestPlan struct { initialVersion string finalVersion string + startClusterID int steps []testStep } @@ -35,9 +36,11 @@ type ( // a test plan from the given rng and user-provided hooks. testPlanner struct { stepCount int + startClusterID int initialVersion string crdbNodes option.NodeListOption rt test.Test + options testOptions hooks *testHooks prng *rand.Rand bgChans []shouldStop @@ -56,7 +59,7 @@ const ( // following high level outline: // // - start all nodes in the cluster from the predecessor version, -// using fixtures. +// maybe using fixtures. // - set `preserve_downgrade_option`. // - run startup hooks. // - upgrade all nodes to the current cockroach version (running @@ -94,6 +97,7 @@ func (p *testPlanner) Plan() *TestPlan { return &TestPlan{ initialVersion: p.initialVersion, finalVersion: versionMsg(clusterupgrade.MainVersion), + startClusterID: p.startClusterID, steps: steps, } } @@ -120,12 +124,21 @@ func (p *testPlanner) finalContext(finalizing bool) Context { // upgrading/downgrading. It will also run any startup hooks the user // may have provided. func (p *testPlanner) initSteps() []testStep { - return append([]testStep{ - startFromCheckpointStep{id: p.nextID(), version: p.initialVersion, rt: p.rt, crdbNodes: p.crdbNodes}, - uploadCurrentVersionStep{id: p.nextID(), rt: p.rt, crdbNodes: p.crdbNodes, dest: CurrentCockroachPath}, - waitForStableClusterVersionStep{id: p.nextID(), nodes: p.crdbNodes}, - preserveDowngradeOptionStep{id: p.nextID(), prng: p.newRNG(), crdbNodes: p.crdbNodes}, - }, p.hooks.StartupSteps(p.nextID, p.initialContext())...) + var steps []testStep + if p.prng.Float64() < p.options.useFixturesProbability { + steps = []testStep{installFixturesStep{id: p.nextID(), version: p.initialVersion, crdbNodes: p.crdbNodes}} + } + p.startClusterID = p.nextID() + steps = append(steps, startStep{id: p.startClusterID, version: p.initialVersion, rt: p.rt, crdbNodes: p.crdbNodes}) + + return append( + append(steps, + uploadCurrentVersionStep{id: p.nextID(), rt: p.rt, crdbNodes: p.crdbNodes, dest: CurrentCockroachPath}, + waitForStableClusterVersionStep{id: p.nextID(), nodes: p.crdbNodes}, + 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 diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go index 5ed1b42346f3..4e72a13dffa4 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go @@ -67,7 +67,7 @@ const ( ) func TestTestPlanner(t *testing.T) { - mvt := newTest(t) + mvt := newTest() mvt.InMixedVersion("mixed-version 1", dummyHook) mvt.InMixedVersion("mixed-version 2", dummyHook) initBank := roachtestutil.NewCommand("./cockroach workload bank init") @@ -80,50 +80,45 @@ func TestTestPlanner(t *testing.T) { plan, err := mvt.plan() require.NoError(t, err) - require.Len(t, plan.steps, 12) + require.Len(t, plan.steps, 11) // Assert on the pretty-printed version of the test plan as that // asserts the ordering of the steps we want to take, and as a bonus // tests the printing function itself. expectedPrettyPlan := fmt.Sprintf(` mixed-version test plan for upgrading from %[1]s to : -├── starting cluster from fixtures for version "%[1]s" (1) +├── starting cluster at version "%[1]s" (1) ├── upload current binary to all cockroach nodes (:1-4) (2) ├── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (3) ├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+` (4) ├── run "initialize bank workload" (5) ├── start background hooks concurrently │ ├── run "bank workload", after 50ms delay (6) -│ ├── run "rand workload", after 50ms delay (7) -│ └── run "csv server", after 200ms delay (8) +│ ├── run "rand workload", after 200ms delay (7) +│ └── run "csv server", after 500ms delay (8) ├── upgrade nodes :1-4 from "%[1]s" to "" -│ ├── restart node 4 with binary version (9) -│ ├── run mixed-version hooks concurrently -│ │ ├── run "mixed-version 1", after 100ms delay (10) -│ │ └── run "mixed-version 2", after 100ms delay (11) +│ ├── restart node 1 with binary version (9) +│ ├── run "mixed-version 1" (10) +│ ├── restart node 4 with binary version (11) │ ├── restart node 3 with binary version (12) -│ ├── restart node 2 with binary version (13) -│ └── restart node 1 with binary version (14) +│ ├── run "mixed-version 2" (13) +│ └── restart node 2 with binary version (14) ├── downgrade nodes :1-4 from "" to "%[1]s" -│ ├── restart node 2 with binary version %[1]s (15) -│ ├── run "mixed-version 1" (16) -│ ├── restart node 1 with binary version %[1]s (17) -│ ├── run "mixed-version 2" (18) -│ ├── restart node 3 with binary version %[1]s (19) -│ └── restart node 4 with binary version %[1]s (20) +│ ├── restart node 4 with binary version %[1]s (15) +│ ├── run "mixed-version 2" (16) +│ ├── restart node 2 with binary version %[1]s (17) +│ ├── restart node 3 with binary version %[1]s (18) +│ ├── restart node 1 with binary version %[1]s (19) +│ └── run "mixed-version 1" (20) ├── upgrade nodes :1-4 from "%[1]s" to "" │ ├── restart node 4 with binary version (21) -│ ├── restart node 3 with binary version (22) +│ ├── run "mixed-version 1" (22) │ ├── restart node 1 with binary version (23) -│ ├── run mixed-version hooks concurrently -│ │ ├── run "mixed-version 1", after 0s delay (24) -│ │ └── run "mixed-version 2", after 0s delay (25) -│ └── restart node 2 with binary version (26) +│ ├── restart node 2 with binary version (24) +│ ├── run "mixed-version 2" (25) +│ └── restart node 3 with binary version (26) ├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+` (27) -├── run mixed-version hooks concurrently -│ ├── run "mixed-version 1", after 100ms delay (28) -│ └── run "mixed-version 2", after 0s delay (29) -└── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (30) +└── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (28) `, previousVersion, ) @@ -133,7 +128,7 @@ mixed-version test plan for upgrading from %[1]s to : // Assert that startup hooks are scheduled to run before any // upgrades, i.e., after cluster is initialized (step 1), and after // we wait for the cluster version to match on all nodes (step 2). - mvt = newTest(t) + mvt = newTest() mvt.OnStartup("startup 1", dummyHook) mvt.OnStartup("startup 2", dummyHook) plan, err = mvt.plan() @@ -142,7 +137,7 @@ mixed-version test plan for upgrading from %[1]s to : // Assert that AfterUpgradeFinalized hooks are scheduled to run in // the last step of the test. - mvt = newTest(t) + mvt = newTest() mvt.AfterUpgradeFinalized("finalizer 1", dummyHook) mvt.AfterUpgradeFinalized("finalizer 2", dummyHook) mvt.AfterUpgradeFinalized("finalizer 3", dummyHook) @@ -156,7 +151,7 @@ mixed-version test plan for upgrading from %[1]s to : // the same seed multiple times yields the same plan every time. func TestDeterministicTestPlan(t *testing.T) { makePlan := func() *TestPlan { - mvt := newTest(t) + mvt := newTest() mvt.InMixedVersion("mixed-version 1", dummyHook) mvt.InMixedVersion("mixed-version 2", dummyHook) @@ -185,7 +180,7 @@ var unused float64 func TestDeterministicHookSeeds(t *testing.T) { generateData := func(generateMoreRandomNumbers bool) [][]int { var generatedData [][]int - mvt := newTest(t) + mvt := newTest() mvt.InMixedVersion("do something", func(_ context.Context, _ *logger.Logger, rng *rand.Rand, _ *Helper) error { var data []int for j := 0; j < 5; j++ { @@ -216,31 +211,26 @@ func TestDeterministicHookSeeds(t *testing.T) { // We can hardcode these paths since we are using a fixed seed in // these tests. - firstRun := plan.steps[4].(sequentialRunStep).steps[2].(runHookStep) + firstRun := plan.steps[4].(sequentialRunStep).steps[4].(runHookStep) require.Equal(t, "do something", firstRun.hook.name) require.NoError(t, firstRun.Run(ctx, nilLogger, nilCluster, emptyHelper)) - secondRun := plan.steps[5].(sequentialRunStep).steps[3].(runHookStep) + secondRun := plan.steps[5].(sequentialRunStep).steps[1].(runHookStep) require.Equal(t, "do something", secondRun.hook.name) require.NoError(t, secondRun.Run(ctx, nilLogger, nilCluster, emptyHelper)) - thirdRun := plan.steps[6].(sequentialRunStep).steps[1].(runHookStep) + thirdRun := plan.steps[6].(sequentialRunStep).steps[3].(runHookStep) require.Equal(t, "do something", thirdRun.hook.name) require.NoError(t, thirdRun.Run(ctx, nilLogger, nilCluster, emptyHelper)) - fourthRun := plan.steps[8].(runHookStep) - require.Equal(t, "do something", fourthRun.hook.name) - require.NoError(t, fourthRun.Run(ctx, nilLogger, nilCluster, emptyHelper)) - - require.Len(t, generatedData, 4) + require.Len(t, generatedData, 3) return generatedData } expectedData := [][]int{ - {82, 1, 17, 3, 87}, - {73, 17, 6, 37, 43}, - {82, 35, 57, 54, 8}, - {7, 95, 26, 31, 65}, + {97, 94, 35, 65, 21}, + {40, 30, 46, 88, 46}, + {96, 91, 48, 85, 76}, } const numRums = 50 for j := 0; j < numRums; j++ { @@ -250,12 +240,49 @@ func TestDeterministicHookSeeds(t *testing.T) { } } -func newTest(t *testing.T) *Test { +// Test_startClusterID tests that the plan generated by the test +// planner keeps track of the correct ID for the test's start step. +func Test_startClusterID(t *testing.T) { + // When fixtures are disabled, the startStep should always be the + // first step of the test (ID = 1). + mvt := newTest(NeverUseFixtures) + plan, err := mvt.plan() + require.NoError(t, err) + + step, isStartStep := plan.steps[0].(startStep) + require.True(t, isStartStep) + require.Equal(t, 1, step.ID()) + require.Equal(t, 1, plan.startClusterID) + + // Overwrite probability to 1 so that our test plan will always + // start the cluster from fixtures. + origProbability := defaultTestOptions.useFixturesProbability + defaultTestOptions.useFixturesProbability = 1 + defer func() { defaultTestOptions.useFixturesProbability = origProbability }() + + // When fixtures are used, the startStep should always be the second + // step of the test (ID = 2), after fixtures are installed. + mvt = newTest() + plan, err = mvt.plan() + require.NoError(t, err) + step, isStartStep = plan.steps[1].(startStep) + require.True(t, isStartStep) + require.Equal(t, 2, step.ID()) + require.Equal(t, 2, plan.startClusterID) +} + +func newTest(options ...customOption) *Test { + testOptions := defaultTestOptions + for _, fn := range options { + fn(&testOptions) + } + prng := rand.New(rand.NewSource(seed)) return &Test{ ctx: ctx, logger: nilLogger, crdbNodes: nodes, + options: testOptions, _buildVersion: buildVersion, prng: prng, hooks: &testHooks{prng: prng, crdbNodes: nodes}, diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go index f108de6055a0..693dbb838df0 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go @@ -173,15 +173,7 @@ func (tr *testRunner) run() error { // recursively in the case of sequentialRunStep and concurrentRunStep. func (tr *testRunner) runStep(ctx context.Context, step testStep) error { 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 { + if ss.ID() > tr.plan.startClusterID { // update the runner's view of the cluster's binary and cluster // versions before every non-initialization `singleStep` is // executed @@ -410,15 +402,6 @@ 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. When the function returns, either the `connCache` field // is populated with a connection for every crdb node, or the field is diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go index 3d435e94b21f..4299ed89369c 100644 --- a/pkg/cmd/roachtest/tests/versionupgrade.go +++ b/pkg/cmd/roachtest/tests/versionupgrade.go @@ -100,7 +100,7 @@ DROP TABLE splitmerge.t; func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.All()) - mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All()) + mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All(), mixedversion.AlwaysUseFixtures) mvt.OnStartup( "setup schema changer workload", func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error {