From 8d36b4176288f26c371e829622e76c8888cbfc4a Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Fri, 30 Aug 2024 17:01:11 -0400 Subject: [PATCH 1/4] roachtest: reset `defaultVirtualCluster` on reuse Epic: none Release note: None --- pkg/cmd/roachtest/cluster.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 4090bee7238d..dac262240799 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -3118,6 +3118,10 @@ func (c *clusterImpl) WipeForReuse( // particular, this overwrites the reuse policy to reflect what the test // intends to do with it. c.spec = newClusterSpec + // Reset the default virtual cluster before running a new test on + // this cluster. + c.defaultVirtualCluster = "" + return nil } From 7761c2fb5dd147fe338129cafa7a1fb6e35fbd90 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Tue, 3 Sep 2024 13:57:30 -0400 Subject: [PATCH 2/4] roachtest: pass an `ExpanderConfig` in the `RunWithDetails` API This API was missed when the ability to set a default virtual cluster was introduced. This meant that commands called using `c.RunWithDetails` (and sibling functions) would not expand in the same way, particularly when it comes to computing pgurls and ports. Epic: none Release note: None --- pkg/cmd/roachtest/cluster.go | 5 ++++- pkg/roachprod/install/cluster_synced.go | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index dac262240799..e2102773026d 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -2565,9 +2565,12 @@ func (c *clusterImpl) RunWithDetails( } l.Printf("> %s", cmd) + expanderCfg := install.ExpanderConfig{ + DefaultVirtualCluster: c.defaultVirtualCluster, + } results, err := roachprod.RunWithDetails( ctx, l, c.MakeNodes(nodes), "" /* SSHOptions */, "", /* processTag */ - c.IsSecure(), args, options, + c.IsSecure(), args, options.WithExpanderConfig(expanderCfg), ) var logFileFull string diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index b788a22a7742..e2f65e1d7e70 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -1404,6 +1404,7 @@ func (c *SyncedCluster) RunWithDetails( includeRoachprodEnvVars: true, stdout: l.Stdout, stderr: l.Stderr, + expanderConfig: options.ExpanderConfig, } result, err := c.runCmdOnSingleNode(ctx, l, node, cmd, opts) return result, err From 72c8c7eb0936b689a544d1f851ba80d0745c5d56 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Thu, 5 Sep 2024 16:30:21 -0400 Subject: [PATCH 3/4] roachtest: mixedversion: sanitize both log path and prefix Avoiding filenames with spaces on them is generally a good idea. Epic: none Release note: None --- .../roachtest/roachtestutil/mixedversion/mixedversion.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go index 9ba4784bfe9c..683f24c0f14d 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go @@ -1064,16 +1064,15 @@ func newSingleStep(context *Context, impl singleStepProtocol, rng *rand.Rand) *s // prefixedLogger returns a logger instance off of the given `l` // parameter. The path and prefix are the same. func prefixedLogger(l *logger.Logger, prefix string) (*logger.Logger, error) { - filename := sanitizePath(prefix) - return prefixedLoggerWithFilename(l, filename, filename) + return prefixedLoggerWithFilename(l, prefix, prefix) } // prefixedLoggerWithFilename returns a logger instance with the given // prefix. The logger will write to a file on the given `path`, // relative to the logger `l`'s location. func prefixedLoggerWithFilename(l *logger.Logger, prefix, path string) (*logger.Logger, error) { - formattedPrefix := fmt.Sprintf("[%s] ", prefix) - return l.ChildLogger(path, logger.LogPrefix(formattedPrefix)) + formattedPrefix := fmt.Sprintf("[%s] ", sanitizePath(prefix)) + return l.ChildLogger(sanitizePath(path), logger.LogPrefix(formattedPrefix)) } func sanitizePath(s string) string { From fcc447c02b67480fc67b6a2ee3fa87aa16466723 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Thu, 8 Aug 2024 16:31:19 -0400 Subject: [PATCH 4/4] roachtest: separate-process deployments in mixedversion This commit adds support `separate-process` deployments in `mixedversion` tests. It is a natural enhancement of previous work where we introduced `shared-process` deployments. When a test runs in a `separate-process` deployment mode, each upgrade will involve restarting the binaries for both the system tenant (also known as the storage cluster), as well as for the application (non-system) tenant. The storage cluster needs to finalize its upgrade before the application tenant starts upgrading. Mixed-version functions can be called while the system tenant is upgrading and/or while the application tenant is upgrading. For simplicity, only one separate-process tenant is created, and handles traffic generated by the test. In addition, a SQL server is started on every node where we run a storage cluster binary. In other words, the two processes are colocated (without any form of resource allocation) on the same nodes. For now, separate-process deployments are disabled on a number of tests where it's known to fail. We should, over time, investigate these failures and re-enable the new deployment mode on them. Epic: none Release note: None --- pkg/cmd/roachtest/cluster.go | 16 +- pkg/cmd/roachtest/option/options.go | 11 + .../clusterupgrade/clusterupgrade.go | 29 +- .../roachtestutil/mixedversion/BUILD.bazel | 2 + .../roachtestutil/mixedversion/context.go | 23 +- .../roachtestutil/mixedversion/helper.go | 4 + .../mixedversion/mixedversion.go | 142 +++-- .../mixedversion/mixedversion_test.go | 52 ++ .../roachtestutil/mixedversion/planner.go | 537 +++++++++++++----- .../mixedversion/planner_test.go | 30 +- .../roachtestutil/mixedversion/runner.go | 265 +++++---- .../roachtestutil/mixedversion/steps.go | 241 +++++--- .../planner/basic_test_mixed_version_hooks | 6 +- .../testdata/planner/cluster_setting | 8 +- .../testdata/planner/conflicting_mutators | 4 +- .../planner/local_runs_reduced_wait_time | 8 +- .../planner/minimum_supported_version | 12 +- .../testdata/planner/multiple_upgrades | 8 +- .../testdata/planner/mutator_probabilities | 4 +- .../preserve_downgrade_option_randomizer | 12 +- .../testdata/planner/separate_process | 194 +++++++ .../testdata/planner/shared_process | 140 ++--- .../testdata/planner/skip_version_upgrades | 8 +- .../mixedversion/testdata/planner/step_stages | 12 +- pkg/cmd/roachtest/tests/follower_reads.go | 27 +- .../roachtest/tests/mixed_version_backup.go | 47 +- pkg/cmd/roachtest/tests/mixed_version_cdc.go | 21 +- .../roachtest/tests/mixed_version_import.go | 6 + pkg/cmd/roachtest/tests/rebalance_load.go | 8 + ...ate_system_schema_after_version_upgrade.go | 39 +- pkg/cmd/roachtest/tests/versionupgrade.go | 13 +- pkg/roachprod/install/cockroach.go | 2 +- 32 files changed, 1327 insertions(+), 604 deletions(-) create mode 100644 pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/separate_process diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index e2102773026d..680549d82d9e 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1333,7 +1333,11 @@ func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, l *logger.Logger) sec = fmt.Sprintf("--certs-dir=%s", certs) } if err := c.RunE( - ctx, option.WithNodes(c.Node(node)), fmt.Sprintf("%s debug tsdump %s --port={pgport%s} --format=raw > tsdump.gob", test.DefaultCockroachPath, sec, c.Node(node)), + ctx, option.WithNodes(c.Node(node)), + fmt.Sprintf( + "%s debug tsdump %s --port={pgport%s:%s} --format=raw > tsdump.gob", + test.DefaultCockroachPath, sec, c.Node(node), install.SystemInterfaceName, + ), ); err != nil { return err } @@ -1408,8 +1412,14 @@ func (c *clusterImpl) FetchDebugZip( // assumption that a down node will refuse the connection, so it won't // waste our time. for _, node := range nodes { - // `cockroach debug zip` does not support non root authentication. - nodePgUrl, err := c.InternalPGUrl(ctx, l, c.Node(node), roachprod.PGURLOptions{Auth: install.AuthRootCert}) + pgURLOpts := roachprod.PGURLOptions{ + // `cockroach debug zip` does not support non root authentication. + Auth: install.AuthRootCert, + // request the system tenant specifically in case the test + // changed the default virtual cluster. + VirtualClusterName: install.SystemInterfaceName, + } + nodePgUrl, err := c.InternalPGUrl(ctx, l, c.Node(node), pgURLOpts) if err != nil { l.Printf("cluster.FetchDebugZip failed to retrieve PGUrl on node %d: %v", node, err) continue diff --git a/pkg/cmd/roachtest/option/options.go b/pkg/cmd/roachtest/option/options.go index 7e4acbacebdc..2f0a770e988d 100644 --- a/pkg/cmd/roachtest/option/options.go +++ b/pkg/cmd/roachtest/option/options.go @@ -189,6 +189,17 @@ func SkipInit(opts interface{}) { } } +// Tag sets a process tag when stopping processes. Useful if we want +// to kill a cockroach process that was started with `settings.TagOption`. +func Tag(tag string) func(opts interface{}) { + return func(opts interface{}) { + switch opts := opts.(type) { + case *StopOpts: + opts.RoachprodOpts.ProcessTag = tag + } + } +} + // WithInitTarget allows the caller to configure which node is used as // `InitTarget` when starting cockroach. Specially useful when // starting clusters in a subset of VMs in the cluster that doesn't diff --git a/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go b/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go index 05bcd89a6ffc..9fbff178bd25 100644 --- a/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go +++ b/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go @@ -368,19 +368,26 @@ func RestartNodesWithNewBinary( rand.Shuffle(len(nodes), func(i, j int) { nodes[i], nodes[j] = nodes[j], nodes[i] }) + + // Stop the cockroach process gracefully in order to drain it properly. + // This makes the upgrade closer to how users do it in production, but + // it's also needed to eliminate flakiness. In particular, this will + // make sure that DistSQL draining information is communicated through + // gossip so that other nodes running an older version don't consider + // this upgraded node for DistSQL plans (see #87154 for more details). + stopOptions := []option.StartStopOption{option.Graceful(gracePeriod)} + + // If we are starting the cockroach process with a tag, we apply the + // same tag when stopping. + for _, s := range settings { + if t, ok := s.(install.TagOption); ok { + stopOptions = append(stopOptions, option.Tag(string(t))) + } + } + for _, node := range nodes { l.Printf("restarting node %d into version %s", node, newVersion.String()) - // Stop the cockroach process gracefully in order to drain it properly. - // This makes the upgrade closer to how users do it in production, but - // it's also needed to eliminate flakiness. In particular, this will - // make sure that DistSQL draining information is dissipated through - // gossip so that other nodes running an older version don't consider - // this upgraded node for DistSQL plans (see #87154 for more details). - // TODO(yuzefovich): ideally, we would also check that the drain was - // successful since if it wasn't, then we might see flakes too. - if err := c.StopE( - ctx, l, option.NewStopOpts(option.Graceful(gracePeriod)), c.Node(node), - ); err != nil { + if err := c.StopE(ctx, l, option.NewStopOpts(stopOptions...), c.Node(node)); err != nil { return err } diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel b/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel index 94800181e1e3..5a9bf046fecc 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/cmd/roachtest/registry", "//pkg/cmd/roachtest/roachtestutil", "//pkg/cmd/roachtest/roachtestutil/clusterupgrade", + "//pkg/cmd/roachtest/spec", "//pkg/cmd/roachtest/test", "//pkg/roachpb", "//pkg/roachprod/install", @@ -56,6 +57,7 @@ go_test( "//pkg/cmd/roachtest/registry", "//pkg/cmd/roachtest/roachtestutil", "//pkg/cmd/roachtest/roachtestutil/clusterupgrade", + "//pkg/cmd/roachtest/spec", "//pkg/roachpb", "//pkg/roachprod/install", "//pkg/roachprod/logger", diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/context.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/context.go index 041834dc6c5a..a7da2f12fd93 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/context.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/context.go @@ -125,6 +125,10 @@ func (sc *ServiceContext) changeVersion(node int, v *clusterupgrade.Version) err return nil } +func (sc *ServiceContext) IsSystem() bool { + return sc.Descriptor.Name == install.SystemInterfaceName +} + // NodeVersion returns the release version the given `node` is // currently running. Returns an error if the node is not valid (i.e., // the underlying service is not deployed on the node passed). @@ -267,25 +271,6 @@ func (c *Context) DefaultService() *ServiceContext { return c.Tenant } -// SetFinalizing sets the `Finalizing` field on all services -// available. -func (c *Context) SetFinalizing(b bool) { - c.forEachService(func(s *ServiceContext) { s.Finalizing = b }) -} - -// SetStage is a helper function to set the upgrade stage on all -// services available. -func (c *Context) SetStage(stage UpgradeStage) { - c.forEachService(func(s *ServiceContext) { s.Stage = stage }) -} - -func (c *Context) forEachService(f func(*ServiceContext)) { - f(c.System) - if c.Tenant != nil { - f(c.Tenant) - } -} - // clone copies the caller Context and returns the copy. func (c *Context) clone() Context { return Context{ diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/helper.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/helper.go index 3217285282b7..512b2d0a2d61 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/helper.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/helper.go @@ -169,6 +169,10 @@ func (h *Helper) IsMultitenant() bool { return h.Tenant != nil } +func (h *Helper) DeploymentMode() DeploymentMode { + return h.runner.plan.deploymentMode +} + func (h *Helper) DefaultService() *Service { if h.Tenant != nil { return h.Tenant diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go index 683f24c0f14d..af6275c8a35c 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go @@ -86,6 +86,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" @@ -96,12 +97,14 @@ import ( ) const ( - logPrefix = "mixed-version-test" - startupLabel = "run startup hooks" - backgroundLabel = "start background hooks" - mixedVersionLabel = "run mixed-version hooks" - afterTestLabel = "run after test hooks" - genericLabel = "run following steps" // used by mutators to group steps + logPrefix = "mixed-version-test" + startupLabel = "run startup hooks" + backgroundLabel = "start background hooks" + mixedVersionLabel = "run mixed-version hooks" + afterTestLabel = "run after test hooks" + upgradeStorageClusterLabel = "upgrade storage cluster" + upgradeTenantLabel = "upgrade tenant" + genericLabel = "run following steps" // used by mutators to group steps unreachable = "internal error: unreachable" @@ -132,8 +135,9 @@ const ( // These `*Deployment` constants are used to indicate different // deployment modes that a test may choose to enable/disable. - SystemOnlyDeployment = DeploymentMode("system-only") - SharedProcessDeployment = DeploymentMode("shared-process") + SystemOnlyDeployment = DeploymentMode("system-only") + SharedProcessDeployment = DeploymentMode("shared-process") + SeparateProcessDeployment = DeploymentMode("separate-process") ) var ( @@ -162,6 +166,7 @@ var ( allDeploymentModes = []DeploymentMode{ SystemOnlyDeployment, SharedProcessDeployment, + SeparateProcessDeployment, } // OldestSupportedVersion is the oldest cockroachdb version @@ -180,21 +185,21 @@ var ( // [1] https://www.cockroachlabs.com/docs/releases/release-support-policy#current-supported-releases OldestSupportedVersion = clusterupgrade.MustParseVersion("v23.1.0") - // OldestSupportedVersionVC is similar to `OldestSupportedVersion`, - // but applies only to virtual cluster deployments. The reason it is - // different for now is that v23.1.5 is the first version to ship - // with the ability to change the default target cluster, a core - // component of tests generated by the framework. + // OldestSupportedVersionSP is similar to `OldestSupportedVersion`, + // but applies only to shared-process virtual cluster deployments. + // The reason it is different for now is that v23.1.5 is the first + // version to ship with the ability to change the default target + // cluster, a core component of tests generated by the framework. // // TODO(testeng): remove this constant when 23.1 reaches EOL. - OldestSupportedVersionVC = clusterupgrade.MustParseVersion("v23.1.5") + OldestSupportedVersionSP = clusterupgrade.MustParseVersion("v23.1.5") // TenantsAndSystemAlignedSettingsVersion is the version in which - // some work was done to align virtual clusters and the storage - // cluster settings. This means that shared-process virtual clusters - // get all capabilities by default, and some settings have their - // values switched to enabled by default (such as the ability to - // split and scatter tables). + // some work was done to align shared-process virtual clusters and + // the storage cluster settings. This means that shared-process + // virtual clusters get all capabilities by default, and some + // settings have their values switched to enabled by default (such + // as the ability to split and scatter tables). TenantsAndSystemAlignedSettingsVersion = clusterupgrade.MustParseVersion("v24.1.0") // tenantSettingsVersionOverrideFixVersion is the lowest version @@ -543,6 +548,7 @@ func NewTest( for _, fn := range options { fn(&opts) } + opts.enabledDeploymentModes = validDeploymentModesForCloud(c.Cloud(), opts.enabledDeploymentModes) testLogger, err := prefixedLogger(l, filepath.Join(opts.tag, logPrefix)) if err != nil { @@ -611,16 +617,27 @@ func (t *Test) InMixedVersion(desc string, fn stepFunc) { return t.prng.Float64() < runWhileMigratingProbability } + upgradingService := testContext.DefaultService() + if upgradingService.Stage == UpgradingSystemStage { + // This condition represents the situation where we are + // upgrading the storage cluster in a separate-process + // deployment. In that case, we want to look at the storage + // cluster when checking the number of nodes in the + // previous/next versions. + upgradingService = testContext.System + } + // This check makes sure we only schedule a mixed-version hook - // once while upgrading from one version to another. The number of - // nodes we wait to be running the new version is determined when - // the version changes for the first time. - if testContext.DefaultService().Stage != prevUpgradeStage { - prevUpgradeStage = testContext.DefaultService().Stage + // once while upgrading (or downgrading) from one version to + // another. The number of nodes we wait to be running the new + // version is determined when the version changes for the first + // time. + if upgradingService.Stage != prevUpgradeStage { + prevUpgradeStage = upgradingService.Stage numUpgradedNodes = t.prng.Intn(len(t.crdbNodes)) + 1 } - return len(testContext.NodesInNextVersion()) == numUpgradedNodes + return len(upgradingService.NodesInNextVersion()) == numUpgradedNodes } t.hooks.AddMixedVersion(versionUpgradeHook{name: desc, predicate: predicate, fn: fn}) @@ -976,13 +993,11 @@ func randomPredecessor( } func (t *Test) updateOptionsForDeploymentMode(mode DeploymentMode) { - if mode == SystemOnlyDeployment { + if mode != SharedProcessDeployment { return } - // Make sure to enforce the minimum supported version for virtual - // clusters. - if v := OldestSupportedVersionVC; !t.options.minimumSupportedVersion.AtLeast(v) { + if v := OldestSupportedVersionSP; !t.options.minimumSupportedVersion.AtLeast(v) { t.options.minimumSupportedVersion = v } } @@ -992,7 +1007,7 @@ func (t *Test) tenantDescriptor(deploymentMode DeploymentMode) *ServiceDescripto case SystemOnlyDeployment: return nil - case SharedProcessDeployment: + case SharedProcessDeployment, SeparateProcessDeployment: return &ServiceDescriptor{ Name: virtualClusterName(t.prng), Nodes: t.crdbNodes, @@ -1090,15 +1105,12 @@ func (h hooks) Filter(testContext Context) hooks { return result } -// AsSteps transforms the sequence of hooks into a corresponding test -// step. If there is only one hook, the corresponding `runHookStep` is -// returned. Otherwise, a `concurrentRunStep` is returned, where every -// hook is run concurrently. `stopChans` should either be `nil` for -// steps that are not meant to be run in the background, or contain -// one stop channel (`shouldStop`) for each hook. -func (h hooks) AsSteps( - label string, prng *rand.Rand, testContext *Context, stopChans []shouldStop, isLocal bool, -) []testStep { +// AsSteps transforms the sequence of hooks into a sequence of +// `*singleStep` structs to be run in some way by the planner +// (sequentially, concurrently, etc). `stopChans` should either be +// `nil` for steps that are not meant to be run in the background, or +// contain one stop channel (`shouldStop`) for each hook. +func (h hooks) AsSteps(prng *rand.Rand, testContext *Context, stopChans []shouldStop) []testStep { steps := make([]testStep, 0, len(h)) stopChanFor := func(j int) shouldStop { if len(stopChans) == 0 { @@ -1114,11 +1126,7 @@ func (h hooks) AsSteps( }, rngFromRNG(prng))) } - if len(steps) <= 1 { - return steps - } - - return []testStep{newConcurrentRunStep(label, steps, prng, isLocal)} + return steps } func (th *testHooks) AddStartup(hook versionUpgradeHook) { @@ -1137,29 +1145,29 @@ func (th *testHooks) AddAfterUpgradeFinalized(hook versionUpgradeHook) { th.afterUpgradeFinalized = append(th.afterUpgradeFinalized, hook) } -func (th *testHooks) StartupSteps(testContext *Context, rng *rand.Rand, isLocal bool) []testStep { - return th.startup.AsSteps(startupLabel, rng, testContext, nil, isLocal) +func (th *testHooks) StartupSteps(testContext *Context, rng *rand.Rand) []testStep { + return th.startup.AsSteps(rng, testContext, nil) } func (th *testHooks) BackgroundSteps( - testContext *Context, stopChans []shouldStop, rng *rand.Rand, isLocal bool, + testContext *Context, stopChans []shouldStop, rng *rand.Rand, ) []testStep { - testContext.SetStage(BackgroundStage) - return th.background.AsSteps(backgroundLabel, rng, testContext, stopChans, isLocal) + testContext.System.Stage = BackgroundStage + if testContext.Tenant != nil { + testContext.Tenant.Stage = BackgroundStage + } + + return th.background.AsSteps(rng, testContext, stopChans) } -func (th *testHooks) MixedVersionSteps( - testContext *Context, rng *rand.Rand, isLocal bool, -) []testStep { +func (th *testHooks) MixedVersionSteps(testContext *Context, rng *rand.Rand) []testStep { return th.mixedVersion. Filter(*testContext). - AsSteps(mixedVersionLabel, rng, testContext, nil, isLocal) + AsSteps(rng, testContext, nil) } -func (th *testHooks) AfterUpgradeFinalizedSteps( - testContext *Context, rng *rand.Rand, isLocal bool, -) []testStep { - return th.afterUpgradeFinalized.AsSteps(afterTestLabel, rng, testContext, nil, isLocal) +func (th *testHooks) AfterUpgradeFinalizedSteps(testContext *Context, rng *rand.Rand) []testStep { + return th.afterUpgradeFinalized.AsSteps(rng, testContext, nil) } // pickRandomDelay chooses a random duration from the list passed, @@ -1193,6 +1201,26 @@ func virtualClusterName(rng *rand.Rand) string { ) } +// validDeploymentModesForCloud computes a subset of the given +// parameter `modes` containing the deployment modes that can be run +// on the given cloud. Specifically, the only rule enforced at the +// moment is that separate-process deployments are only possible +// locally or on GCE, since they require service registration. +func validDeploymentModesForCloud(cloud spec.Cloud, modes []DeploymentMode) []DeploymentMode { + if cloud == spec.GCE || cloud == spec.Local { + return modes + } + + var validModes []DeploymentMode + for _, m := range modes { + if m != SeparateProcessDeployment { + validModes = append(validModes, m) + } + } + + return validModes +} + func assertValidTest(test *Test, fatalFunc func(...interface{})) { fail := func(err error) { 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 index 7d5957b9abd8..21e460ab9a38 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/testutils/release" "github.com/cockroachdb/cockroach/pkg/util/version" @@ -58,6 +59,47 @@ var testReleaseData = func() map[string]release.Series { return result }() +func Test_validDeploymentModesForCloud(t *testing.T) { + testCases := []struct { + name string + cloud spec.Cloud + modes []DeploymentMode + expectedModes []DeploymentMode + }{ + { + name: "locally, all modes are allowed", + cloud: spec.Local, + modes: allDeploymentModes, + expectedModes: allDeploymentModes, + }, + { + name: "on gce, all modes are allowed", + cloud: spec.GCE, + modes: allDeploymentModes, + expectedModes: allDeploymentModes, + }, + { + name: "on aws, we can't run separate process deployments", + cloud: spec.AWS, + modes: allDeploymentModes, + expectedModes: []DeploymentMode{SystemOnlyDeployment, SharedProcessDeployment}, + }, + { + name: "on azure, we can't run separate process deployments", + cloud: spec.Azure, + modes: allDeploymentModes, + expectedModes: []DeploymentMode{SystemOnlyDeployment, SharedProcessDeployment}, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + actual := validDeploymentModesForCloud(tc.cloud, tc.modes) + require.Equal(t, tc.expectedModes, actual) + }) + } +} + func Test_assertValidTest(t *testing.T) { var fatalErr error fatalFunc := func() func(...interface{}) { @@ -140,6 +182,16 @@ func Test_assertValidTest(t *testing.T) { fatalErr.Error(), ) + // simulate a NewTest call with an actual `cluster` implementation + mvt = newTest(EnabledDeploymentModes(SeparateProcessDeployment)) + mvt.options.enabledDeploymentModes = validDeploymentModesForCloud(spec.AWS, mvt.options.enabledDeploymentModes) + assertValidTest(mvt, fatalFunc()) + require.Error(t, fatalErr) + require.Equal(t, + `mixedversion.NewTest: invalid test options: no deployment modes enabled`, + fatalErr.Error(), + ) + mvt = newTest(MinimumSupportedVersion("v22.2.0")) 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 ccf40851e98b..523193d7513c 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go @@ -177,11 +177,11 @@ const ( const ( // Upgrade stages are defined in the order they happen during test - // runs, so that we are able to select, for example, "stages after - // rollback" by doing `stage > RollbackUpgrade`. Note that - // `BackgroundStage` is special in the sense that it may continue to - // run across stage changes, so doing direct stage comparisons as - // mentioned above doesn't make sense for background functions. + // runs, so that we are able to implement mutators that select, for + // example, "stages after rollback" by doing `stage > RollbackUpgrade`. + // Note that `BackgroundStage` is special in the sense that it may + // continue to run across stage changes, so doing direct stage comparisons + // as mentioned above doesn't make sense for background functions. SystemSetupStage UpgradeStage = iota TenantSetupStage OnStartupStage @@ -193,10 +193,24 @@ const ( RunningUpgradeMigrationsStage AfterUpgradeFinalizedStage + // These upgrade stages are service-specific and therefore not + // comparable to the upgrade stages defined above (which apply for + // system and tenant). To make this explicit, we set them to an + // arbitrarily large constant. + UpgradingSystemStage = 1000 // only applicable to tenant + UpgradingTenantStage = 1001 // only applicable to system + mutationInsertBefore mutationOp = iota mutationInsertAfter mutationInsertConcurrent mutationRemove + + // spanConfigTenantLimit is the value assigned to the + // `spanconfig.tenant_limit` cluster setting, controling the number + // of spans that a separate-process tenant can create. The value of + // 50k chosen here matches Serverless deployments and helps tests + // that perform lots of schema changes. + spanConfigTenantLimit = 50000 ) // planMutators includes a list of all known `mutator` @@ -254,31 +268,89 @@ var planMutators = []mutator{ func (p *testPlanner) Plan() *TestPlan { setup, testUpgrades := p.setupTest() - planUpgrade := func(upgrade *upgradePlan, virtualClusterSetup, scheduleHooks bool) { - for _, s := range p.services() { - s.startUpgrade(upgrade.to) + upgradeStepsForService := func( + service *ServiceContext, from, to *clusterupgrade.Version, virtualClusterRunning, scheduleHooks bool, + ) []testStep { + if p.deploymentMode == SeparateProcessDeployment { + // Separate-process deployments are different because the two + // services involved (system and tenant) restart at different + // times (since they run separate binaries): the system (storage + // cluster) needs to upgrade to completion first, and only then + // the tenant is allowed to start upgrading. + // + // To allow for internal and external hooks to recognize this + // stage, we set the service's stage accordingly below. + if service.IsSystem() { + // Indicates that tenant is waiting for system to finish + // upgrading. + p.currentContext.Tenant.Stage = UpgradingSystemStage + } else { + // Indicates that system is waiting for tenant to finish + // upgrading. + p.currentContext.System.Stage = UpgradingTenantStage + } + } + + var steps []testStep + addSteps := func(ss []testStep) { + steps = append(steps, ss...) } - upgrade.Add(p.initUpgradeSteps(virtualClusterSetup)) - if p.shouldRollback(upgrade.to) { + + addSteps(p.initUpgradeSteps(service, virtualClusterRunning)) + if p.shouldRollback(to) { // previous -> next - upgrade.Add(p.upgradeSteps( - TemporaryUpgradeStage, upgrade.from, upgrade.to, scheduleHooks, virtualClusterSetup, + addSteps(p.upgradeSteps( + service, TemporaryUpgradeStage, from, to, scheduleHooks, virtualClusterRunning, )) // next -> previous (rollback) - upgrade.Add(p.downgradeSteps( - upgrade.to, upgrade.from, scheduleHooks, virtualClusterSetup, + addSteps(p.downgradeSteps( + service, to, from, scheduleHooks, virtualClusterRunning, )) } // previous -> next - upgrade.Add(p.upgradeSteps( - LastUpgradeStage, upgrade.from, upgrade.to, scheduleHooks, virtualClusterSetup, + addSteps(p.upgradeSteps( + service, LastUpgradeStage, from, to, scheduleHooks, virtualClusterRunning, )) // finalize -- i.e., run upgrade migrations. - upgrade.Add(p.finalizeUpgradeSteps(upgrade.from, upgrade.to, scheduleHooks, virtualClusterSetup)) + addSteps(p.finalizeUpgradeSteps(service, to, scheduleHooks, virtualClusterRunning)) // run after upgrade steps, if any, - upgrade.Add(p.afterUpgradeSteps(upgrade.from, upgrade.to, scheduleHooks)) + addSteps(p.afterUpgradeSteps(service, from, to, scheduleHooks)) + + return steps + } + + planUpgrade := func(upgrade *upgradePlan, virtualClusterRunning, scheduleHooks bool) { + for _, s := range p.services() { + s.startUpgrade(upgrade.to) + } + + // systemUpgradeSteps does a rolling restart of the system tenant + // (storage cluster) in this deployment. In system-only or + // shared-process deployments, this is the only upgrade to be + // performed. + systemUpgradeSteps := upgradeStepsForService( + p.currentContext.System, upgrade.from, upgrade.to, virtualClusterRunning, scheduleHooks, + ) + + if p.deploymentMode != SeparateProcessDeployment || !virtualClusterRunning { + upgrade.Add(systemUpgradeSteps) + return + } + + // At this point, we know we are in a separate-process deployment, + // and that the virtual cluster used during the test has been + // created and is running. Therefore, we need to do a rolling + // restart of the tenant binaries as well. + tenantUpgradeSteps := upgradeStepsForService( + p.currentContext.Tenant, upgrade.from, upgrade.to, virtualClusterRunning, scheduleHooks, + ) + + upgrade.Add([]testStep{ + sequentialRunStep{label: upgradeStorageClusterLabel, steps: systemUpgradeSteps}, + sequentialRunStep{label: upgradeTenantLabel, steps: tenantUpgradeSteps}, + }) } for _, upgrade := range setup.systemSetup.upgrades { @@ -411,11 +483,16 @@ func (p *testPlanner) setupTest() (testSetup, []*upgradePlan) { tenantBootstrapVersion := allUpgrades[0].from if len(setupUpgrades) > 0 && p.isMultitenant() { - // If we are in a multi-tenant deployment and we have some setup + oldestSupported := OldestSupportedVersion + if p.deploymentMode == SharedProcessDeployment { + oldestSupported = OldestSupportedVersionSP + } + + // If we are in a multitenant deployment and we have some setup // upgrades to run, we find the first setup upgrade before which a // tenant can be started. firstSupportedUpgradeIdx := slices.IndexFunc(setupUpgrades, func(upgrade *upgradePlan) bool { - return upgrade.from.AtLeast(OldestSupportedVersionVC) + return upgrade.from.AtLeast(oldestSupported) }) if firstSupportedUpgradeIdx == -1 { @@ -446,6 +523,19 @@ func (p *testPlanner) setupTest() (testSetup, []*upgradePlan) { var tenantSetup *serviceSetup if p.isMultitenant() { + for _, n := range p.currentContext.Tenant.Descriptor.Nodes { + // Update the context for the tenant with the actual version we + // are using to bootstrap it. `changeVersion` is only called + // when upgrading a service, and the tenant might be created + // until *after* the storage cluster has gone through a few + // upgrades. This is primarily important in separate-process + // deployments, where we display the released version the tenant + // is running after it is created. + handleInternalError( + p.currentContext.Tenant.changeVersion(n, tenantBootstrapVersion), + ) + } + tenantSetup = &serviceSetup{ steps: p.tenantSetupSteps(tenantBootstrapVersion), upgrades: upgradesAfterTenantSetup, @@ -475,7 +565,7 @@ func (p *testPlanner) systemSetupSteps() []testStep { rt: p.rt, initTarget: p.currentContext.System.Descriptor.Nodes[0], waitForReplication: p.shouldWaitForReplication(), - settings: p.clusterSettingsForSystem(), + settings: p.clusterSettingsForSystem(initialVersion), }), p.newSingleStepWithContext(setupContext, waitForStableClusterVersionStep{ nodes: p.currentContext.System.Descriptor.Nodes, @@ -491,30 +581,57 @@ func (p *testPlanner) systemSetupSteps() []testStep { // passed is the version in which the tenant is created. func (p *testPlanner) tenantSetupSteps(v *clusterupgrade.Version) []testStep { setupContext := p.nonUpgradeContext(v, TenantSetupStage) - shouldGrantCapabilities := p.deploymentMode == SharedProcessDeployment && - !v.AtLeast(TenantsAndSystemAlignedSettingsVersion) + shouldGrantCapabilities := p.deploymentMode == SeparateProcessDeployment || + (p.deploymentMode == SharedProcessDeployment && !v.AtLeast(TenantsAndSystemAlignedSettingsVersion)) - // In case we are starting a shared-process virtual cluster, create - // it, wait for the cluster version to match the expected version, - // set it as the default cluster, and give it all capabilities if - // necessary. - steps := []testStep{ - p.newSingleStepWithContext(setupContext, startSharedProcessVirtualClusterStep{ + var startStep singleStepProtocol + if p.deploymentMode == SharedProcessDeployment { + startStep = startSharedProcessVirtualClusterStep{ name: p.tenantName(), initTarget: p.currentContext.Tenant.Descriptor.Nodes[0], - settings: p.clusterSettingsForTenant(), - }), + settings: p.clusterSettingsForTenant(v), + } + } else { + startStep = startSeparateProcessVirtualClusterStep{ + name: p.tenantName(), + rt: p.rt, + version: v, + settings: p.clusterSettingsForTenant(v), + } + } + + // We are creating a virtual cluster: we first create it, then wait + // for the cluster version to match the expected version, then set + // it as the default cluster, and finally give it all capabilities + // if necessary. + steps := []testStep{ + p.newSingleStepWithContext(setupContext, startStep), p.newSingleStepWithContext(setupContext, waitForStableClusterVersionStep{ nodes: p.currentContext.Tenant.Descriptor.Nodes, timeout: p.options.upgradeTimeout, desiredVersion: versionToClusterVersion(v), virtualClusterName: p.tenantName(), }), - p.newSingleStepWithContext(setupContext, setClusterSettingStep{ + } + + // We only use the 'default tenant' cluster setting in + // shared-process deployments. For separate-process deployments, we + // rely on roachtest's `c.SetDefaultVirtualCluster`. + if p.deploymentMode == SharedProcessDeployment { + steps = append(steps, p.newSingleStepWithContext(setupContext, setClusterSettingStep{ name: defaultTenantClusterSetting(v), value: p.tenantName(), virtualClusterName: install.SystemInterfaceName, - }), + })) + } + + if p.deploymentMode == SeparateProcessDeployment { + steps = append(steps, p.newSingleStepWithContext(setupContext, setClusterSettingStep{ + name: "spanconfig.tenant_limit", + value: spanConfigTenantLimit, + virtualClusterName: p.tenantName(), + systemVisible: true, + })) } if shouldGrantCapabilities { @@ -541,9 +658,9 @@ func (p *testPlanner) tenantSetupSteps(v *clusterupgrade.Version) []testStep { // startupSteps returns the list of steps that should be executed once // the test (as defined by user-provided functions) is ready to start. func (p *testPlanner) startupSteps(firstUpgradeVersion *clusterupgrade.Version) []testStep { - return p.hooks.StartupSteps( - p.nonUpgradeContext(firstUpgradeVersion, OnStartupStage), p.prng, p.isLocal, - ) + return p.concurrently(startupLabel, p.hooks.StartupSteps( + p.nonUpgradeContext(firstUpgradeVersion, OnStartupStage), p.prng, + )) } // testStartSteps are the user-provided steps that should run when the @@ -552,17 +669,19 @@ func (p *testPlanner) startupSteps(firstUpgradeVersion *clusterupgrade.Version) func (p *testPlanner) testStartSteps(firstUpgradeVersion *clusterupgrade.Version) []testStep { return append( p.startupSteps(firstUpgradeVersion), - p.hooks.BackgroundSteps( - p.nonUpgradeContext(firstUpgradeVersion, OnStartupStage), p.bgChans, p.prng, p.isLocal, - )..., + p.concurrently(backgroundLabel, p.hooks.BackgroundSteps( + p.nonUpgradeContext(firstUpgradeVersion, OnStartupStage), p.bgChans, p.prng, + ))..., ) } // initUpgradeSteps returns the sequence of steps that should be // executed before we start changing binaries on nodes in the process // of upgrading/downgrading. -func (p *testPlanner) initUpgradeSteps(virtualClusterSetup bool) []testStep { - p.currentContext.SetStage(InitUpgradeStage) +func (p *testPlanner) initUpgradeSteps( + service *ServiceContext, virtualClusterRunning bool, +) []testStep { + p.setStage(service, InitUpgradeStage) preserveDowngradeForService := func(name string) *singleStep { return p.newSingleStep(preserveDowngradeOptionStep{ @@ -570,16 +689,31 @@ func (p *testPlanner) initUpgradeSteps(virtualClusterSetup bool) []testStep { }) } - steps := []testStep{preserveDowngradeForService(install.SystemInterfaceName)} - + preserveDowngradeSystem := preserveDowngradeForService(install.SystemInterfaceName) switch p.deploymentMode { + case SystemOnlyDeployment: + return []testStep{preserveDowngradeSystem} + case SharedProcessDeployment: - if virtualClusterSetup { - steps = append(steps, preserveDowngradeForService(p.tenantName())) + if virtualClusterRunning { + return []testStep{ + preserveDowngradeSystem, + preserveDowngradeForService(p.tenantName()), + } } - } - return steps + return []testStep{preserveDowngradeSystem} + + case SeparateProcessDeployment: + if service.IsSystem() { + return []testStep{preserveDowngradeSystem} + } + + return nil // nothing to do, separate-process servers don't auto-upgrade + + default: + panic(unreachable) + } } // afterUpgradeSteps are the steps to be run once the nodes have been @@ -587,12 +721,12 @@ func (p *testPlanner) initUpgradeSteps(virtualClusterSetup bool) []testStep { // the same and then run any after-finalization hooks the user may // have provided. func (p *testPlanner) afterUpgradeSteps( - fromVersion, toVersion *clusterupgrade.Version, scheduleHooks bool, + service *ServiceContext, fromVersion, toVersion *clusterupgrade.Version, scheduleHooks bool, ) []testStep { - p.currentContext.SetFinalizing(false) - p.currentContext.SetStage(AfterUpgradeFinalizedStage) + p.setFinalizing(service, false) + p.setStage(service, AfterUpgradeFinalizedStage) if scheduleHooks { - return p.hooks.AfterUpgradeFinalizedSteps(p.currentContext, p.prng, p.isLocal) + return p.concurrently(afterTestLabel, p.hooks.AfterUpgradeFinalizedSteps(p.currentContext, p.prng)) } // Currently, we only schedule user-provided hooks after the upgrade @@ -601,21 +735,26 @@ func (p *testPlanner) afterUpgradeSteps( } func (p *testPlanner) upgradeSteps( - stage UpgradeStage, from, to *clusterupgrade.Version, scheduleHooks, virtualClusterSetup bool, + service *ServiceContext, + stage UpgradeStage, + from, to *clusterupgrade.Version, + scheduleHooks, virtualClusterRunning bool, ) []testStep { - p.currentContext.SetStage(stage) - nodes := p.currentContext.System.Descriptor.Nodes + p.setStage(service, stage) + nodes := service.Descriptor.Nodes msg := fmt.Sprintf("upgrade nodes %v from %q to %q", nodes, from.String(), to.String()) - return p.changeVersionSteps(from, to, msg, scheduleHooks, virtualClusterSetup) + return p.changeVersionSteps(service, from, to, msg, scheduleHooks, virtualClusterRunning) } func (p *testPlanner) downgradeSteps( - from, to *clusterupgrade.Version, scheduleHooks, virtualClusterSetup bool, + service *ServiceContext, + from, to *clusterupgrade.Version, + scheduleHooks, virtualClusterRunning bool, ) []testStep { - p.currentContext.SetStage(RollbackUpgradeStage) + p.setStage(service, RollbackUpgradeStage) nodes := p.currentContext.System.Descriptor.Nodes msg := fmt.Sprintf("downgrade nodes %v from %q to %q", nodes, from.String(), to.String()) - return p.changeVersionSteps(from, to, msg, scheduleHooks, virtualClusterSetup) + return p.changeVersionSteps(service, from, to, msg, scheduleHooks, virtualClusterRunning) } // changeVersionSteps returns the sequence of steps to be performed @@ -625,9 +764,12 @@ func (p *testPlanner) downgradeSteps( // mixed-version hooks will be scheduled randomly during the // upgrade/downgrade process. func (p *testPlanner) changeVersionSteps( - from, to *clusterupgrade.Version, label string, scheduleHooks, virtualClusterSetup bool, + service *ServiceContext, + from, to *clusterupgrade.Version, + label string, + scheduleHooks, virtualClusterRunning bool, ) []testStep { - nodes := p.currentContext.System.Descriptor.Nodes + nodes := service.Descriptor.Nodes // copy system `Nodes` here so that shuffling won't mutate that array previousVersionNodes := append(option.NodeListOption{}, nodes...) @@ -647,23 +789,43 @@ func (p *testPlanner) changeVersionSteps( var steps []testStep for j, node := range previousVersionNodes { - steps = append(steps, p.newSingleStep( - restartWithNewBinaryStep{ - version: to, - node: node, - rt: p.rt, - settings: p.clusterSettingsForSystem(), - sharedProcessStarted: virtualClusterSetup, - initTarget: p.currentContext.System.Descriptor.Nodes[0], - waitForReplication: p.shouldWaitForReplication(), - }, - )) - for _, s := range p.services() { + var restartStep singleStepProtocol + if service.IsSystem() { + restartStep = restartWithNewBinaryStep{ + version: to, + node: node, + rt: p.rt, + settings: p.clusterSettingsForSystem(to), + tenantRunning: virtualClusterRunning, + deploymentMode: p.deploymentMode, + initTarget: p.currentContext.System.Descriptor.Nodes[0], + waitForReplication: p.shouldWaitForReplication(), + } + } else { + restartStep = restartVirtualClusterStep{ + version: to, + node: node, + virtualCluster: p.tenantName(), + rt: p.rt, + settings: p.clusterSettingsForTenant(to), + } + } + + steps = append(steps, p.newSingleStep(restartStep)) + + affectedServices := p.services() + if p.deploymentMode == SeparateProcessDeployment { + affectedServices = []*ServiceContext{service} + } + for _, s := range affectedServices { handleInternalError(s.changeVersion(node, to)) } if scheduleHooks { - steps = append(steps, p.hooks.MixedVersionSteps(p.currentContext, p.prng, p.isLocal)...) + steps = append( + steps, + p.concurrently(mixedVersionLabel, p.hooks.MixedVersionSteps(p.currentContext, p.prng))..., + ) } else if j == waitIndex { // If we are not scheduling user-provided hooks, we wait a short // while in this state to allow some time for background @@ -683,90 +845,124 @@ func (p *testPlanner) changeVersionSteps( // hooks while the cluster version is changing. At the end of this // process, we wait for all the migrations to finish running. func (p *testPlanner) finalizeUpgradeSteps( - fromVersion, toVersion *clusterupgrade.Version, scheduleHooks, virtualClusterSetup bool, + service *ServiceContext, + toVersion *clusterupgrade.Version, + scheduleHooks, virtualClusterRunning bool, ) []testStep { - p.currentContext.SetFinalizing(true) - p.currentContext.SetStage(RunningUpgradeMigrationsStage) - runSystemMigrations := p.newSingleStep(allowUpgradeStep{ - virtualClusterName: install.SystemInterfaceName, - }) + p.setFinalizing(service, true) + p.setStage(service, RunningUpgradeMigrationsStage) - steps := []testStep{runSystemMigrations} + allowAutoUpgrade := p.newSingleStep(allowUpgradeStep{ + virtualClusterName: service.Descriptor.Name, + }) - var mixedVersionStepsDuringTenantMigrations []testStep - if scheduleHooks { - mixedVersionSteps := p.hooks.MixedVersionSteps(p.currentContext, p.prng, p.isLocal) + var steps []testStep - switch p.deploymentMode { - case SystemOnlyDeployment: - steps = append(steps, mixedVersionSteps...) + // runStepsWithTenantMigrations adds a step to `SET` the tenant's + // cluster version to the version we are upgrading to. In addition, + // if `scheduleHooks` is true, we choose a (possibly empty) subset + // of user-hooks to run concurrently with these migrations. + runStepsWithTenantMigrations := func() { + upgradeSteps := []testStep{ + p.newSingleStep(setClusterVersionStep{ + v: toVersion, + virtualClusterName: p.tenantName(), + }), + } - case SharedProcessDeployment: - // If we are in a multi-tenant deployment, we run some - // mixed-version hooks while the system is finalizing, and the - // remaining hooks while the tenant is finalizing. - if len(mixedVersionSteps) > 0 { - idx := p.prng.Intn(len(mixedVersionSteps)) + if scheduleHooks { + upgradeSteps = append(upgradeSteps, p.hooks.MixedVersionSteps(p.currentContext, p.prng)...) + } - mixedVersionStepsDuringSystemMigrations := mixedVersionSteps[:idx] - steps = append(steps, mixedVersionStepsDuringSystemMigrations...) + steps = append(steps, p.concurrently(genericLabel, upgradeSteps)...) + } - mixedVersionStepsDuringTenantMigrations = mixedVersionSteps[idx:] - } + // If the service being upgraded is the system service, we allow + // auto-upgrades since they are supported. + if service.IsSystem() { + steps = []testStep{allowAutoUpgrade} + if scheduleHooks { + steps = append( + steps, + p.concurrently(mixedVersionLabel, p.hooks.MixedVersionSteps(p.currentContext, p.prng))..., + ) } + } else { + // In separate-process deployments, we set the cluster setting + // explicitly, as auto-upgrades are not supported. We also run + // all steps selected to run during finalization at this stage. + runStepsWithTenantMigrations() } steps = append(steps, p.newSingleStep( waitForStableClusterVersionStep{ - nodes: p.currentContext.System.Descriptor.Nodes, + nodes: service.Descriptor.Nodes, timeout: p.options.upgradeTimeout, desiredVersion: versionToClusterVersion(toVersion), - virtualClusterName: install.SystemInterfaceName, + virtualClusterName: service.Descriptor.Name, }, )) - // At this point, we just upgraded the storage cluster; we might - // need to patch up the `system.tenant_settings` table in order for - // the tenant upgrade to work. + // In system-only and shared-process deployments, this function is + // only called once (for the system service, since the tenant is + // co-located in the same process). At this point, we have finished + // upgrading and finalizing the system. This is all we need to do in + // system-only and separate-process deployments in this + // call. However, in shared-process deployments, we also need to + // update the cluster version for the in-memory tenant (see below). + if p.deploymentMode == SystemOnlyDeployment || p.deploymentMode == SeparateProcessDeployment { + return steps + } + + // At this point, we need to upgrade the shared-process tenant. We + // might need to patch up the `system.tenant_settings` table first + // in order for the upgrade to work. steps = append(steps, p.maybeDeleteAllTenantsVersionOverride(toVersion)...) - switch p.deploymentMode { - case SharedProcessDeployment: - if virtualClusterSetup { + if virtualClusterRunning { + steps = append( + steps, + p.newSingleStep(allowUpgradeStep{ + virtualClusterName: p.tenantName(), + }), + ) + + // If we are upgrading to a version that does not support + // auto-upgrading after resetting a cluster setting, we need to + // "manually" run the migrations at this point. + if toVersion.AtLeast(tenantSupportsAutoUpgradeVersion) { steps = append( steps, - p.newSingleStep(allowUpgradeStep{ - virtualClusterName: p.tenantName(), - }), + p.concurrently(mixedVersionLabel, p.hooks.MixedVersionSteps(p.currentContext, p.prng))..., ) - - // If we are upgrading to a version that does not support - // auto-upgrading after resetting a cluster setting, we need to - // "manually" run the migrations at this point. - if !toVersion.AtLeast(tenantSupportsAutoUpgradeVersion) { - steps = append(steps, - p.newSingleStep( - setTenantClusterVersionStep{ - virtualClusterName: p.tenantName(), - nodes: p.currentContext.Tenant.Descriptor.Nodes, - targetVersion: versionToClusterVersion(toVersion), - }, - )) - } - - steps = append(steps, mixedVersionStepsDuringTenantMigrations...) - steps = append(steps, p.newSingleStep(waitForStableClusterVersionStep{ - nodes: p.currentContext.Tenant.Descriptor.Nodes, - timeout: p.options.upgradeTimeout, - desiredVersion: versionToClusterVersion(toVersion), - virtualClusterName: p.currentContext.Tenant.Descriptor.Name, - })) + } else { + runStepsWithTenantMigrations() } + + // Finally, we confirm that every node is aware of the new + // `version` cluster setting. + steps = append(steps, p.newSingleStep(waitForStableClusterVersionStep{ + nodes: p.currentContext.Tenant.Descriptor.Nodes, + timeout: p.options.upgradeTimeout, + desiredVersion: versionToClusterVersion(toVersion), + virtualClusterName: p.currentContext.Tenant.Descriptor.Name, + })) } return steps } +// concurrently returns the same `steps` input if there is less than 1 +// step in the slice. Otherwise, it groups all steps into a +// `concurrentRunStep` for parallel execution. +func (p *testPlanner) concurrently(label string, steps []testStep) []testStep { + if len(steps) <= 1 { + return steps + } + + return []testStep{newConcurrentRunStep(label, steps, p.prng, p.isLocal)} +} + // shouldRollback returns whether the test will attempt a rollback. If // we are upgrading to the current version being tested, we always // rollback, as we want to expose that upgrade to complex upgrade @@ -783,17 +979,11 @@ func (p *testPlanner) shouldRollback(toVersion *clusterupgrade.Version) bool { // maybeDeleteAllTenantsVersionOverride will delete the bad 'version' // key from the system.tenant_settings table when necessary. -// Specifically, doing this is necessary when we are in virtual -// cluster deployment mode (shared or external process) and we just -// upgraded to a version in the 23.1 release series older than -// v23.1.9. +// Specifically, doing this is necessary when we just upgraded to a +// version in the 23.1 release series older than v23.1.9. func (p *testPlanner) maybeDeleteAllTenantsVersionOverride( toVersion *clusterupgrade.Version, ) []testStep { - if p.deploymentMode == SystemOnlyDeployment { - return nil - } - if isAffected := toVersion.Series() == "23.1" && toVersion.Patch() <= 8; !isAffected { return nil } @@ -829,6 +1019,25 @@ func (p *testPlanner) serviceDescriptors() []*ServiceDescriptor { return descriptors } +func (p *testPlanner) setStage(service *ServiceContext, stage UpgradeStage) { + p.changeServiceProcess(service, func(s *ServiceContext) { s.Stage = stage }) +} + +func (p *testPlanner) setFinalizing(service *ServiceContext, b bool) { + p.changeServiceProcess(service, func(s *ServiceContext) { s.Finalizing = b }) +} + +func (p *testPlanner) changeServiceProcess(service *ServiceContext, fn func(*ServiceContext)) { + switch p.deploymentMode { + case SystemOnlyDeployment, SeparateProcessDeployment: + fn(service) + + case SharedProcessDeployment: + fn(p.currentContext.System) + fn(p.currentContext.Tenant) + } +} + func (p *testPlanner) isMultitenant() bool { return p.deploymentMode != SystemOnlyDeployment } @@ -841,10 +1050,24 @@ func (p *testPlanner) shouldWaitForReplication() bool { return p.options.waitForReplication && len(p.currentContext.System.Descriptor.Nodes) >= 3 } -func (p *testPlanner) clusterSettingsForSystem() []install.ClusterSettingOption { +func (p *testPlanner) clusterSettingsForSystem( + v *clusterupgrade.Version, +) []install.ClusterSettingOption { cs := []install.ClusterSettingOption{} cs = append(cs, defaultClusterSettings...) cs = append(cs, p.options.settings...) + + // 23.1 releases commonly suffer from a `use of Span after finish` + // error in separate-process deployments. We ignore these to reduce + // noise. + // + // TODO(testeng): remove this logic after 23.1 reaches EOL. + if v.Series() == "23.1" && p.deploymentMode == SeparateProcessDeployment { + cs = append(cs, install.EnvOption([]string{ + "COCKROACH_CRASH_ON_SPAN_USE_AFTER_FINISH=false", + })) + } + return cs } @@ -857,8 +1080,10 @@ func (p *testPlanner) clusterSettingsForSystem() []install.ClusterSettingOption // multiple versions into account. If really needed, we could provide // some option for the caller to indicate which options apply to // system vs tenant, but that's not necessary at the moment. -func (p *testPlanner) clusterSettingsForTenant() []install.ClusterSettingOption { - settings := p.clusterSettingsForSystem() +func (p *testPlanner) clusterSettingsForTenant( + v *clusterupgrade.Version, +) []install.ClusterSettingOption { + settings := p.clusterSettingsForSystem(v) var tenantSettings []install.ClusterSettingOption for _, s := range settings { @@ -1006,7 +1231,7 @@ func newStepIndex(plan *TestPlan) stepIndex { // the given `mutationOp` relative to the step passed as argument // (which is expected to exist in the underlying step sequence). // -// The logic in this function relies on the assumption (withheld by +// The logic in this function relies on the assumption (upheld by // `mutator` implementations) that steps inserted by mutations do not // change the `Context` they run in (in other words, they don't // restart nodes with different binaries). In that case, when @@ -1228,12 +1453,13 @@ func (plan *TestPlan) applyMutations(rng *rand.Rand, mutations []mutation) { } // mutationApplicationOrder rearranges the collection of mutations -// passed so that all insertions happen before any removal. This is to -// avoid the situation where an insertion references a step that is -// removed by a previous mutation. This is a safe operation because -// mutators generate mutations based on a fixed view of the test plan; -// in other words, a mutation is never relative to a step created by -// a previous mutation. +// *generated by a single mutator* so that all insertions happen +// before any removal. This is to avoid the situation where an +// insertion references a step that is removed by a previous +// mutation. This is a safe operation because mutators generate +// mutations based on a fixed view of the test plan; in other words, a +// mutation is never relative to a step created by a previous +// mutation. func mutationApplicationOrder(mutations []mutation) []mutation { var insertions []mutation var removals []mutation @@ -1264,7 +1490,9 @@ func (plan *TestPlan) assignIDs() { plan.mapSingleSteps(func(ss *singleStep, _ bool) []testStep { stepID := nextID() _, isStartSystem := ss.impl.(startStep) - _, isStartTenant := ss.impl.(startSharedProcessVirtualClusterStep) + _, isStartSharedProcess := ss.impl.(startSharedProcessVirtualClusterStep) + _, isStartSeparateProcess := ss.impl.(startSeparateProcessVirtualClusterStep) + isStartTenant := isStartSharedProcess || isStartSeparateProcess if plan.startSystemID == 0 && isStartSystem { plan.startSystemID = stepID @@ -1419,7 +1647,12 @@ func (plan *TestPlan) prettyPrintStep( if ss.context.Finalizing() { finalizingStr = ",finalizing" } - debugInfo = fmt.Sprintf(" [stage=%s%s]", ss.context.System.Stage, finalizingStr) + + stageStr := ss.context.System.Stage.String() + if plan.deploymentMode != SystemOnlyDeployment { + stageStr = fmt.Sprintf("system:%s;tenant:%s", ss.context.System.Stage, ss.context.Tenant.Stage) + } + debugInfo = fmt.Sprintf(" [stage=%s%s]", stageStr, finalizingStr) } out.WriteString(fmt.Sprintf( @@ -1452,7 +1685,7 @@ func treeBranchString(idx, sliceLen int) string { // that changes the default tenant on a cluster. The setting changed // its name from 23.1 to 23.2. func defaultTenantClusterSetting(v *clusterupgrade.Version) string { - if !v.AtLeast(OldestSupportedVersionVC) { + if !v.AtLeast(OldestSupportedVersionSP) { handleInternalError(fmt.Errorf("defaultTenantClusterSetting called on version %s", v)) } @@ -1485,6 +1718,10 @@ func (u UpgradeStage) String() string { return "running-upgrade-migrations" case AfterUpgradeFinalizedStage: return "after-upgrade-finished" + case UpgradingSystemStage: + return "upgrading-system" + case UpgradingTenantStage: + return "upgrading-tenant" default: return fmt.Sprintf("invalid upgrade stage (%d)", u) } diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go index 95b51678125f..109fce37e19f 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go @@ -81,15 +81,7 @@ func TestTestPlanner(t *testing.T) { datadriven.Walk(t, datapathutils.TestDataPath(t, "planner"), func(t *testing.T, path string) { defer withTestBuildVersion("v24.3.0")() resetMutators() - // Unless specified, treat every test as a system-only deployment - // test. Tests can use the deployment-mode option in the - // mixed-version-test directive to change the deployment mode. - defaultOpts := []CustomOption{ - EnabledDeploymentModes(SystemOnlyDeployment), - DisableSkipVersionUpgrades, - } - - mvt := newTest(defaultOpts...) + mvt := newTest() datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { if d.Cmd == "plan" { @@ -123,7 +115,7 @@ func TestTestPlanner(t *testing.T) { planMutators = append(planMutators, m) } case "mixed-version-test": - mvt = createDataDrivenMixedVersionTest(t, d.CmdArgs, defaultOpts) + mvt = createDataDrivenMixedVersionTest(t, d.CmdArgs) case "on-startup": mvt.OnStartup(d.CmdArgs[0].Vals[0], dummyHook) case "in-mixed-version": @@ -339,6 +331,18 @@ func newRand() *rand.Rand { func newTest(options ...CustomOption) *Test { testOptions := defaultTestOptions() + // Enforce some default options by default in tests; those that test + // multitenant deployments or skip-version upgrades specifically + // should pass the corresponding option explicitly. + defaultTestOverrides := []CustomOption{ + EnabledDeploymentModes(SystemOnlyDeployment), + DisableSkipVersionUpgrades, + } + + for _, fn := range defaultTestOverrides { + fn(&testOptions) + } + for _, fn := range options { fn(&testOptions) } @@ -392,10 +396,8 @@ func testPredecessorFunc( // createDataDrivenMixedVersionTest creates a `*Test` instance based // on the parameters passed to the `mixed-version-test` datadriven // directive. -func createDataDrivenMixedVersionTest( - t *testing.T, args []datadriven.CmdArg, defaultOpts []CustomOption, -) *Test { - opts := append([]CustomOption{}, defaultOpts...) +func createDataDrivenMixedVersionTest(t *testing.T, args []datadriven.CmdArg) *Test { + var opts []CustomOption var predecessors []*clusterupgrade.Version var isLocal *bool diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go index 6fdbd844a619..87b615cf1bf1 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go @@ -57,28 +57,6 @@ type ( stopFuncs []StopFunc } - serviceFailureDetails struct { - descriptor *ServiceDescriptor - binaryVersions []roachpb.Version - // Cluster versions before and after the failure occurred. Before - // each step is executed, the test runner will cache each node's - // view of the cluster version; after a failure occurs, we'll try - // to read the cluster version from every node again. This context - // is added to the failure message displayed to the user with the - // intention of highlighting whether the cluster version changed - // during the failure, which is useful for test failures that - // happen while the upgrade is finalizing. - clusterVersionsBefore []roachpb.Version - clusterVersionsAfter []roachpb.Version - } - - testFailureDetails struct { - seed int64 - testContext *Context - systemService *serviceFailureDetails - tenantService *serviceFailureDetails - } - // crdbMonitor is a thin wrapper around the roachtest monitor API // (cluster.NewMonitor) that produces error events through a channel // whenever an unexpected node death happens. It also allows us to @@ -386,55 +364,22 @@ func (tr *testRunner) stepError( func (tr *testRunner) testFailure( ctx context.Context, err error, l *logger.Logger, testContext *Context, ) error { - detailsForService := func(service *serviceRuntime) *serviceFailureDetails { - return &serviceFailureDetails{ - descriptor: service.descriptor, - binaryVersions: loadAtomicVersions(service.binaryVersions), - clusterVersionsBefore: loadAtomicVersions(service.clusterVersions), - clusterVersionsAfter: loadAtomicVersions(service.clusterVersions), - } - } - - var systemDetails *serviceFailureDetails - var tenantDetails *serviceFailureDetails - for _, service := range tr.allServices() { - if service.descriptor.Name == install.SystemInterfaceName { - systemDetails = detailsForService(service) - } else { - tenantDetails = detailsForService(service) - } - } - - currentClusterVersions := func(service *serviceRuntime) []roachpb.Version { - if tr.connCacheInitialized(service) { - if err := tr.refreshClusterVersions(ctx, service); err == nil { - return loadAtomicVersions(service.clusterVersions) - } else { - tr.logger.Printf( - "failed to fetch cluster versions for service %s after failure: %s", - service.descriptor.Name, err, - ) - } - } - - return loadAtomicVersions(service.clusterVersions) - } - - systemDetails.clusterVersionsAfter = currentClusterVersions(tr.systemService) - if tenantDetails != nil { - tenantDetails.clusterVersionsAfter = currentClusterVersions(tr.tenantService) + lines := []string{ + "test failed:", + fmt.Sprintf("test random seed: %d (use COCKROACH_RANDOM_SEED to reproduce)\n", tr.plan.seed), } - tf := &testFailureDetails{ - seed: tr.plan.seed, - testContext: testContext, - systemService: systemDetails, - tenantService: tenantDetails, + if testContext != nil { + lines = append(lines, versionsTable( + tr.plan.deploymentMode, + tr.systemService, tr.tenantService, + testContext.System, testContext.Tenant, + )) } // failureErr wraps the original error, adding mixed-version state // information as error details. - failureErr := errors.WithDetailf(err, "%s", tf.Format()) + failureErr := errors.WithDetailf(err, "%s", strings.Join(lines, "\n")) // Print the test failure on the step's logger for convenience, and // to reduce cross referencing of logs. @@ -487,41 +432,109 @@ 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.impl.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, testContext Context) { - binaryVersions := loadAtomicVersions(tr.systemService.binaryVersions) - systemClusterVersions := loadAtomicVersions(tr.systemService.clusterVersions) + tbl := versionsTable( + tr.plan.deploymentMode, + tr.systemService, tr.tenantService, + testContext.System, testContext.Tenant, + ) + + if tbl != "" { + l.Printf("current cluster configuration:\n%s", tbl) + } +} + +// versionsTable returns a string with a table representation of the +// current cached versions of the binary and cluster versions on each +// node, for both system and tenant services. +func versionsTable( + deploymentMode DeploymentMode, + systemRuntime, tenantRuntime *serviceRuntime, + systemContext, tenantContext *ServiceContext, +) string { + systemBinaryVersions := loadAtomicVersions(systemRuntime.binaryVersions) + systemClusterVersions := loadAtomicVersions(systemRuntime.clusterVersions) + + if systemBinaryVersions == nil || systemClusterVersions == nil { + return "" + } + var tenantClusterVersions []roachpb.Version - if tr.tenantService != nil { - tenantClusterVersions = loadAtomicVersions(tr.tenantService.clusterVersions) + if tenantRuntime != nil { + tenantClusterVersions = loadAtomicVersions(tenantRuntime.clusterVersions) + } + + serviceReleasedVersions := func(service *ServiceContext) []*clusterupgrade.Version { + releasedVersions := make([]*clusterupgrade.Version, 0, len(service.Descriptor.Nodes)) + for _, node := range service.Descriptor.Nodes { + nv, err := service.NodeVersion(node) + handleInternalError(err) + releasedVersions = append(releasedVersions, nv) + } + + return releasedVersions + } + + systemReleasedVersions := serviceReleasedVersions(systemContext) + var tenantReleasedVersions []*clusterupgrade.Version + var tenantBinaryVersions []roachpb.Version + if deploymentMode == SeparateProcessDeployment { + tenantBinaryVersions = loadAtomicVersions(tenantRuntime.binaryVersions) + tenantReleasedVersions = serviceReleasedVersions(tenantContext) } - releasedVersions := make([]*clusterupgrade.Version, 0, len(testContext.System.Descriptor.Nodes)) - for _, node := range testContext.System.Descriptor.Nodes { - nv, err := testContext.NodeVersion(node) - handleInternalError(err) - releasedVersions = append(releasedVersions, nv) + withLabel := func(name, label string) string { + return fmt.Sprintf("%s (%s)", name, label) } - if binaryVersions == nil || systemClusterVersions == nil { - return + withSystemLabel := func(name string, perTenant bool) string { + if !perTenant { + return name + } + + return withLabel(name, install.SystemInterfaceName) } - tw := newTableWriter(testContext.System.Descriptor.Nodes) - tw.AddRow("released versions", toString(releasedVersions)...) - tw.AddRow("logical binary versions", toString(binaryVersions)...) + tw := newTableWriter(systemContext.Descriptor.Nodes) + + // Released (e.g., v24.1.4) and logical (e.g., '24.1') binary + // versions are only per-tenant if we are in an separate-process + // deployment: otherwise, they are shared properties of system and + // tenants. + tw.AddRow( + withSystemLabel("released versions", deploymentMode == SeparateProcessDeployment), + toString(systemReleasedVersions)..., + ) + tw.AddRow( + withSystemLabel("binary versions", deploymentMode == SeparateProcessDeployment), + toString(systemBinaryVersions)..., + ) + // Cluster versions are per-tenant in any multitenant deployment. + tw.AddRow( + withSystemLabel("cluster versions", deploymentMode != SystemOnlyDeployment), + toString(systemClusterVersions)..., + ) + + if tenantRuntime != nil { + if deploymentMode == SeparateProcessDeployment { + tw.AddRow( + withLabel("released versions", tenantRuntime.descriptor.Name), + toString(tenantReleasedVersions)..., + ) + + tw.AddRow( + withLabel("binary versions", tenantRuntime.descriptor.Name), + toString(tenantBinaryVersions)..., + ) + } - tw.AddRow("cluster versions (system)", toString(systemClusterVersions)...) - if len(tenantClusterVersions) > 0 { tw.AddRow( - fmt.Sprintf("cluster versions (%s)", tr.tenantService.descriptor.Name), + withLabel("cluster versions", tenantRuntime.descriptor.Name), toString(tenantClusterVersions)..., ) } - l.Printf("current cluster configuration:\n%s", tw.String()) + return tw.String() } // loggerFor creates a logger instance to be used by a test step. Logs @@ -541,7 +554,7 @@ func (tr *testRunner) loggerFor(step *singleStep) (*logger.Logger, error) { // cluster. We use the `atomic` package here as this function may be // called by two steps that are running concurrently. func (tr *testRunner) refreshBinaryVersions(ctx context.Context, service *serviceRuntime) error { - newBinaryVersions := make([]roachpb.Version, len(tr.systemService.descriptor.Nodes)) + newBinaryVersions := make([]roachpb.Version, len(service.descriptor.Nodes)) connectionCtx, cancel := context.WithTimeout(ctx, internalQueryTimeout) defer cancel() @@ -550,7 +563,10 @@ func (tr *testRunner) refreshBinaryVersions(ctx context.Context, service *servic group.GoCtx(func(ctx context.Context) error { bv, err := clusterupgrade.BinaryVersion(ctx, tr.conn(node, service.descriptor.Name)) if err != nil { - return fmt.Errorf("failed to get binary version for node %d: %w", node, err) + return fmt.Errorf( + "failed to get binary version for node %d (%s): %w", + node, service.descriptor.Name, err, + ) } newBinaryVersions[j] = bv @@ -579,7 +595,10 @@ func (tr *testRunner) refreshClusterVersions(ctx context.Context, service *servi group.GoCtx(func(ctx context.Context) error { cv, err := clusterupgrade.ClusterVersion(ctx, tr.conn(node, service.descriptor.Name)) if err != nil { - return fmt.Errorf("failed to get cluster version for node %d: %w", node, err) + return fmt.Errorf( + "failed to get cluster version for node %d (%s): %w", + node, service.descriptor.Name, err, + ) } newClusterVersions[j] = cv @@ -596,6 +615,8 @@ func (tr *testRunner) refreshClusterVersions(ctx context.Context, service *servi } func (tr *testRunner) refreshServiceData(ctx context.Context, service *serviceRuntime) error { + isSystem := service == tr.systemService + // Update the runner's view of the cluster's binary and cluster // versions for given service before every non-initialization // `singleStep` is executed. @@ -603,7 +624,7 @@ func (tr *testRunner) refreshServiceData(ctx context.Context, service *serviceRu return err } - if service == tr.systemService { + if isSystem || tr.plan.deploymentMode == SeparateProcessDeployment { if err := tr.refreshBinaryVersions(ctx, service); err != nil { return err } @@ -613,7 +634,22 @@ func (tr *testRunner) refreshServiceData(ctx context.Context, service *serviceRu return err } - tr.monitor.Init() + // We only want to start the monitor once we know every relevant + // cockroach binary is running. This is due to a limitation on the + // roachprod monitor: it is only able to monitor cockroach processes + // that are running at the time the monitor is created. + // + // For system-only and separate-process deployments, we can + // initialize the monitor right away, since this function is only + // called once the storage cluster is running. For separate-process + // deployments, we start the monitor if this function is called with + // the tenant service. The system is always started first, so when + // this function is called with the tenant service, we know that + // every relevant cockroach binary is running at this point. + if tr.plan.deploymentMode != SeparateProcessDeployment || !isSystem { + tr.monitor.Init() + } + return nil } @@ -645,13 +681,6 @@ func (tr *testRunner) maybeInitConnections(service *serviceRuntime) error { return nil } -func (tr *testRunner) connCacheInitialized(service *serviceRuntime) bool { - service.connCache.mu.Lock() - defer service.connCache.mu.Unlock() - - return service.connCache.cache != nil -} - func (tr *testRunner) newHelper( ctx context.Context, l *logger.Logger, testContext Context, ) *Helper { @@ -850,46 +879,6 @@ func (br *backgroundRunner) CompletedEvents() <-chan backgroundEvent { return br.events } -func (tfd *testFailureDetails) Format() string { - lines := []string{ - "test failed:", - fmt.Sprintf("test random seed: %d\n", tfd.seed), - } - - tw := newTableWriter(tfd.systemService.descriptor.Nodes) - if tfd.testContext != nil { - releasedVersions := make([]*clusterupgrade.Version, 0, len(tfd.testContext.System.Descriptor.Nodes)) - for _, node := range tfd.testContext.System.Descriptor.Nodes { - nv, err := tfd.testContext.NodeVersion(node) - handleInternalError(err) - releasedVersions = append(releasedVersions, nv) - } - tw.AddRow("released versions", toString(releasedVersions)...) - } - - tw.AddRow("logical binary versions", toString(tfd.systemService.binaryVersions)...) - for _, service := range []*serviceFailureDetails{tfd.systemService, tfd.tenantService} { - if service == nil { - continue - } - - tw.AddRow( - fmt.Sprintf("cluster versions before failure (%s)", service.descriptor.Name), - toString(service.clusterVersionsBefore)..., - ) - - if cv := service.clusterVersionsAfter; cv != nil { - tw.AddRow( - fmt.Sprintf("cluster versions after failure (%s)", service.descriptor.Name), - toString(cv)..., - ) - } - } - - lines = append(lines, tw.String()) - return strings.Join(lines, "\n") -} - // tableWriter is a thin wrapper around the `tabwriter` package used // by the test runner to display logical and released binary versions // in a tabular format. diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/steps.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/steps.go index c74f636b9df6..a881f085d1e2 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/steps.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/steps.go @@ -27,6 +27,14 @@ import ( "github.com/cockroachdb/errors" ) +// systemTag is the roachprod tag applied to the cockroach process +// associated with the storage cluster. This is necessary in +// separate-process deployments where, when performing a rolling +// restart, we want to be able to stop *only* the process for the +// system tenant on a node. By default, roachtest's `Stop` function +// will stop every cockroach process. +const systemTag = "mixedversion-system" + // startTimeout is the maximum amount of time we will wait for a node // to start up (including restarts). Especially useful in cases where // we wait for a 3x replication after a restart, to fail early in @@ -84,6 +92,7 @@ func (s startStep) Run(ctx context.Context, l *logger.Logger, _ *rand.Rand, h *H clusterSettings := append( append([]install.ClusterSettingOption{}, s.settings...), install.BinaryOption(binaryPath), + install.TagOption(systemTag), ) customStartOpts := []option.StartStopOption{option.WithInitTarget(s.initTarget)} @@ -132,6 +141,85 @@ func (s startSharedProcessVirtualClusterStep) Run( return waitForSharedProcess(ctx, l, h, h.Tenant.Descriptor.Nodes) } +// startSeparateProcessVirtualCluster step creates a new separate-process +// virtual cluster with the given name, and starts it. +type startSeparateProcessVirtualClusterStep struct { + name string + rt test.Test + version *clusterupgrade.Version + settings []install.ClusterSettingOption +} + +func (s startSeparateProcessVirtualClusterStep) Background() shouldStop { return nil } + +func (s startSeparateProcessVirtualClusterStep) Description() string { + return fmt.Sprintf( + "start separate process virtual cluster %s with binary version %s", + s.name, s.version, + ) +} + +func (s startSeparateProcessVirtualClusterStep) Run( + ctx context.Context, l *logger.Logger, _ *rand.Rand, h *Helper, +) error { + l.Printf("starting separate process virtual cluster %s at version %s", s.name, s.version) + startOpts := option.StartVirtualClusterOpts( + s.name, + h.Tenant.Descriptor.Nodes, + option.StorageCluster(h.System.Descriptor.Nodes), + ) + + binaryPath := clusterupgrade.BinaryPathForVersion(s.rt, s.version, "cockroach") + settings := install.MakeClusterSettings(append(s.settings, install.BinaryOption(binaryPath))...) + + if err := h.runner.cluster.StartServiceForVirtualClusterE(ctx, l, startOpts, settings); err != nil { + return err + } + + h.runner.cluster.SetDefaultVirtualCluster(s.name) + return nil +} + +type restartVirtualClusterStep struct { + virtualCluster string + version *clusterupgrade.Version + rt test.Test + node int + settings []install.ClusterSettingOption +} + +func (s restartVirtualClusterStep) Background() shouldStop { return nil } + +func (s restartVirtualClusterStep) Description() string { + return fmt.Sprintf( + "restart %s server on node %d with binary version %s", + s.virtualCluster, s.node, s.version, + ) +} + +func (s restartVirtualClusterStep) Run( + ctx context.Context, l *logger.Logger, rng *rand.Rand, h *Helper, +) error { + const maxWait = 300 // 5 minutes + + l.Printf("restarting node %d (tenant %s) into version %s", s.node, s.virtualCluster, s.version) + node := h.runner.cluster.Node(s.node) + + h.ExpectDeath() + stopOpts := option.StopVirtualClusterOpts(s.virtualCluster, node, option.Graceful(maxWait)) + if err := h.runner.cluster.StopServiceForVirtualClusterE(ctx, l, stopOpts); err != nil { + return errors.Wrap(err, "failed to stop cockroach process for tenant") + } + + // Assume the binary already exists on the node as this step should + // only be scheduled after the storage cluster has already upgraded. + binaryPath := clusterupgrade.BinaryPathForVersion(s.rt, s.version, "cockroach") + + startOpts := option.StartVirtualClusterOpts(s.virtualCluster, node, option.NoBackupSchedule) + settings := install.MakeClusterSettings(append(s.settings, install.BinaryOption(binaryPath))...) + return h.runner.cluster.StartServiceForVirtualClusterE(ctx, l, startOpts, settings) +} + // waitForStableClusterVersionStep implements the process of waiting // for the `version` cluster setting being the same on all nodes of // the cluster and equal to the binary version of the first node in @@ -147,8 +235,8 @@ func (s waitForStableClusterVersionStep) Background() shouldStop { return nil } func (s waitForStableClusterVersionStep) Description() string { return fmt.Sprintf( - "wait for %s tenant on nodes %v to reach cluster version %s", - s.virtualClusterName, s.nodes, quoteVersionForPresentation(s.desiredVersion), + "wait for all nodes (%v) to acknowledge cluster version %s on %s tenant", + s.nodes, quoteVersionForPresentation(s.desiredVersion), s.virtualClusterName, ) } @@ -193,21 +281,32 @@ func (s preserveDowngradeOptionStep) Run( // restartWithNewBinaryStep restarts a certain `node` with a new // cockroach binary. Any existing `cockroach` process will be stopped, // then the new binary will be uploaded and the `cockroach` process -// will restart using the new binary. +// will restart using the new binary. This step only applies to the +// system tenant. For the (separate-process) multitenant equivalent, +// see `restartVirtualClusterStep`. type restartWithNewBinaryStep struct { - version *clusterupgrade.Version - rt test.Test - node int - settings []install.ClusterSettingOption - initTarget int - waitForReplication bool - sharedProcessStarted bool + version *clusterupgrade.Version + rt test.Test + node int + settings []install.ClusterSettingOption + initTarget int + waitForReplication bool + tenantRunning bool // whether the test tenant is running when this step is called + deploymentMode DeploymentMode } func (s restartWithNewBinaryStep) Background() shouldStop { return nil } func (s restartWithNewBinaryStep) Description() string { - return fmt.Sprintf("restart node %d with binary version %s", s.node, s.version.String()) + var systemDesc string + if s.deploymentMode == SeparateProcessDeployment { + systemDesc = " system server on" + } + + return fmt.Sprintf( + "restart%s node %d with binary version %s", + systemDesc, s.node, s.version, + ) } func (s restartWithNewBinaryStep) Run( @@ -221,6 +320,10 @@ func (s restartWithNewBinaryStep) Run( startCtx, cancel := context.WithTimeout(ctx, startTimeout) defer cancel() + settings := append([]install.ClusterSettingOption{ + install.TagOption(systemTag), + }, s.settings...) + h.ExpectDeath() if err := clusterupgrade.RestartNodesWithNewBinary( startCtx, @@ -230,12 +333,12 @@ func (s restartWithNewBinaryStep) Run( h.runner.cluster.Node(s.node), startOpts(customStartOpts...), s.version, - s.settings..., + settings..., ); err != nil { return err } - if s.sharedProcessStarted { + if s.deploymentMode == SharedProcessDeployment && s.tenantRunning { // If we are in shared-process mode and the tenant is already // running at this point, we wait for the server on the restarted // node to be up before moving on. @@ -269,55 +372,6 @@ func (s allowUpgradeStep) Run( ) } -// setTenantClusterVersionStep will `set` the `version` setting on the -// tenant with the associated name. Used in older versions where -// auto-upgrading does not work reliably; in those cases, we block -// and wait for the migrations to run before proceeding. -type setTenantClusterVersionStep struct { - nodes option.NodeListOption - targetVersion string - virtualClusterName string -} - -func (s setTenantClusterVersionStep) Background() shouldStop { return nil } - -func (s setTenantClusterVersionStep) Description() string { - return fmt.Sprintf( - "run upgrades on tenant %s by explicitly setting cluster version to %s", - s.virtualClusterName, quoteVersionForPresentation(s.targetVersion), - ) -} - -func (s setTenantClusterVersionStep) Run( - ctx context.Context, l *logger.Logger, rng *rand.Rand, h *Helper, -) error { - node, db := h.RandomDB(rng) - - targetVersion := s.targetVersion - if s.targetVersion == clusterupgrade.CurrentVersionString { - l.Printf("querying binary version on node %d", node) - bv, err := clusterupgrade.BinaryVersion(ctx, db) - if err != nil { - return err - } - - targetVersion = fmt.Sprintf("%d.%d", bv.Major, bv.Minor) - if !bv.IsFinal() { - targetVersion += fmt.Sprintf("-%d", bv.Internal) - } - } - - l.Printf( - "setting version on tenant %s to %s via node %d", - s.virtualClusterName, targetVersion, node, - ) - - stmt := fmt.Sprintf("SET CLUSTER SETTING version = '%s'", targetVersion) - return serviceByName(h, s.virtualClusterName).ExecWithGateway( - rng, h.runner.cluster.Node(node), stmt, - ) -} - // waitStep does nothing but sleep for the provided duration. Most // commonly used to allow the cluster to stay in a certain state // before attempting node restarts or other upgrade events. @@ -364,6 +418,7 @@ type setClusterSettingStep struct { name string value interface{} virtualClusterName string + systemVisible bool } func (s setClusterSettingStep) Background() shouldStop { return nil } @@ -378,6 +433,16 @@ func (s setClusterSettingStep) Description() string { func (s setClusterSettingStep) Run( ctx context.Context, l *logger.Logger, rng *rand.Rand, h *Helper, ) error { + // We set the cluster setting on the corresponding `virtualClusterName`. + // However, if `systemVisible` is true, it means the setting + // is only settable via the system interface. + var tenantPrefix string + serviceName := s.virtualClusterName + if s.systemVisible { + tenantPrefix = fmt.Sprintf("ALTER TENANT %q ", s.virtualClusterName) + serviceName = install.SystemInterfaceName + } + var stmt string var args []interface{} // We do a type switch on common types to avoid errors when using @@ -385,23 +450,65 @@ func (s setClusterSettingStep) Run( // private cluster settings. switch val := s.value.(type) { case string: - stmt = fmt.Sprintf("SET CLUSTER SETTING %s = '%s'", s.name, val) + stmt = fmt.Sprintf("%sSET CLUSTER SETTING %s = '%s'", tenantPrefix, s.name, val) case bool: - stmt = fmt.Sprintf("SET CLUSTER SETTING %s = %t", s.name, val) + stmt = fmt.Sprintf("%sSET CLUSTER SETTING %s = %t", tenantPrefix, s.name, val) case int: - stmt = fmt.Sprintf("SET CLUSTER SETTING %s = %d", s.name, val) + stmt = fmt.Sprintf("%sSET CLUSTER SETTING %s = %d", tenantPrefix, s.name, val) default: // If not using any of these types, do a best-effort attempt using // a placeholder. - stmt = fmt.Sprintf("SET CLUSTER SETTING %s = $1", s.name) + stmt = fmt.Sprintf("%sSET CLUSTER SETTING %s = $1", tenantPrefix, s.name) args = []interface{}{val} } - return serviceByName(h, s.virtualClusterName).ExecWithGateway( + return serviceByName(h, serviceName).ExecWithGateway( rng, nodesRunningAtLeast(s.virtualClusterName, s.minVersion, h), stmt, args..., ) } +// setClusterVersionStep sets the special `version` cluster setting to +// the provided version. +type setClusterVersionStep struct { + v *clusterupgrade.Version + virtualClusterName string +} + +func (s setClusterVersionStep) Background() shouldStop { return nil } + +func (s setClusterVersionStep) Description() string { + value := versionToClusterVersion(s.v) + if !s.v.IsCurrent() { + value = fmt.Sprintf("'%s'", value) + } + + return fmt.Sprintf( + "set `version` to %s on %s tenant", + value, s.virtualClusterName, + ) +} + +func (s setClusterVersionStep) Run( + ctx context.Context, l *logger.Logger, rng *rand.Rand, h *Helper, +) error { + service := serviceByName(h, s.virtualClusterName) + binaryVersion := versionToClusterVersion(s.v) + if s.v.IsCurrent() { + node, db := service.RandomDB(rng) + l.Printf("fetching binary version via n%d", node) + + bv, err := clusterupgrade.BinaryVersion(ctx, db) + if err != nil { + return errors.Wrapf(err, "getting binary version on n%d", node) + } + + binaryVersion = bv.String() + } + + l.Printf("setting cluster version to '%s'", binaryVersion) + return service.Exec(rng, "SET CLUSTER SETTING version = $1", binaryVersion) +} + // resetClusterSetting resets cluster setting `name`. type resetClusterSettingStep struct { minVersion *clusterupgrade.Version diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/basic_test_mixed_version_hooks b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/basic_test_mixed_version_hooks index f1b57f508bce..09df51b10149 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/basic_test_mixed_version_hooks +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/basic_test_mixed_version_hooks @@ -29,7 +29,7 @@ Deployment mode: system-only Plan: ├── install fixtures for version "v24.1.1" (1) ├── start cluster at version "v24.1.1" (2) -├── wait for system tenant on nodes :1-4 to reach cluster version '24.1' (3) +├── wait for all nodes (:1-4) to acknowledge cluster version '24.1' on system tenant (3) ├── run startup hooks concurrently │ ├── run "initialize bank workload", after 3m0s delay (4) │ └── run "initialize rand workload", after 0s delay (5) @@ -51,7 +51,7 @@ Plan: │ ├── run mixed-version hooks concurrently │ │ ├── run "mixed-version 1", after 100ms delay (17) │ │ └── run "mixed-version 2", after 500ms delay (18) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '24.2' (19) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '24.2' on system tenant (19) └── upgrade cluster from "v24.2.2" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (20) ├── upgrade nodes :1-4 from "v24.2.2" to "" @@ -75,4 +75,4 @@ Plan: │ ├── run "mixed-version 2" (36) │ └── restart node 4 with binary version (37) ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (38) - └── wait for system tenant on nodes :1-4 to reach cluster version (39) + └── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (39) diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/cluster_setting b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/cluster_setting index 1369a9d5f1bd..1d006c4309c3 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/cluster_setting +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/cluster_setting @@ -25,7 +25,7 @@ Mutators: cluster_setting[test_cluster_setting] Plan: ├── install fixtures for version "v22.2.3" (1) ├── start cluster at version "v22.2.3" (2) -├── wait for system tenant on nodes :1-4 to reach cluster version '22.2' (3) +├── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on system tenant (3) ├── run "do something" (4) ├── upgrade cluster from "v22.2.3" to "v23.1.10" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (5) @@ -37,7 +37,7 @@ Plan: │ │ └── restart node 4 with binary version v23.1.10 (10) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (11) │ ├── run "my mixed-version feature" (12) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '23.1' (13) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (13) ├── upgrade cluster from "v23.1.10" to "v23.2.4" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (14) │ ├── upgrade nodes :1-4 from "v23.1.10" to "v23.2.4" @@ -51,7 +51,7 @@ Plan: │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (22) │ ├── run "my mixed-version feature" (23) │ ├── reset cluster setting "test_cluster_setting" on system tenant (24) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '23.2' (25) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on system tenant (25) └── upgrade cluster from "v23.2.4" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (26) ├── upgrade nodes :1-4 from "v23.2.4" to "" @@ -79,4 +79,4 @@ Plan: ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (45) ├── set cluster setting "test_cluster_setting" to '1' on system tenant (46) ├── reset cluster setting "test_cluster_setting" on system tenant (47) - └── wait for system tenant on nodes :1-4 to reach cluster version (48) + └── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (48) diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/conflicting_mutators b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/conflicting_mutators index 5d9118d57b7b..e6dc57a22c6e 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/conflicting_mutators +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/conflicting_mutators @@ -26,7 +26,7 @@ Mutators: concurrent_user_hooks_mutator, remove_user_hooks_mutator Plan: ├── install fixtures for version "v24.2.2" (1) [stage=system-setup] ├── start cluster at version "v24.2.2" (2) [stage=system-setup] -├── wait for system tenant on nodes :1-4 to reach cluster version '24.2' (3) [stage=system-setup] +├── wait for all nodes (:1-4) to acknowledge cluster version '24.2' on system tenant (3) [stage=system-setup] └── upgrade cluster from "v24.2.2" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) [stage=init] ├── upgrade nodes :1-4 from "v24.2.2" to "" @@ -48,4 +48,4 @@ Plan: │ ├── testSingleStep (18) [stage=last-upgrade] │ └── restart node 2 with binary version (19) [stage=last-upgrade] ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (20) [stage=running-upgrade-migrations,finalizing] - └── wait for system tenant on nodes :1-4 to reach cluster version (21) [stage=running-upgrade-migrations,finalizing] + └── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (21) [stage=running-upgrade-migrations,finalizing] diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/local_runs_reduced_wait_time b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/local_runs_reduced_wait_time index 1760f79f46d8..537879db8e23 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/local_runs_reduced_wait_time +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/local_runs_reduced_wait_time @@ -33,7 +33,7 @@ Deployment mode: system-only Plan: ├── install fixtures for version "v22.2.3" (1) ├── start cluster at version "v22.2.3" (2) -├── wait for system tenant on nodes :1-4 to reach cluster version '22.2' (3) +├── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on system tenant (3) ├── upgrade cluster from "v22.2.3" to "v23.1.4" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) │ ├── upgrade nodes :1-4 from "v22.2.3" to "v23.1.4" @@ -43,7 +43,7 @@ Plan: │ │ ├── restart node 2 with binary version v23.1.4 (8) │ │ └── restart node 3 with binary version v23.1.4 (9) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (10) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '23.1' (11) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (11) ├── run "initialize bank workload" (12) ├── start background hooks concurrently │ ├── run "bank workload", after 500ms delay (13) @@ -73,7 +73,7 @@ Plan: │ │ ├── restart node 2 with binary version v23.2.0 (32) │ │ └── restart node 3 with binary version v23.2.0 (33) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (34) -│ ├── wait for system tenant on nodes :1-4 to reach cluster version '23.2' (35) +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on system tenant (35) │ └── run "validate upgrade" (36) └── upgrade cluster from "v23.2.0" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (37) @@ -99,5 +99,5 @@ Plan: │ ├── restart node 2 with binary version (54) │ └── restart node 1 with binary version (55) ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (56) - ├── wait for system tenant on nodes :1-4 to reach cluster version (57) + ├── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (57) └── run "validate upgrade" (58) diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/minimum_supported_version b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/minimum_supported_version index a0fec385bf27..efdf451f7177 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/minimum_supported_version +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/minimum_supported_version @@ -33,7 +33,7 @@ Deployment mode: system-only Plan: ├── install fixtures for version "v21.2.11" (1) ├── start cluster at version "v21.2.11" (2) -├── wait for system tenant on nodes :1-4 to reach cluster version '21.2' (3) +├── wait for all nodes (:1-4) to acknowledge cluster version '21.2' on system tenant (3) ├── upgrade cluster from "v21.2.11" to "v22.1.8" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) │ ├── upgrade nodes :1-4 from "v21.2.11" to "v22.1.8" @@ -43,7 +43,7 @@ Plan: │ │ ├── restart node 4 with binary version v22.1.8 (8) │ │ └── restart node 2 with binary version v22.1.8 (9) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (10) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.1' (11) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.1' on system tenant (11) ├── upgrade cluster from "v22.1.8" to "v22.2.3" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (12) │ ├── upgrade nodes :1-4 from "v22.1.8" to "v22.2.3" @@ -52,7 +52,7 @@ Plan: │ │ ├── restart node 4 with binary version v22.2.3 (15) │ │ └── restart node 2 with binary version v22.2.3 (16) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (17) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.2' (18) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on system tenant (18) ├── upgrade cluster from "v22.2.3" to "v23.1.4" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (19) │ ├── upgrade nodes :1-4 from "v22.2.3" to "v23.1.4" @@ -62,7 +62,7 @@ Plan: │ │ ├── wait for 1m0s (23) │ │ └── restart node 3 with binary version v23.1.4 (24) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (25) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '23.1' (26) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (26) ├── run "initialize bank workload" (27) ├── start background hooks concurrently │ ├── run "bank workload", after 3m0s delay (28) @@ -77,7 +77,7 @@ Plan: │ │ ├── run "mixed-version 2" (35) │ │ └── restart node 4 with binary version v23.2.0 (36) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (37) -│ ├── wait for system tenant on nodes :1-4 to reach cluster version '23.2' (38) +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on system tenant (38) │ └── run "validate upgrade" (39) └── upgrade cluster from "v23.2.0" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (40) @@ -103,5 +103,5 @@ Plan: │ └── restart node 4 with binary version (57) ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (58) ├── run "mixed-version 1" (59) - ├── wait for system tenant on nodes :1-4 to reach cluster version (60) + ├── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (60) └── run "validate upgrade" (61) diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/multiple_upgrades b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/multiple_upgrades index b903831ee626..05fdf68754b5 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/multiple_upgrades +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/multiple_upgrades @@ -21,7 +21,7 @@ Deployment mode: system-only Plan: ├── install fixtures for version "v22.1.8" (1) ├── start cluster at version "v22.1.8" (2) -├── wait for system tenant on nodes :1-4 to reach cluster version '22.1' (3) +├── wait for all nodes (:1-4) to acknowledge cluster version '22.1' on system tenant (3) ├── upgrade cluster from "v22.1.8" to "v22.2.3" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) │ ├── upgrade nodes :1-4 from "v22.1.8" to "v22.2.3" @@ -31,7 +31,7 @@ Plan: │ │ ├── restart node 2 with binary version v22.2.3 (8) │ │ └── restart node 3 with binary version v22.2.3 (9) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (10) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.2' (11) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on system tenant (11) ├── run "initialize bank workload" (12) ├── run "bank workload" (13) ├── upgrade cluster from "v22.2.3" to "v23.1.4" @@ -56,7 +56,7 @@ Plan: │ │ └── restart node 1 with binary version v23.1.4 (29) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (30) │ ├── run "mixed-version 1" (31) -│ └── wait for system tenant on nodes :1-4 to reach cluster version '23.1' (32) +│ └── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (32) └── upgrade cluster from "v23.1.4" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (33) ├── upgrade nodes :1-4 from "v23.1.4" to "" @@ -79,4 +79,4 @@ Plan: │ └── run "mixed-version 1" (48) ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (49) ├── run "mixed-version 1" (50) - └── wait for system tenant on nodes :1-4 to reach cluster version (51) + └── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (51) diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/mutator_probabilities b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/mutator_probabilities index 9ab71c648b3b..d4283838bf3b 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/mutator_probabilities +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/mutator_probabilities @@ -22,7 +22,7 @@ Mutators: concurrent_user_hooks_mutator Plan: ├── install fixtures for version "v24.2.2" (1) [stage=system-setup] ├── start cluster at version "v24.2.2" (2) [stage=system-setup] -├── wait for system tenant on nodes :1-4 to reach cluster version '24.2' (3) [stage=system-setup] +├── wait for all nodes (:1-4) to acknowledge cluster version '24.2' on system tenant (3) [stage=system-setup] └── upgrade cluster from "v24.2.2" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) [stage=init] ├── upgrade nodes :1-4 from "v24.2.2" to "" @@ -50,4 +50,4 @@ Plan: │ │ └── testSingleStep, after 0s delay (21) [stage=last-upgrade] │ └── restart node 2 with binary version (22) [stage=last-upgrade] ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (23) [stage=running-upgrade-migrations,finalizing] - └── wait for system tenant on nodes :1-4 to reach cluster version (24) [stage=running-upgrade-migrations,finalizing] + └── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (24) [stage=running-upgrade-migrations,finalizing] diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/preserve_downgrade_option_randomizer b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/preserve_downgrade_option_randomizer index 3d975b3541e5..b229308fb447 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/preserve_downgrade_option_randomizer +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/preserve_downgrade_option_randomizer @@ -25,7 +25,7 @@ Mutators: preserve_downgrade_option_randomizer Plan: ├── install fixtures for version "v21.2.29" (1) [stage=system-setup] ├── start cluster at version "v21.2.29" (2) [stage=system-setup] -├── wait for system tenant on nodes :1-4 to reach cluster version '21.2' (3) [stage=system-setup] +├── wait for all nodes (:1-4) to acknowledge cluster version '21.2' on system tenant (3) [stage=system-setup] ├── upgrade cluster from "v21.2.29" to "v22.1.8" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) [stage=init] │ ├── upgrade nodes :1-4 from "v21.2.29" to "v22.1.8" @@ -35,7 +35,7 @@ Plan: │ │ ├── wait for 5m0s (8) [stage=last-upgrade] │ │ ├── restart node 1 with binary version v22.1.8 (9) [stage=last-upgrade] │ │ └── restart node 3 with binary version v22.1.8 (10) [stage=last-upgrade] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.1' (11) [stage=running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.1' on system tenant (11) [stage=running-upgrade-migrations,finalizing] ├── upgrade cluster from "v22.1.8" to "v22.2.3" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (12) [stage=init] │ ├── upgrade nodes :1-4 from "v22.1.8" to "v22.2.3" @@ -44,7 +44,7 @@ Plan: │ │ ├── restart node 2 with binary version v22.2.3 (15) [stage=last-upgrade] │ │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (16) [stage=last-upgrade] │ │ └── restart node 3 with binary version v22.2.3 (17) [stage=last-upgrade] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.2' (18) [stage=running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on system tenant (18) [stage=running-upgrade-migrations,finalizing] ├── run "do something" (19) [stage=on-startup] ├── upgrade cluster from "v22.2.3" to "v23.1.10" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (20) [stage=init] @@ -55,7 +55,7 @@ Plan: │ │ ├── restart node 4 with binary version v23.1.10 (24) [stage=last-upgrade] │ │ └── restart node 2 with binary version v23.1.10 (25) [stage=last-upgrade] │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (26) [stage=running-upgrade-migrations,finalizing] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '23.1' (27) [stage=running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (27) [stage=running-upgrade-migrations,finalizing] ├── upgrade cluster from "v23.1.10" to "v23.2.4" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (28) [stage=init] │ ├── upgrade nodes :1-4 from "v23.1.10" to "v23.2.4" @@ -65,7 +65,7 @@ Plan: │ │ ├── run "my mixed-version feature" (32) [stage=last-upgrade] │ │ ├── restart node 1 with binary version v23.2.4 (33) [stage=last-upgrade] │ │ └── restart node 2 with binary version v23.2.4 (34) [stage=last-upgrade] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '23.2' (35) [stage=running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on system tenant (35) [stage=running-upgrade-migrations,finalizing] └── upgrade cluster from "v23.2.4" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (36) [stage=init] ├── upgrade nodes :1-4 from "v23.2.4" to "" @@ -87,4 +87,4 @@ Plan: │ ├── restart node 1 with binary version (50) [stage=last-upgrade] │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (51) [stage=last-upgrade] │ └── restart node 4 with binary version (52) [stage=last-upgrade] - └── wait for system tenant on nodes :1-4 to reach cluster version (53) [stage=running-upgrade-migrations,finalizing] + └── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (53) [stage=running-upgrade-migrations,finalizing] diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/separate_process b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/separate_process new file mode 100644 index 000000000000..cbd4ae4f310d --- /dev/null +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/separate_process @@ -0,0 +1,194 @@ +# Test that we are able to generate test plans for tests using a +# separate-process deployment mode. + +mixed-version-test predecessors=(21.2.11, 22.1.8, 22.2.3, 23.1.4, 23.2.0) num_upgrades=5 deployment_mode=separate-process +---- +ok + +on-startup name=(create tables) +---- +ok + +in-mixed-version name=(mixed-version 1) +---- +ok + +in-mixed-version name=(mixed-version 2) +---- +ok + +workload name=bank +---- +ok + + +after-upgrade-finalized name=(validate upgrade) +---- +ok + +plan debug=true +---- +Seed: 12345 +Upgrades: v21.2.11 → v22.1.8 → v22.2.3 → v23.1.4 → v23.2.0 → +Deployment mode: separate-process +Plan: +├── install fixtures for version "v21.2.11" (1) [stage=system:system-setup;tenant:system-setup] +├── start cluster at version "v21.2.11" (2) [stage=system:system-setup;tenant:system-setup] +├── wait for all nodes (:1-4) to acknowledge cluster version '21.2' on system tenant (3) [stage=system:system-setup;tenant:system-setup] +├── upgrade cluster from "v21.2.11" to "v22.1.8" +│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) [stage=system:init;tenant:upgrading-system] +│ ├── upgrade nodes :1-4 from "v21.2.11" to "v22.1.8" +│ │ ├── restart system server on node 4 with binary version v22.1.8 (5) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ ├── restart system server on node 1 with binary version v22.1.8 (6) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ ├── restart system server on node 2 with binary version v22.1.8 (7) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ ├── wait for 5m0s (8) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ └── restart system server on node 3 with binary version v22.1.8 (9) [stage=system:last-upgrade;tenant:upgrading-system] +│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (10) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.1' on system tenant (11) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +├── upgrade cluster from "v22.1.8" to "v22.2.3" +│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (12) [stage=system:init;tenant:upgrading-system] +│ ├── upgrade nodes :1-4 from "v22.1.8" to "v22.2.3" +│ │ ├── restart system server on node 1 with binary version v22.2.3 (13) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ ├── restart system server on node 3 with binary version v22.2.3 (14) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ ├── restart system server on node 4 with binary version v22.2.3 (15) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ ├── wait for 5m0s (16) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ └── restart system server on node 2 with binary version v22.2.3 (17) [stage=system:last-upgrade;tenant:upgrading-system] +│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (18) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on system tenant (19) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +├── start separate process virtual cluster mixed-version-tenant-cyvju with binary version v22.2.3 (20) [stage=system:tenant-setup;tenant:tenant-setup] +├── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on mixed-version-tenant-cyvju tenant (21) [stage=system:tenant-setup;tenant:tenant-setup] +├── set cluster setting "spanconfig.tenant_limit" to '50000' on mixed-version-tenant-cyvju tenant (22) [stage=system:tenant-setup;tenant:tenant-setup] +├── set cluster setting "server.secondary_tenants.authorization.mode" to 'allow-all' on system tenant (23) [stage=system:tenant-setup;tenant:tenant-setup] +├── delete all-tenants override for the `version` key (24) [stage=system:tenant-setup;tenant:tenant-setup] +├── run startup hooks concurrently +│ ├── run "create tables", after 3m0s delay (25) [stage=system:on-startup;tenant:on-startup] +│ └── run "initialize bank workload", after 100ms delay (26) [stage=system:on-startup;tenant:on-startup] +├── run "bank workload" (27) [stage=system:background;tenant:background] +├── upgrade cluster from "v22.2.3" to "v23.1.4" +│ ├── upgrade storage cluster +│ │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (28) [stage=system:init;tenant:upgrading-system] +│ │ ├── upgrade nodes :1-4 from "v22.2.3" to "v23.1.4" +│ │ │ ├── restart system server on node 3 with binary version v23.1.4 (29) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ ├── restart system server on node 4 with binary version v23.1.4 (30) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ ├── restart system server on node 1 with binary version v23.1.4 (31) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ ├── run "mixed-version 1" (32) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ ├── restart system server on node 2 with binary version v23.1.4 (33) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ └── run "mixed-version 2" (34) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (35) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ │ ├── run mixed-version hooks concurrently +│ │ │ ├── run "mixed-version 1", after 30s delay (36) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ │ │ └── run "mixed-version 2", after 0s delay (37) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ │ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (38) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ │ └── run "validate upgrade" (39) [stage=system:after-upgrade-finished;tenant:upgrading-system] +│ └── upgrade tenant +│ ├── upgrade nodes :1-4 from "v22.2.3" to "v23.1.4" +│ │ ├── restart mixed-version-tenant-cyvju server on node 2 with binary version v23.1.4 (40) [stage=system:upgrading-tenant;tenant:temporary-upgrade] +│ │ ├── restart mixed-version-tenant-cyvju server on node 3 with binary version v23.1.4 (41) [stage=system:upgrading-tenant;tenant:temporary-upgrade] +│ │ ├── run mixed-version hooks concurrently +│ │ │ ├── run "mixed-version 1", after 0s delay (42) [stage=system:upgrading-tenant;tenant:temporary-upgrade] +│ │ │ └── run "mixed-version 2", after 5s delay (43) [stage=system:upgrading-tenant;tenant:temporary-upgrade] +│ │ ├── restart mixed-version-tenant-cyvju server on node 1 with binary version v23.1.4 (44) [stage=system:upgrading-tenant;tenant:temporary-upgrade] +│ │ └── restart mixed-version-tenant-cyvju server on node 4 with binary version v23.1.4 (45) [stage=system:upgrading-tenant;tenant:temporary-upgrade] +│ ├── downgrade nodes :1-4 from "v23.1.4" to "v22.2.3" +│ │ ├── restart mixed-version-tenant-cyvju server on node 2 with binary version v22.2.3 (46) [stage=system:upgrading-tenant;tenant:rollback-upgrade] +│ │ ├── restart mixed-version-tenant-cyvju server on node 1 with binary version v22.2.3 (47) [stage=system:upgrading-tenant;tenant:rollback-upgrade] +│ │ ├── run "mixed-version 1" (48) [stage=system:upgrading-tenant;tenant:rollback-upgrade] +│ │ ├── restart mixed-version-tenant-cyvju server on node 4 with binary version v22.2.3 (49) [stage=system:upgrading-tenant;tenant:rollback-upgrade] +│ │ └── restart mixed-version-tenant-cyvju server on node 3 with binary version v22.2.3 (50) [stage=system:upgrading-tenant;tenant:rollback-upgrade] +│ ├── upgrade nodes :1-4 from "v22.2.3" to "v23.1.4" +│ │ ├── restart mixed-version-tenant-cyvju server on node 2 with binary version v23.1.4 (51) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ ├── run "mixed-version 2" (52) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ ├── restart mixed-version-tenant-cyvju server on node 1 with binary version v23.1.4 (53) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ ├── restart mixed-version-tenant-cyvju server on node 3 with binary version v23.1.4 (54) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ ├── run "mixed-version 1" (55) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ └── restart mixed-version-tenant-cyvju server on node 4 with binary version v23.1.4 (56) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ ├── run following steps concurrently +│ │ ├── set `version` to '23.1' on mixed-version-tenant-cyvju tenant, after 5s delay (57) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] +│ │ ├── run "mixed-version 1", after 0s delay (58) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] +│ │ └── run "mixed-version 2", after 5s delay (59) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on mixed-version-tenant-cyvju tenant (60) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] +│ └── run "validate upgrade" (61) [stage=system:upgrading-tenant;tenant:after-upgrade-finished] +├── upgrade cluster from "v23.1.4" to "v23.2.0" +│ ├── upgrade storage cluster +│ │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (62) [stage=system:init;tenant:upgrading-system] +│ │ ├── upgrade nodes :1-4 from "v23.1.4" to "v23.2.0" +│ │ │ ├── restart system server on node 4 with binary version v23.2.0 (63) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ ├── run "mixed-version 2" (64) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ ├── restart system server on node 2 with binary version v23.2.0 (65) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ ├── restart system server on node 3 with binary version v23.2.0 (66) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ ├── run "mixed-version 1" (67) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ │ └── restart system server on node 1 with binary version v23.2.0 (68) [stage=system:last-upgrade;tenant:upgrading-system] +│ │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (69) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ │ ├── run mixed-version hooks concurrently +│ │ │ ├── run "mixed-version 1", after 0s delay (70) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ │ │ └── run "mixed-version 2", after 3m0s delay (71) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ │ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on system tenant (72) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] +│ │ └── run "validate upgrade" (73) [stage=system:after-upgrade-finished;tenant:upgrading-system] +│ └── upgrade tenant +│ ├── upgrade nodes :1-4 from "v23.1.4" to "v23.2.0" +│ │ ├── restart mixed-version-tenant-cyvju server on node 1 with binary version v23.2.0 (74) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ ├── run "mixed-version 2" (75) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ ├── restart mixed-version-tenant-cyvju server on node 4 with binary version v23.2.0 (76) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ ├── restart mixed-version-tenant-cyvju server on node 3 with binary version v23.2.0 (77) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ ├── run "mixed-version 1" (78) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ │ └── restart mixed-version-tenant-cyvju server on node 2 with binary version v23.2.0 (79) [stage=system:upgrading-tenant;tenant:last-upgrade] +│ ├── run following steps concurrently +│ │ ├── set `version` to '23.2' on mixed-version-tenant-cyvju tenant, after 500ms delay (80) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] +│ │ ├── run "mixed-version 1", after 5s delay (81) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] +│ │ └── run "mixed-version 2", after 0s delay (82) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on mixed-version-tenant-cyvju tenant (83) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] +│ └── run "validate upgrade" (84) [stage=system:upgrading-tenant;tenant:after-upgrade-finished] +└── upgrade cluster from "v23.2.0" to "" + ├── upgrade storage cluster + │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (85) [stage=system:init;tenant:upgrading-system] + │ ├── upgrade nodes :1-4 from "v23.2.0" to "" + │ │ ├── restart system server on node 3 with binary version (86) [stage=system:temporary-upgrade;tenant:upgrading-system] + │ │ ├── run "mixed-version 1" (87) [stage=system:temporary-upgrade;tenant:upgrading-system] + │ │ ├── restart system server on node 4 with binary version (88) [stage=system:temporary-upgrade;tenant:upgrading-system] + │ │ ├── restart system server on node 2 with binary version (89) [stage=system:temporary-upgrade;tenant:upgrading-system] + │ │ ├── run "mixed-version 2" (90) [stage=system:temporary-upgrade;tenant:upgrading-system] + │ │ └── restart system server on node 1 with binary version (91) [stage=system:temporary-upgrade;tenant:upgrading-system] + │ ├── downgrade nodes :1-4 from "" to "v23.2.0" + │ │ ├── restart system server on node 4 with binary version v23.2.0 (92) [stage=system:rollback-upgrade;tenant:upgrading-system] + │ │ ├── restart system server on node 1 with binary version v23.2.0 (93) [stage=system:rollback-upgrade;tenant:upgrading-system] + │ │ ├── restart system server on node 2 with binary version v23.2.0 (94) [stage=system:rollback-upgrade;tenant:upgrading-system] + │ │ └── restart system server on node 3 with binary version v23.2.0 (95) [stage=system:rollback-upgrade;tenant:upgrading-system] + │ ├── upgrade nodes :1-4 from "v23.2.0" to "" + │ │ ├── restart system server on node 1 with binary version (96) [stage=system:last-upgrade;tenant:upgrading-system] + │ │ ├── restart system server on node 4 with binary version (97) [stage=system:last-upgrade;tenant:upgrading-system] + │ │ ├── run "mixed-version 1" (98) [stage=system:last-upgrade;tenant:upgrading-system] + │ │ ├── restart system server on node 2 with binary version (99) [stage=system:last-upgrade;tenant:upgrading-system] + │ │ ├── run "mixed-version 2" (100) [stage=system:last-upgrade;tenant:upgrading-system] + │ │ └── restart system server on node 3 with binary version (101) [stage=system:last-upgrade;tenant:upgrading-system] + │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (102) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] + │ ├── run "mixed-version 2" (103) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] + │ ├── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (104) [stage=system:running-upgrade-migrations;tenant:upgrading-system,finalizing] + │ └── run "validate upgrade" (105) [stage=system:after-upgrade-finished;tenant:upgrading-system] + └── upgrade tenant + ├── upgrade nodes :1-4 from "v23.2.0" to "" + │ ├── restart mixed-version-tenant-cyvju server on node 4 with binary version (106) [stage=system:upgrading-tenant;tenant:temporary-upgrade] + │ ├── restart mixed-version-tenant-cyvju server on node 3 with binary version (107) [stage=system:upgrading-tenant;tenant:temporary-upgrade] + │ ├── restart mixed-version-tenant-cyvju server on node 2 with binary version (108) [stage=system:upgrading-tenant;tenant:temporary-upgrade] + │ ├── run "mixed-version 1" (109) [stage=system:upgrading-tenant;tenant:temporary-upgrade] + │ ├── restart mixed-version-tenant-cyvju server on node 1 with binary version (110) [stage=system:upgrading-tenant;tenant:temporary-upgrade] + │ └── run "mixed-version 2" (111) [stage=system:upgrading-tenant;tenant:temporary-upgrade] + ├── downgrade nodes :1-4 from "" to "v23.2.0" + │ ├── restart mixed-version-tenant-cyvju server on node 3 with binary version v23.2.0 (112) [stage=system:upgrading-tenant;tenant:rollback-upgrade] + │ ├── restart mixed-version-tenant-cyvju server on node 2 with binary version v23.2.0 (113) [stage=system:upgrading-tenant;tenant:rollback-upgrade] + │ ├── run "mixed-version 2" (114) [stage=system:upgrading-tenant;tenant:rollback-upgrade] + │ ├── restart mixed-version-tenant-cyvju server on node 1 with binary version v23.2.0 (115) [stage=system:upgrading-tenant;tenant:rollback-upgrade] + │ └── restart mixed-version-tenant-cyvju server on node 4 with binary version v23.2.0 (116) [stage=system:upgrading-tenant;tenant:rollback-upgrade] + ├── upgrade nodes :1-4 from "v23.2.0" to "" + │ ├── restart mixed-version-tenant-cyvju server on node 3 with binary version (117) [stage=system:upgrading-tenant;tenant:last-upgrade] + │ ├── restart mixed-version-tenant-cyvju server on node 1 with binary version (118) [stage=system:upgrading-tenant;tenant:last-upgrade] + │ ├── restart mixed-version-tenant-cyvju server on node 2 with binary version (119) [stage=system:upgrading-tenant;tenant:last-upgrade] + │ ├── restart mixed-version-tenant-cyvju server on node 4 with binary version (120) [stage=system:upgrading-tenant;tenant:last-upgrade] + │ └── run mixed-version hooks concurrently + │ ├── run "mixed-version 1", after 30s delay (121) [stage=system:upgrading-tenant;tenant:last-upgrade] + │ └── run "mixed-version 2", after 100ms delay (122) [stage=system:upgrading-tenant;tenant:last-upgrade] + ├── run following steps concurrently + │ ├── set `version` to on mixed-version-tenant-cyvju tenant, after 0s delay (123) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] + │ ├── run "mixed-version 1", after 100ms delay (124) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] + │ └── run "mixed-version 2", after 500ms delay (125) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] + ├── wait for all nodes (:1-4) to acknowledge cluster version on mixed-version-tenant-cyvju tenant (126) [stage=system:upgrading-tenant;tenant:running-upgrade-migrations,finalizing] + └── run "validate upgrade" (127) [stage=system:upgrading-tenant;tenant:after-upgrade-finished] diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/shared_process b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/shared_process index dc85e396f5bb..58776990d49d 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/shared_process +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/shared_process @@ -32,87 +32,87 @@ Seed: 12345 Upgrades: v21.2.11 → v22.1.8 → v22.2.3 → v23.1.4 → v23.2.0 → Deployment mode: shared-process Plan: -├── install fixtures for version "v21.2.11" (1) [stage=system-setup] -├── start cluster at version "v21.2.11" (2) [stage=system-setup] -├── wait for system tenant on nodes :1-4 to reach cluster version '21.2' (3) [stage=system-setup] +├── install fixtures for version "v21.2.11" (1) [stage=system:system-setup;tenant:system-setup] +├── start cluster at version "v21.2.11" (2) [stage=system:system-setup;tenant:system-setup] +├── wait for all nodes (:1-4) to acknowledge cluster version '21.2' on system tenant (3) [stage=system:system-setup;tenant:system-setup] ├── upgrade cluster from "v21.2.11" to "v22.1.8" -│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) [stage=init] +│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) [stage=system:init;tenant:init] │ ├── upgrade nodes :1-4 from "v21.2.11" to "v22.1.8" -│ │ ├── restart node 4 with binary version v22.1.8 (5) [stage=last-upgrade] -│ │ ├── restart node 1 with binary version v22.1.8 (6) [stage=last-upgrade] -│ │ ├── restart node 2 with binary version v22.1.8 (7) [stage=last-upgrade] -│ │ ├── wait for 5m0s (8) [stage=last-upgrade] -│ │ └── restart node 3 with binary version v22.1.8 (9) [stage=last-upgrade] -│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (10) [stage=running-upgrade-migrations,finalizing] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.1' (11) [stage=running-upgrade-migrations,finalizing] +│ │ ├── restart node 4 with binary version v22.1.8 (5) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── restart node 1 with binary version v22.1.8 (6) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── restart node 2 with binary version v22.1.8 (7) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── wait for 5m0s (8) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ └── restart node 3 with binary version v22.1.8 (9) [stage=system:last-upgrade;tenant:last-upgrade] +│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (10) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.1' on system tenant (11) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] ├── upgrade cluster from "v22.1.8" to "v22.2.3" -│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (12) [stage=init] +│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (12) [stage=system:init;tenant:init] │ ├── upgrade nodes :1-4 from "v22.1.8" to "v22.2.3" -│ │ ├── restart node 1 with binary version v22.2.3 (13) [stage=last-upgrade] -│ │ ├── restart node 3 with binary version v22.2.3 (14) [stage=last-upgrade] -│ │ ├── restart node 4 with binary version v22.2.3 (15) [stage=last-upgrade] -│ │ ├── wait for 5m0s (16) [stage=last-upgrade] -│ │ └── restart node 2 with binary version v22.2.3 (17) [stage=last-upgrade] -│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (18) [stage=running-upgrade-migrations,finalizing] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.2' (19) [stage=running-upgrade-migrations,finalizing] +│ │ ├── restart node 1 with binary version v22.2.3 (13) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── restart node 3 with binary version v22.2.3 (14) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── restart node 4 with binary version v22.2.3 (15) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── wait for 5m0s (16) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ └── restart node 2 with binary version v22.2.3 (17) [stage=system:last-upgrade;tenant:last-upgrade] +│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (18) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on system tenant (19) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] ├── upgrade cluster from "v22.2.3" to "v23.1.4" -│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (20) [stage=init] +│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (20) [stage=system:init;tenant:init] │ ├── upgrade nodes :1-4 from "v22.2.3" to "v23.1.4" -│ │ ├── restart node 3 with binary version v23.1.4 (21) [stage=last-upgrade] -│ │ ├── restart node 4 with binary version v23.1.4 (22) [stage=last-upgrade] -│ │ ├── restart node 1 with binary version v23.1.4 (23) [stage=last-upgrade] -│ │ ├── wait for 1m0s (24) [stage=last-upgrade] -│ │ └── restart node 2 with binary version v23.1.4 (25) [stage=last-upgrade] -│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (26) [stage=running-upgrade-migrations,finalizing] -│ ├── wait for system tenant on nodes :1-4 to reach cluster version '23.1' (27) [stage=running-upgrade-migrations,finalizing] -│ └── delete all-tenants override for the `version` key (28) [stage=running-upgrade-migrations,finalizing] +│ │ ├── restart node 3 with binary version v23.1.4 (21) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── restart node 4 with binary version v23.1.4 (22) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── restart node 1 with binary version v23.1.4 (23) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── wait for 1m0s (24) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ └── restart node 2 with binary version v23.1.4 (25) [stage=system:last-upgrade;tenant:last-upgrade] +│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (26) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (27) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] +│ └── delete all-tenants override for the `version` key (28) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] ├── upgrade cluster from "v23.1.4" to "v23.2.0" -│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (29) [stage=init] +│ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (29) [stage=system:init;tenant:init] │ ├── upgrade nodes :1-4 from "v23.1.4" to "v23.2.0" -│ │ ├── restart node 2 with binary version v23.2.0 (30) [stage=last-upgrade] -│ │ ├── wait for 10m0s (31) [stage=last-upgrade] -│ │ ├── restart node 3 with binary version v23.2.0 (32) [stage=last-upgrade] -│ │ ├── restart node 4 with binary version v23.2.0 (33) [stage=last-upgrade] -│ │ └── restart node 1 with binary version v23.2.0 (34) [stage=last-upgrade] -│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (35) [stage=running-upgrade-migrations,finalizing] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '23.2' (36) [stage=running-upgrade-migrations,finalizing] -├── start shared-process tenant "mixed-version-tenant-cyvju" (37) [stage=tenant-setup] -├── wait for mixed-version-tenant-cyvju tenant on nodes :1-4 to reach cluster version '23.2' (38) [stage=tenant-setup] -├── set cluster setting "server.controller.default_target_cluster" to 'mixed-version-tenant-cyvju' on system tenant (39) [stage=tenant-setup] -├── set cluster setting "server.secondary_tenants.authorization.mode" to 'allow-all' on system tenant (40) [stage=tenant-setup] -├── delete all-tenants override for the `version` key (41) [stage=tenant-setup] +│ │ ├── restart node 2 with binary version v23.2.0 (30) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── wait for 10m0s (31) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── restart node 3 with binary version v23.2.0 (32) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ ├── restart node 4 with binary version v23.2.0 (33) [stage=system:last-upgrade;tenant:last-upgrade] +│ │ └── restart node 1 with binary version v23.2.0 (34) [stage=system:last-upgrade;tenant:last-upgrade] +│ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (35) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on system tenant (36) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] +├── start shared-process tenant "mixed-version-tenant-cyvju" (37) [stage=system:tenant-setup;tenant:tenant-setup] +├── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on mixed-version-tenant-cyvju tenant (38) [stage=system:tenant-setup;tenant:tenant-setup] +├── set cluster setting "server.controller.default_target_cluster" to 'mixed-version-tenant-cyvju' on system tenant (39) [stage=system:tenant-setup;tenant:tenant-setup] +├── set cluster setting "server.secondary_tenants.authorization.mode" to 'allow-all' on system tenant (40) [stage=system:tenant-setup;tenant:tenant-setup] +├── delete all-tenants override for the `version` key (41) [stage=system:tenant-setup;tenant:tenant-setup] ├── run startup hooks concurrently -│ ├── run "create tables", after 5s delay (42) [stage=on-startup] -│ └── run "initialize bank workload", after 100ms delay (43) [stage=on-startup] -├── run "bank workload" (44) [stage=background] +│ ├── run "create tables", after 100ms delay (42) [stage=system:on-startup;tenant:on-startup] +│ └── run "initialize bank workload", after 30s delay (43) [stage=system:on-startup;tenant:on-startup] +├── run "bank workload" (44) [stage=system:background;tenant:background] └── upgrade cluster from "v23.2.0" to "" - ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (45) [stage=init] - ├── prevent auto-upgrades on mixed-version-tenant-cyvju tenant by setting `preserve_downgrade_option` (46) [stage=init] + ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (45) [stage=system:init;tenant:init] + ├── prevent auto-upgrades on mixed-version-tenant-cyvju tenant by setting `preserve_downgrade_option` (46) [stage=system:init;tenant:init] ├── upgrade nodes :1-4 from "v23.2.0" to "" - │ ├── restart node 3 with binary version (47) [stage=temporary-upgrade] - │ ├── run "mixed-version 2" (48) [stage=temporary-upgrade] - │ ├── restart node 4 with binary version (49) [stage=temporary-upgrade] - │ ├── restart node 2 with binary version (50) [stage=temporary-upgrade] - │ ├── restart node 1 with binary version (51) [stage=temporary-upgrade] - │ └── run "mixed-version 1" (52) [stage=temporary-upgrade] + │ ├── restart node 3 with binary version (47) [stage=system:temporary-upgrade;tenant:temporary-upgrade] + │ ├── run "mixed-version 2" (48) [stage=system:temporary-upgrade;tenant:temporary-upgrade] + │ ├── restart node 4 with binary version (49) [stage=system:temporary-upgrade;tenant:temporary-upgrade] + │ ├── restart node 2 with binary version (50) [stage=system:temporary-upgrade;tenant:temporary-upgrade] + │ ├── restart node 1 with binary version (51) [stage=system:temporary-upgrade;tenant:temporary-upgrade] + │ └── run "mixed-version 1" (52) [stage=system:temporary-upgrade;tenant:temporary-upgrade] ├── downgrade nodes :1-4 from "" to "v23.2.0" - │ ├── restart node 1 with binary version v23.2.0 (53) [stage=rollback-upgrade] + │ ├── restart node 1 with binary version v23.2.0 (53) [stage=system:rollback-upgrade;tenant:rollback-upgrade] │ ├── run mixed-version hooks concurrently - │ │ ├── run "mixed-version 1", after 3m0s delay (54) [stage=rollback-upgrade] - │ │ └── run "mixed-version 2", after 0s delay (55) [stage=rollback-upgrade] - │ ├── restart node 2 with binary version v23.2.0 (56) [stage=rollback-upgrade] - │ ├── restart node 3 with binary version v23.2.0 (57) [stage=rollback-upgrade] - │ └── restart node 4 with binary version v23.2.0 (58) [stage=rollback-upgrade] + │ │ ├── run "mixed-version 1", after 3m0s delay (54) [stage=system:rollback-upgrade;tenant:rollback-upgrade] + │ │ └── run "mixed-version 2", after 0s delay (55) [stage=system:rollback-upgrade;tenant:rollback-upgrade] + │ ├── restart node 2 with binary version v23.2.0 (56) [stage=system:rollback-upgrade;tenant:rollback-upgrade] + │ ├── restart node 3 with binary version v23.2.0 (57) [stage=system:rollback-upgrade;tenant:rollback-upgrade] + │ └── restart node 4 with binary version v23.2.0 (58) [stage=system:rollback-upgrade;tenant:rollback-upgrade] ├── upgrade nodes :1-4 from "v23.2.0" to "" - │ ├── restart node 2 with binary version (59) [stage=last-upgrade] - │ ├── restart node 4 with binary version (60) [stage=last-upgrade] - │ ├── restart node 3 with binary version (61) [stage=last-upgrade] - │ ├── run "mixed-version 1" (62) [stage=last-upgrade] - │ ├── restart node 1 with binary version (63) [stage=last-upgrade] - │ └── run "mixed-version 2" (64) [stage=last-upgrade] - ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (65) [stage=running-upgrade-migrations,finalizing] - ├── wait for system tenant on nodes :1-4 to reach cluster version (66) [stage=running-upgrade-migrations,finalizing] - ├── allow upgrade to happen on mixed-version-tenant-cyvju tenant by resetting `preserve_downgrade_option` (67) [stage=running-upgrade-migrations,finalizing] - ├── run "mixed-version 2" (68) [stage=running-upgrade-migrations,finalizing] - ├── wait for mixed-version-tenant-cyvju tenant on nodes :1-4 to reach cluster version (69) [stage=running-upgrade-migrations,finalizing] - └── run "validate upgrade" (70) [stage=after-upgrade-finished] + │ ├── restart node 2 with binary version (59) [stage=system:last-upgrade;tenant:last-upgrade] + │ ├── restart node 4 with binary version (60) [stage=system:last-upgrade;tenant:last-upgrade] + │ ├── restart node 3 with binary version (61) [stage=system:last-upgrade;tenant:last-upgrade] + │ ├── run "mixed-version 1" (62) [stage=system:last-upgrade;tenant:last-upgrade] + │ ├── restart node 1 with binary version (63) [stage=system:last-upgrade;tenant:last-upgrade] + │ └── run "mixed-version 2" (64) [stage=system:last-upgrade;tenant:last-upgrade] + ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (65) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] + ├── run "mixed-version 2" (66) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] + ├── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (67) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] + ├── allow upgrade to happen on mixed-version-tenant-cyvju tenant by resetting `preserve_downgrade_option` (68) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] + ├── wait for all nodes (:1-4) to acknowledge cluster version on mixed-version-tenant-cyvju tenant (69) [stage=system:running-upgrade-migrations;tenant:running-upgrade-migrations,finalizing] + └── run "validate upgrade" (70) [stage=system:after-upgrade-finished;tenant:after-upgrade-finished] diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/skip_version_upgrades b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/skip_version_upgrades index 16e154945db2..dc387eed58f6 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/skip_version_upgrades +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/skip_version_upgrades @@ -33,7 +33,7 @@ Deployment mode: system-only Plan: ├── install fixtures for version "v23.1.4" (1) ├── start cluster at version "v23.1.4" (2) -├── wait for system tenant on nodes :1-4 to reach cluster version '23.1' (3) +├── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (3) ├── run "initialize bank workload" (4) ├── start background hooks concurrently │ ├── run "bank workload", after 3m0s delay (5) @@ -52,7 +52,7 @@ Plan: │ ├── run mixed-version hooks concurrently │ │ ├── run "mixed-version 1", after 100ms delay (15) │ │ └── run "mixed-version 2", after 500ms delay (16) -│ ├── wait for system tenant on nodes :1-4 to reach cluster version '23.2' (17) +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on system tenant (17) │ └── run "validate upgrade" (18) ├── upgrade cluster from "v23.2.0" to "v24.1.1" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (19) @@ -76,7 +76,7 @@ Plan: │ │ ├── run "mixed-version 2" (34) │ │ └── restart node 4 with binary version v24.1.1 (35) │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (36) -│ ├── wait for system tenant on nodes :1-4 to reach cluster version '24.1' (37) +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '24.1' on system tenant (37) │ └── run "validate upgrade" (38) └── upgrade cluster from "v24.1.1" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (39) @@ -102,5 +102,5 @@ Plan: │ └── restart node 4 with binary version (56) ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (57) ├── run "mixed-version 1" (58) - ├── wait for system tenant on nodes :1-4 to reach cluster version (59) + ├── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (59) └── run "validate upgrade" (60) diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/step_stages b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/step_stages index 96bc87690f5a..2bc3ca085c1a 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/step_stages +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/testdata/planner/step_stages @@ -33,7 +33,7 @@ Deployment mode: system-only Plan: ├── install fixtures for version "v21.2.11" (1) [stage=system-setup] ├── start cluster at version "v21.2.11" (2) [stage=system-setup] -├── wait for system tenant on nodes :1-4 to reach cluster version '21.2' (3) [stage=system-setup] +├── wait for all nodes (:1-4) to acknowledge cluster version '21.2' on system tenant (3) [stage=system-setup] ├── upgrade cluster from "v21.2.11" to "v22.1.8" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (4) [stage=init] │ ├── upgrade nodes :1-4 from "v21.2.11" to "v22.1.8" @@ -43,7 +43,7 @@ Plan: │ │ ├── restart node 4 with binary version v22.1.8 (8) [stage=last-upgrade] │ │ └── restart node 2 with binary version v22.1.8 (9) [stage=last-upgrade] │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (10) [stage=running-upgrade-migrations,finalizing] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.1' (11) [stage=running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.1' on system tenant (11) [stage=running-upgrade-migrations,finalizing] ├── upgrade cluster from "v22.1.8" to "v22.2.3" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (12) [stage=init] │ ├── upgrade nodes :1-4 from "v22.1.8" to "v22.2.3" @@ -52,7 +52,7 @@ Plan: │ │ ├── restart node 4 with binary version v22.2.3 (15) [stage=last-upgrade] │ │ └── restart node 2 with binary version v22.2.3 (16) [stage=last-upgrade] │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (17) [stage=running-upgrade-migrations,finalizing] -│ └── wait for system tenant on nodes :1-4 to reach cluster version '22.2' (18) [stage=running-upgrade-migrations,finalizing] +│ └── wait for all nodes (:1-4) to acknowledge cluster version '22.2' on system tenant (18) [stage=running-upgrade-migrations,finalizing] ├── run "initialize bank workload" (19) [stage=on-startup] ├── start background hooks concurrently │ ├── run "bank workload", after 30s delay (20) [stage=background] @@ -68,7 +68,7 @@ Plan: │ │ └── run "mixed-version 2" (28) [stage=last-upgrade] │ ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (29) [stage=running-upgrade-migrations,finalizing] │ ├── run "mixed-version 2" (30) [stage=running-upgrade-migrations,finalizing] -│ ├── wait for system tenant on nodes :1-4 to reach cluster version '23.1' (31) [stage=running-upgrade-migrations,finalizing] +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.1' on system tenant (31) [stage=running-upgrade-migrations,finalizing] │ └── run "validate upgrade" (32) [stage=after-upgrade-finished] ├── upgrade cluster from "v23.1.4" to "v23.2.0" │ ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (33) [stage=init] @@ -83,7 +83,7 @@ Plan: │ ├── run mixed-version hooks concurrently │ │ ├── run "mixed-version 1", after 0s delay (41) [stage=running-upgrade-migrations,finalizing] │ │ └── run "mixed-version 2", after 5s delay (42) [stage=running-upgrade-migrations,finalizing] -│ ├── wait for system tenant on nodes :1-4 to reach cluster version '23.2' (43) [stage=running-upgrade-migrations,finalizing] +│ ├── wait for all nodes (:1-4) to acknowledge cluster version '23.2' on system tenant (43) [stage=running-upgrade-migrations,finalizing] │ └── run "validate upgrade" (44) [stage=after-upgrade-finished] └── upgrade cluster from "v23.2.0" to "" ├── prevent auto-upgrades on system tenant by setting `preserve_downgrade_option` (45) [stage=init] @@ -110,5 +110,5 @@ Plan: │ │ └── run "mixed-version 2", after 5s delay (62) [stage=last-upgrade] │ └── restart node 4 with binary version (63) [stage=last-upgrade] ├── allow upgrade to happen on system tenant by resetting `preserve_downgrade_option` (64) [stage=running-upgrade-migrations,finalizing] - ├── wait for system tenant on nodes :1-4 to reach cluster version (65) [stage=running-upgrade-migrations,finalizing] + ├── wait for all nodes (:1-4) to acknowledge cluster version on system tenant (65) [stage=running-upgrade-migrations,finalizing] └── run "validate upgrade" (66) [stage=after-upgrade-finished] diff --git a/pkg/cmd/roachtest/tests/follower_reads.go b/pkg/cmd/roachtest/tests/follower_reads.go index f61d96b47a63..89ed1567447d 100644 --- a/pkg/cmd/roachtest/tests/follower_reads.go +++ b/pkg/cmd/roachtest/tests/follower_reads.go @@ -791,7 +791,9 @@ func verifyHighFollowerReadRatios( adminNode = i break } - adminURLs, err := c.ExternalAdminUIAddr(ctx, l, c.Node(adminNode)) + adminURLs, err := c.ExternalAdminUIAddr( + ctx, l, c.Node(adminNode), option.VirtualClusterName(install.SystemInterfaceName), + ) require.NoError(t, err) url := "https://" + adminURLs[0] + "/ts/query" @@ -902,7 +904,9 @@ func getFollowerReadCounts(ctx context.Context, t test.Test, c cluster.Cluster) followerReadCounts := make([]int, c.Spec().NodeCount) getFollowerReadCount := func(ctx context.Context, node int) func() error { return func() error { - adminUIAddrs, err := c.ExternalAdminUIAddr(ctx, t.L(), c.Node(node)) + adminUIAddrs, err := c.ExternalAdminUIAddr( + ctx, t.L(), c.Node(node), option.VirtualClusterName(install.SystemInterfaceName), + ) if err != nil { return err } @@ -980,7 +984,16 @@ func runFollowerReadsMixedVersionSingleRegionTest( ctx context.Context, t test.Test, c cluster.Cluster, ) { topology := topologySpec{multiRegion: false} - runFollowerReadsMixedVersionTest(ctx, t, c, topology, exactStaleness) + runFollowerReadsMixedVersionTest(ctx, t, c, topology, exactStaleness, + // This test does not currently work with shared-process + // deployments (#129546), so we do not run it in separate-process + // mode either to reduce noise. We should reevaluate once the test + // works in shared-process. + mixedversion.EnabledDeploymentModes( + mixedversion.SystemOnlyDeployment, + mixedversion.SharedProcessDeployment, + ), + ) } // runFollowerReadsMixedVersionGlobalTableTest runs a multi-region follower-read @@ -1000,6 +1013,14 @@ func runFollowerReadsMixedVersionGlobalTableTest( // Use a longer upgrade timeout to give the migrations enough time to finish // considering the cross-region latency. mixedversion.UpgradeTimeout(60*time.Minute), + // This test does not currently work with shared-process + // deployments (#129167), so we do not run it in separate-process + // mode either to reduce noise. We should reevaluate once the test + // works in shared-process. + mixedversion.EnabledDeploymentModes( + mixedversion.SystemOnlyDeployment, + mixedversion.SharedProcessDeployment, + ), ) } diff --git a/pkg/cmd/roachtest/tests/mixed_version_backup.go b/pkg/cmd/roachtest/tests/mixed_version_backup.go index cf30ef76ff09..5f8ac0639ddd 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_backup.go +++ b/pkg/cmd/roachtest/tests/mixed_version_backup.go @@ -2182,11 +2182,16 @@ func (u *CommonTestUtils) enableJobAdoption( func (mvb *mixedVersionBackup) planAndRunBackups( ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper, ) error { + upgradingService := h.DefaultService() + if upgradingService.Stage == mixedversion.UpgradingSystemStage { + upgradingService = h.System + } + onPrevious := labeledNodes{ - Nodes: h.Context().NodesInPreviousVersion(), Version: sanitizeVersionForBackup(h.Context().FromVersion), + Nodes: upgradingService.NodesInPreviousVersion(), Version: sanitizeVersionForBackup(h.Context().FromVersion), } onNext := labeledNodes{ - Nodes: h.Context().NodesInNextVersion(), Version: sanitizeVersionForBackup(h.Context().ToVersion), + Nodes: upgradingService.NodesInNextVersion(), Version: sanitizeVersionForBackup(h.Context().ToVersion), } onRandom := labeledNodes{Nodes: mvb.roachNodes, Version: "random node"} defaultPauseProbability := 0.2 @@ -2516,8 +2521,12 @@ func (mvb *mixedVersionBackup) verifySomeBackups( func (mvb *mixedVersionBackup) verifyAllBackups( ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper, ) error { - l.Printf("stopping background functions and workloads") - mvb.stopBackground() + isFinalUpgrade := h.Context().ToVersion.IsCurrent() + + if isFinalUpgrade { + l.Printf("stopping background functions and workloads") + mvb.stopBackground() + } u, err := mvb.CommonTestUtils(ctx, h) if err != nil { @@ -2525,11 +2534,11 @@ func (mvb *mixedVersionBackup) verifyAllBackups( } var restoreErrors []error - verify := func(version *clusterupgrade.Version) { - l.Printf("%s: verifying %d collections created during this test", version.String(), len(mvb.collections)) + verify := func(v *clusterupgrade.Version) { + l.Printf("%s: verifying %d collections created during this test", v, len(mvb.collections)) for _, collection := range mvb.collections { - if !version.IsCurrent() && strings.Contains(collection.name, finalizingLabel) { + if v.Equal(h.Context().FromVersion) && strings.Contains(collection.name, finalizingLabel) { // Do not attempt to restore, in the previous version, a // backup that was taken while the cluster was finalizing, as // that will most likely fail (the backup version will be past @@ -2537,10 +2546,18 @@ func (mvb *mixedVersionBackup) verifyAllBackups( continue } - if _, ok := collection.btype.(*clusterBackup); ok { - err := u.resetCluster(ctx, l, version, h.ExpectDeaths, []install.ClusterSettingOption{}) + _, isClusterBackup := collection.btype.(*clusterBackup) + if isClusterBackup && !isFinalUpgrade { + // We only verify cluster backups once we upgraded all the way + // to the final version in this test. Wiping and restarting + // nodes does not work well with the mixedversion framework. + continue + } + + if isClusterBackup { + err := u.resetCluster(ctx, l, v, h.ExpectDeaths, []install.ClusterSettingOption{}) if err != nil { - err := errors.Wrapf(err, "%s", version) + err := errors.Wrapf(err, "%s", v) l.Printf("error resetting cluster: %v", err) restoreErrors = append(restoreErrors, err) continue @@ -2565,7 +2582,7 @@ func (mvb *mixedVersionBackup) verifyAllBackups( } if err := collection.verifyBackupCollection(ctx, l, rng, mvb.backupRestoreTestDriver, checkFiles, internalSystemJobs); err != nil { - err := errors.Wrapf(err, "%s", version) + err := errors.Wrapf(err, "%s", v) l.Printf("restore error: %v", err) // Attempt to collect logs and debug.zip at the time of this // restore failure; if we can't, log the error encountered and @@ -2647,6 +2664,12 @@ func registerBackupMixedVersion(r registry.Registry) { // attempted. mixedversion.UpgradeTimeout(30*time.Minute), mixedversion.AlwaysUseLatestPredecessors, + // This test sometimes flake on separate-process + // deployments. Needs investigation. + mixedversion.EnabledDeploymentModes( + mixedversion.SystemOnlyDeployment, + mixedversion.SharedProcessDeployment, + ), // We disable cluster setting mutators because this test // resets the cluster to older versions when verifying cluster // backups. This makes the mixed-version context inaccurate @@ -2704,7 +2727,7 @@ func registerBackupMixedVersion(r registry.Registry) { mvt.InMixedVersion("plan and run backups", backupTest.planAndRunBackups) mvt.InMixedVersion("verify some backups", backupTest.verifySomeBackups) - mvt.AfterUpgradeFinalized("verify all backups", backupTest.verifyAllBackups) + mvt.AfterUpgradeFinalized("maybe verify all backups", backupTest.verifyAllBackups) backupTest.stopBackground = func() { stopBank() diff --git a/pkg/cmd/roachtest/tests/mixed_version_cdc.go b/pkg/cmd/roachtest/tests/mixed_version_cdc.go index 9f2544aecfac..c25aff0ebe79 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_cdc.go +++ b/pkg/cmd/roachtest/tests/mixed_version_cdc.go @@ -426,9 +426,8 @@ func (cmvt *cdcMixedVersionTester) muxRangeFeedSupported( ) (bool, option.NodeListOption, error) { // changefeed.mux_rangefeed.enabled was added in 22.2 and deleted in 24.1. return canMixedVersionUseDeletedClusterSetting(h, - false, /* isSystem */ clusterupgrade.MustParseVersion("v22.2.0"), - clusterupgrade.MustParseVersion("v24.1.0-alpha.00000000"), + clusterupgrade.MustParseVersion("v24.1.0"), ) } @@ -448,21 +447,17 @@ func (cmvt *cdcMixedVersionTester) distributionStrategySupported( return h.ClusterVersionAtLeast(r, v241CV) } -// canMixedVersionUseDeletedClusterSetting returns whether a mixed-version -// cluster can use a deleted cluster setting. If it returns true, it will -// also return the subset of nodes that understand the setting. +// canMixedVersionUseDeletedClusterSetting returns whether a +// mixed-version cluster can use a deleted (system) cluster +// setting. If it returns true, it will also return the subset of +// nodes that understand the setting. func canMixedVersionUseDeletedClusterSetting( h *mixedversion.Helper, - isSystem bool, addedVersion *clusterupgrade.Version, deletedVersion *clusterupgrade.Version, ) (bool, option.NodeListOption, error) { fromVersion := h.System.FromVersion toVersion := h.System.ToVersion - if !isSystem { - fromVersion = h.DefaultService().FromVersion - toVersion = h.DefaultService().ToVersion - } // Cluster setting was deleted at or before the from version so no nodes // know about the setting. @@ -475,7 +470,7 @@ func canMixedVersionUseDeletedClusterSetting( // all the nodes on that version will know about the setting. if toVersion.AtLeast(deletedVersion) { if fromVersion.AtLeast(addedVersion) { - fromVersionNodes := h.Context().NodesInPreviousVersion() + fromVersionNodes := h.System.NodesInPreviousVersion() if len(fromVersionNodes) > 0 { return true, fromVersionNodes, nil } @@ -487,11 +482,11 @@ func canMixedVersionUseDeletedClusterSetting( // at least the added version will know about the setting. if fromVersion.AtLeast(addedVersion) { - return true, h.Context().Descriptor.Nodes, nil + return true, h.System.Descriptor.Nodes, nil } if toVersion.AtLeast(addedVersion) { - toVersionNodes := h.Context().NodesInNextVersion() + toVersionNodes := h.System.NodesInNextVersion() if len(toVersionNodes) > 0 { return true, toVersionNodes, nil } diff --git a/pkg/cmd/roachtest/tests/mixed_version_import.go b/pkg/cmd/roachtest/tests/mixed_version_import.go index 3dcb1e4f48f7..6f30f24a68f1 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_import.go +++ b/pkg/cmd/roachtest/tests/mixed_version_import.go @@ -52,6 +52,12 @@ func runImportMixedVersions(ctx context.Context, t test.Test, c cluster.Cluster, mixedversion.MinimumSupportedVersion("v23.2.0"), // Only use the latest version of each release to work around #127029. mixedversion.AlwaysUseLatestPredecessors, + // This test sometimes flake on separate-process + // deployments. Needs investigation. + mixedversion.EnabledDeploymentModes( + mixedversion.SystemOnlyDeployment, + mixedversion.SharedProcessDeployment, + ), ) runImport := func(ctx context.Context, l *logger.Logger, r *rand.Rand, h *mixedversion.Helper) error { if err := h.Exec(r, "DROP DATABASE IF EXISTS tpcc CASCADE;"); err != nil { diff --git a/pkg/cmd/roachtest/tests/rebalance_load.go b/pkg/cmd/roachtest/tests/rebalance_load.go index 649dc27f38cc..f45034b4ab06 100644 --- a/pkg/cmd/roachtest/tests/rebalance_load.go +++ b/pkg/cmd/roachtest/tests/rebalance_load.go @@ -100,6 +100,14 @@ func registerRebalanceLoad(r registry.Registry) { mixedversion.ClusterSettingOption( install.ClusterSettingsOption(settings.ClusterSettings), ), + // This test does not currently work with shared-process + // deployments (#129389), so we do not run it in + // separate-process mode either to reduce noise. We should + // reevaluate once the test works in shared-process. + mixedversion.EnabledDeploymentModes( + mixedversion.SystemOnlyDeployment, + mixedversion.SharedProcessDeployment, + ), ) mvt.OnStartup("maybe enable split/scatter on tenant", func(ctx context.Context, l *logger.Logger, r *rand.Rand, h *mixedversion.Helper) error { diff --git a/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go b/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go index efa5119527fa..6170047bac35 100644 --- a/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go +++ b/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go @@ -118,14 +118,31 @@ func runValidateSystemSchemaAfterVersionUpgrade( systemComparison := newTenantSystemSchemaComparison(install.SystemInterfaceName) var tenantComparison *tenantSystemSchemaComparison + var deploymentMode mixedversion.DeploymentMode - mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All(), + opts := []mixedversion.CustomOption{ // We limit the number of upgrades since the test is not expected to work // on versions older than 22.2. mixedversion.MaxUpgrades(3), // Fixtures are generated on a version that's too old for this test. mixedversion.NeverUseFixtures, - ) + } + + if c.IsLocal() { + opts = append(opts, + // Separate-process deployments are still in its early + // days. Disable it in local clusters (such as in CI) to avoid + // disruptions. + // + // TODO(testeng): enable separate-process deployments once it is + // considered stable enough. + mixedversion.EnabledDeploymentModes( + mixedversion.SystemOnlyDeployment, + mixedversion.SharedProcessDeployment, + )) + } + + mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All(), opts...) mvt.AfterUpgradeFinalized( "obtain system schema from the upgraded cluster", func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { @@ -135,6 +152,7 @@ func runValidateSystemSchemaAfterVersionUpgrade( return nil } + deploymentMode = h.DeploymentMode() systemComparison.upgraded = obtainSystemSchema(ctx, l, c, 1, systemComparison.name) if h.IsMultitenant() { tenantComparison = newTenantSystemSchemaComparison(h.Tenant.Descriptor.Name) @@ -157,8 +175,21 @@ func runValidateSystemSchemaAfterVersionUpgrade( validateTenant := tenantComparison != nil && clusterupgrade.CurrentVersion().AtLeast(validateSystemSchemaTenantVersion) if validateTenant { - t.L().Printf("creating shared-process tenant") - startOpts := option.StartSharedVirtualClusterOpts(tenantComparison.name) + var startOpts option.StartOpts + + switch deploymentMode { + case mixedversion.SharedProcessDeployment: + t.L().Printf("creating shared-process tenant") + startOpts = option.StartSharedVirtualClusterOpts(tenantComparison.name) + + case mixedversion.SeparateProcessDeployment: + t.L().Printf("creating separate-process tenant") + startOpts = option.StartVirtualClusterOpts(tenantComparison.name, c.Node(1)) + + default: + t.Fatal(fmt.Errorf("programming error: unexpected deployment mode %q", deploymentMode)) + } + c.StartServiceForVirtualCluster(ctx, t.L(), startOpts, settings) tenantComparison.bootstrapped = obtainSystemSchema(ctx, t.L(), c, 1, tenantComparison.name) } diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go index ae0650aef359..76c853f33747 100644 --- a/pkg/cmd/roachtest/tests/versionupgrade.go +++ b/pkg/cmd/roachtest/tests/versionupgrade.go @@ -108,7 +108,18 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { var cancel context.CancelFunc testCtx, cancel = context.WithTimeout(ctx, localTimeout) defer cancel() - opts = append(opts, mixedversion.NumUpgrades(1)) + opts = append( + opts, + mixedversion.NumUpgrades(1), + // Disable separate-proces deployments in local runs, as it is + // currently failing on the `BACKUP` step, and we don't want to + // disrupt CI. Once we figure out a fix for it in nightly runs, + // we can re-enable it. + mixedversion.EnabledDeploymentModes( + mixedversion.SystemOnlyDeployment, + mixedversion.SharedProcessDeployment, + ), + ) } mvt := mixedversion.NewTest(testCtx, t, t.L(), c, c.All(), opts...) diff --git a/pkg/roachprod/install/cockroach.go b/pkg/roachprod/install/cockroach.go index b351a8af0041..a1bf488d4056 100644 --- a/pkg/roachprod/install/cockroach.go +++ b/pkg/roachprod/install/cockroach.go @@ -1428,7 +1428,7 @@ func (c *SyncedCluster) upsertVirtualClusterMetadata( ) (int, error) { runSQL := func(stmt string) (string, error) { results, err := startOpts.StorageCluster.ExecSQL( - ctx, l, startOpts.StorageCluster.Nodes[:1], "", 0, DefaultAuthMode(), "", /* database */ + ctx, l, startOpts.StorageCluster.Nodes[:1], SystemInterfaceName, 0, DefaultAuthMode(), "", /* database */ []string{"--format", "csv", "-e", stmt}) if err != nil { return "", err