From 43ec4e57f0fc37bf5171347dd2fab5f739caad51 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Thu, 11 Jul 2024 14:47:25 -0400 Subject: [PATCH] roachtest: change-replicas/mixed-version supports shared-process This commit updates the `change-replicas/mixed-version` roachtest in order to enable the shared-process deployment mode. This involves enabling certain features in tenants (such as the ability to split and scatter) in versions where they are not enabled by default. One of the builtins used also require a connection to the system tenant. Informs: #127378 Release note: None --- .../tests/mixed_version_change_replicas.go | 108 ++++++++++++++++-- 1 file changed, 100 insertions(+), 8 deletions(-) diff --git a/pkg/cmd/roachtest/tests/mixed_version_change_replicas.go b/pkg/cmd/roachtest/tests/mixed_version_change_replicas.go index fe94efe93067..176ff2b48c6a 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_change_replicas.go +++ b/pkg/cmd/roachtest/tests/mixed_version_change_replicas.go @@ -19,10 +19,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/mixedversion" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/errors" ) @@ -39,18 +41,18 @@ func registerChangeReplicasMixedVersion(r registry.Registry) { }) } +var v232 = clusterupgrade.MustParseVersion("v23.2.0") + // runChangeReplicasMixedVersion is a regression test for // https://github.com/cockroachdb/cockroach/issues/94834. It runs replica config // changes (moves replicas around) in mixed-version clusters, both explicitly // with ALTER RANGE RELOCATE and implicitly via zone configs and the replicate // queue. func runChangeReplicasMixedVersion(ctx context.Context, t test.Test, c cluster.Cluster) { - // createTable creates a test table, and splits and scatters it. createTable := func( ctx context.Context, l *logger.Logger, r *rand.Rand, h *mixedversion.Helper, ) error { - l.Printf("creating table") if err := h.Exec(r, `CREATE TABLE test (id INT PRIMARY KEY)`); err != nil { return err @@ -60,6 +62,11 @@ func runChangeReplicasMixedVersion(ctx context.Context, t test.Test, c cluster.C return err } + // Enable split/scatter on tenants if necessary. + if err := enableTenantSplitScatter(l, r, h); err != nil { + return err + } + l.Printf("splitting table") const ranges = 100 err := h.Exec(r, `ALTER TABLE test SPLIT AT SELECT i FROM generate_series(1, $1) AS g(i)`, @@ -85,6 +92,18 @@ func runChangeReplicasMixedVersion(ctx context.Context, t test.Test, c cluster.C node int, ) error { l.Printf("moving replicas off of n%d using zone config", node) + // Enable necessary features on tenant deployments if running on a + // version where they are not enabled by default. + if !h.Context().FromVersion.AtLeast(mixedversion.TenantsAndSystemAlignedSettingsVersion) { + for _, name := range []string{ + "sql.virtual_cluster.feature_access.zone_configs.enabled", + "sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled", + } { + if err := setTenantSetting(l, r, h, name, true); err != nil { + return errors.Wrapf(err, "setting %s", name) + } + } + } err := h.Exec(r, fmt.Sprintf( `ALTER TABLE test CONFIGURE ZONE USING constraints = '[-node%d]'`, node)) @@ -127,7 +146,7 @@ func runChangeReplicasMixedVersion(ctx context.Context, t test.Test, c cluster.C ) error { setReplicateQueueEnabled := func(enabled bool) error { for _, node := range c.All() { - _, err := h.Connect(node).ExecContext(ctx, + _, err := h.System.Connect(node).ExecContext(ctx, `SELECT crdb_internal.kv_set_queue_active('replicate', $1)`, enabled) if err != nil { return err @@ -212,6 +231,13 @@ func runChangeReplicasMixedVersion(ctx context.Context, t test.Test, c cluster.C moveReplicas := func( ctx context.Context, l *logger.Logger, r *rand.Rand, h *mixedversion.Helper, ) error { + // Skip this step if we are in multitenant mode and we are not + // running at least 23.2 yet. This function is not supported in this + // scenario as we can't configure zones. + if h.IsMultitenant() && !h.Context().FromVersion.AtLeast(v232) { + l.Printf("multitenant deployment running an unsupported version; skipping") + return nil + } // First, scatter the range. l.Printf("scattering table") @@ -253,14 +279,14 @@ func runChangeReplicasMixedVersion(ctx context.Context, t test.Test, c cluster.C } // Set up and run test. - mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All(), mixedversion.ClusterSettingOption( - // Speed up the queues. - install.EnvOption{"COCKROACH_SCAN_MAX_IDLE_TIME=10ms"}), + mvt := mixedversion.NewTest( + ctx, t, t.L(), c, c.All(), mixedversion.ClusterSettingOption( + // Speed up the queues. + install.EnvOption{"COCKROACH_SCAN_MAX_IDLE_TIME=10ms"}, + ), // Avoid repeatedly running into #114549 on earlier minor versions. // TODO(kvoli): Remove in 24.2. mixedversion.AlwaysUseLatestPredecessors, - // Multi-tenant deployments are currently unsupported. See #127378. - mixedversion.EnabledDeploymentModes(mixedversion.SystemOnlyDeployment), ) mvt.OnStartup("create test table", createTable) @@ -268,3 +294,69 @@ func runChangeReplicasMixedVersion(ctx context.Context, t test.Test, c cluster.C mvt.AfterUpgradeFinalized("move replicas", moveReplicas) mvt.Run() } + +// enableTenantSplitScatter updates cluster settings that allow +// tenants to use SPLIT AT and SCATTER. This is only performed if the +// mixedversion test is running on a multitenant deployment, and only +// if required by the active version. +func enableTenantSplitScatter(l *logger.Logger, r *rand.Rand, h *mixedversion.Helper) error { + if h.Context().FromVersion.AtLeast(mixedversion.TenantsAndSystemAlignedSettingsVersion) { + return nil + } + + settings := []string{ + "sql.split_at.allow_for_secondary_tenant.enabled", + "sql.scatter.allow_for_secondary_tenant.enabled", + } + + for _, s := range settings { + // Only enable the relevant settings if they are not already + // enabled by default. + if err := setTenantSetting(l, r, h, s, true); err != nil { + return errors.Wrapf(err, "failed to set cluster setting %s", s) + } + } + + return nil +} + +// setTenantSetting sets the cluster setting of the given name on +// the tenant created in for the mixedversion test. After setting it +// via the system tenant, it also waits until the update is visible to +// the actual tenant, making sure that statements that need the +// setting to be enabled can run successfully. +// +// It is a no-op to call this function on single tenant (system-only) +// deployments. +func setTenantSetting( + l *logger.Logger, r *rand.Rand, h *mixedversion.Helper, name string, value bool, +) error { + if !h.IsMultitenant() { + return nil + } + + if err := h.System.Exec( + r, + fmt.Sprintf(`ALTER TENANT $1 SET CLUSTER SETTING %s = $2`, name), + h.Tenant.Descriptor.Name, value, + ); err != nil { + return errors.Wrapf(err, "failed to set %s", name) + } + + return testutils.SucceedsSoonError(func() error { + var currentValue bool + if err := h.QueryRow(r, fmt.Sprintf("SHOW CLUSTER SETTING %s", name)).Scan(¤tValue); err != nil { + return errors.Wrapf(err, "failed to retrieve setting %s", name) + } + + if currentValue != value { + err := fmt.Errorf( + "waiting for setting %s: current (%t) != expected (%t)", name, currentValue, value, + ) + l.Printf("%v", err) + return err + } + + return nil + }) +}