diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 22d3e090f5c7..da94c79bf2b5 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -153,6 +153,7 @@ ALL_TESTS = [ "//pkg/config/zonepb:zonepb_test", "//pkg/config:config_disallowed_imports_test", "//pkg/config:config_test", + "//pkg/configprofiles:configprofiles_test", "//pkg/geo/geogen:geogen_test", "//pkg/geo/geogfn:geogfn_test", "//pkg/geo/geographiclib:geographiclib_test", @@ -1108,6 +1109,8 @@ GO_TARGETS = [ "//pkg/config/zonepb:zonepb_test", "//pkg/config:config", "//pkg/config:config_test", + "//pkg/configprofiles:configprofiles", + "//pkg/configprofiles:configprofiles_test", "//pkg/docs:docs", "//pkg/featureflag:featureflag", "//pkg/gen/genbzl:genbzl", @@ -2612,6 +2615,7 @@ GET_X_DATA_TARGETS = [ "//pkg/compose/compare/compare:get_x_data", "//pkg/config:get_x_data", "//pkg/config/zonepb:get_x_data", + "//pkg/configprofiles:get_x_data", "//pkg/docs:get_x_data", "//pkg/featureflag:get_x_data", "//pkg/gen/genbzl:get_x_data", diff --git a/pkg/base/BUILD.bazel b/pkg/base/BUILD.bazel index d39145ed26dc..b9cc043c76be 100644 --- a/pkg/base/BUILD.bazel +++ b/pkg/base/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/cli/cliflags", "//pkg/roachpb", "//pkg/security/username", + "//pkg/server/autoconfig/acprovider", "//pkg/settings/cluster", "//pkg/util", "//pkg/util/envutil", diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index 2b2edebadfab..43c09524d59d 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -168,6 +169,10 @@ type TestServerArgs struct { // ObsServiceAddr is the address to which events will be exported over OTLP. // If empty, exporting events is inhibited. ObsServiceAddr string + + // AutoConfigProvider provides auto-configuration tasks to apply on + // the cluster during server initialization. + AutoConfigProvider acprovider.Provider } // TestClusterArgs contains the parameters one can set when creating a test diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel index 8ee3fbb49330..95da54039830 100644 --- a/pkg/cli/BUILD.bazel +++ b/pkg/cli/BUILD.bazel @@ -119,6 +119,7 @@ go_library( "//pkg/cloud/userfile", "//pkg/clusterversion", "//pkg/config", + "//pkg/configprofiles", "//pkg/docs", "//pkg/geo/geos", "//pkg/gossip", @@ -143,6 +144,7 @@ go_library( "//pkg/security/securitytest", "//pkg/security/username", "//pkg/server", + "//pkg/server/autoconfig/acprovider", "//pkg/server/pgurl", "//pkg/server/profiler", "//pkg/server/serverpb", diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index 0c06a8bfb85e..b47b9624c2c3 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -1449,6 +1449,12 @@ Disable the creation of a default dataset in the demo shell. This makes 'cockroach demo' faster to start.`, } + ConfigProfile = FlagInfo{ + Name: "config-profile", + EnvVar: "COCKROACH_CONFIG_PROFILE", + Description: `Select a configuration profile to apply.`, + } + GeoLibsDir = FlagInfo{ Name: "spatial-libs", Description: ` diff --git a/pkg/cli/context.go b/pkg/cli/context.go index 0375f44e64a5..2d46ff787aa0 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/clientsecopts" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/storage" @@ -639,6 +640,7 @@ func setDemoContextDefaults() { demoCtx.Multitenant = true demoCtx.DisableServerController = false demoCtx.DefaultEnableRangefeeds = true + demoCtx.AutoConfigProvider = acprovider.NoTaskProvider{} demoCtx.pidFile = "" demoCtx.disableEnterpriseFeatures = false diff --git a/pkg/cli/democluster/BUILD.bazel b/pkg/cli/democluster/BUILD.bazel index d94ad0013c8d..445c7058a842 100644 --- a/pkg/cli/democluster/BUILD.bazel +++ b/pkg/cli/democluster/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "//pkg/security/certnames", "//pkg/security/username", "//pkg/server", + "//pkg/server/autoconfig/acprovider", "//pkg/server/pgurl", "//pkg/server/serverpb", "//pkg/server/status", diff --git a/pkg/cli/democluster/context.go b/pkg/cli/democluster/context.go index 9fe8a420709f..46017f856406 100644 --- a/pkg/cli/democluster/context.go +++ b/pkg/cli/democluster/context.go @@ -14,6 +14,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/cli/clicfg" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider" "github.com/cockroachdb/cockroach/pkg/workload" ) @@ -110,6 +111,10 @@ type Context struct { // DisableServerController is true if we want to avoid the server // controller to instantiate tenant secondary servers. DisableServerController bool + + // AutoConfigProvider provides auto-configuration tasks to apply on + // the cluster during server initialization. + AutoConfigProvider acprovider.Provider } // IsInteractive returns true if the demo cluster configuration diff --git a/pkg/cli/democluster/demo_cluster.go b/pkg/cli/democluster/demo_cluster.go index 7eaa775eb1a5..ae01df358cf5 100644 --- a/pkg/cli/democluster/demo_cluster.go +++ b/pkg/cli/democluster/demo_cluster.go @@ -884,6 +884,7 @@ func (demoCtx *Context) testServerArgsForTransientCluster( StoreSpecs: []base.StoreSpec{storeSpec}, SQLMemoryPoolSize: demoCtx.SQLPoolMemorySize, CacheSize: demoCtx.CacheSize, + AutoConfigProvider: demoCtx.AutoConfigProvider, NoAutoInitializeCluster: true, EnableDemoLoginEndpoint: true, // Demo clusters by default will create their own tenants, so we diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index 284ca090ee4a..8ac6e8cfb17e 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cli/clienturl" "github.com/cockroachdb/cockroach/pkg/cli/cliflagcfg" "github.com/cockroachdb/cockroach/pkg/cli/cliflags" + "github.com/cockroachdb/cockroach/pkg/configprofiles" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -393,6 +394,10 @@ func init() { // planning? if cmd != connectInitCmd && cmd != connectJoinCmd { cliflagcfg.StringFlag(f, &serverCfg.Attrs, cliflags.Attrs) + // Cluster initialization. We only do this for a regular start command; + // SQL-only servers get their initialization payload from their tenant + // configuration. + cliflagcfg.VarFlag(f, configprofiles.NewProfileSetter(&serverCfg.AutoConfigProvider), cliflags.ConfigProfile) } } @@ -828,6 +833,7 @@ func init() { cliflagcfg.IntFlag(f, &demoCtx.HTTPPort, cliflags.DemoHTTPPort) cliflagcfg.StringFlag(f, &demoCtx.ListeningURLFile, cliflags.ListeningURLFile) cliflagcfg.StringFlag(f, &demoCtx.pidFile, cliflags.PIDFile) + cliflagcfg.VarFlag(f, configprofiles.NewProfileSetter(&demoCtx.AutoConfigProvider), cliflags.ConfigProfile) } { diff --git a/pkg/configprofiles/BUILD.bazel b/pkg/configprofiles/BUILD.bazel new file mode 100644 index 000000000000..94238c572e35 --- /dev/null +++ b/pkg/configprofiles/BUILD.bazel @@ -0,0 +1,49 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "configprofiles", + srcs = [ + "doc.go", + "profiles.go", + "provider.go", + "setter.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/configprofiles", + visibility = ["//visibility:public"], + deps = [ + "//pkg/clusterversion", + "//pkg/server/autoconfig/acprovider", + "//pkg/server/autoconfig/autoconfigpb", + "//pkg/util/log", + "//pkg/util/syncutil", + "@com_github_cockroachdb_errors//:errors", + "@com_github_spf13_pflag//:pflag", + ], +) + +go_test( + name = "configprofiles_test", + srcs = [ + "main_test.go", + "profiles_test.go", + ], + args = ["-test.timeout=295s"], + embed = [":configprofiles"], + deps = [ + "//pkg/base", + "//pkg/build", + "//pkg/ccl", + "//pkg/security/securityassets", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/server/autoconfig/autoconfigpb", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/log", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/configprofiles/doc.go b/pkg/configprofiles/doc.go new file mode 100644 index 000000000000..a1867708e1a3 --- /dev/null +++ b/pkg/configprofiles/doc.go @@ -0,0 +1,16 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Package configprofiles contain static configuration profiles embedded +// inside the CockroachDB binary. +// +// Configuration profiles are ways to initialize CockroachDB clusters +// differently (upon cluster creation) depending on purpose. +package configprofiles diff --git a/pkg/configprofiles/main_test.go b/pkg/configprofiles/main_test.go new file mode 100644 index 000000000000..ca8d5095ced1 --- /dev/null +++ b/pkg/configprofiles/main_test.go @@ -0,0 +1,38 @@ +// Copyright 2015 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package configprofiles + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/build" + "github.com/cockroachdb/cockroach/pkg/security/securityassets" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" +) + +func init() { + securityassets.SetLoader(securitytest.EmbeddedAssets) +} + +func TestMain(m *testing.M) { + // CLI tests are sensitive to the server version, but test binaries don't have + // a version injected. Pretend to be a very up-to-date version. + defer build.TestingOverrideVersion("v999.0.0")() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} + +//go:generate ../util/leaktest/add-leaktest.sh *_test.go diff --git a/pkg/configprofiles/profiles.go b/pkg/configprofiles/profiles.go new file mode 100644 index 000000000000..c6594dea8651 --- /dev/null +++ b/pkg/configprofiles/profiles.go @@ -0,0 +1,174 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package configprofiles + +import ( + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/autoconfigpb" +) + +// alias represents a name that points to a pre-defined profile with +// an alternate description. +type alias struct { + aliasTarget string + description string +} + +var aliases = map[string]alias{ + "replication-source": { + aliasTarget: "multitenant+app+sharedservice", + description: "configuration suitable for a replication source cluster", + }, + "replication-target": { + aliasTarget: "multitenant+noapp", + description: "configuration suitable for a replication target cluster", + }, +} + +const defaultProfileName = "default" + +type configProfile struct { + description string + tasks []autoconfigpb.Task +} + +var staticProfiles = map[string]configProfile{ + defaultProfileName: { + // Do not define tasks in the "default" profile. It should continue + // to generate clusters with default configurations. + description: "no extra configuration applied - using source code defaults", + }, + // The "example" profile exists for demonstration and documentation + // purposes. + "example": { + description: "creates an 'example' database and data table, for illustration purposes", + tasks: []autoconfigpb.Task{ + makeTask("create an example database", + /* nonTxnSQL */ []string{ + "CREATE DATABASE IF NOT EXISTS example", + "CREATE TABLE IF NOT EXISTS example.data AS SELECT 'hello' AS value", + }, + nil, /* txnSQL */ + ), + }, + }, + "multitenant+noapp": { + description: "multi-tenant cluster with no secondary tenant defined yet", + tasks: multitenantClusterInitTasks, + }, + "multitenant+app+sharedservice": { + description: "multi-tenant cluster with one secondary tenant configured to serve SQL application traffic", + tasks: append( + multitenantClusterInitTasks, + makeTask("create an application tenant", + nil, /* nonTxnSQL */ + /* txnSQL */ []string{ + // Create the app tenant record. + "CREATE TENANT application", + // Run the service for the application tenant. + "ALTER TENANT application START SERVICE SHARED", + }, + ), + makeTask("activate application tenant", + /* nonTxnSQL */ []string{ + // Make the app tenant receive SQL connections by default. + "SET CLUSTER SETTING server.controller.default_tenant = 'application'", + }, + nil, /* txnSQL */ + ), + ), + }, +} + +var multitenantClusterInitTasks = []autoconfigpb.Task{ + makeTask("initial cluster config", + /* nonTxnSQL */ []string{ + // Disable RU limits. + "SET CLUSTER SETTING kv.tenant_rate_limiter.burst_limit_seconds = 10000000", + "SET CLUSTER SETTING kv.tenant_rate_limiter.rate_limit = -1000", + "SET CLUSTER SETTING kv.tenant_rate_limiter.read_batch_cost = 0", + "SET CLUSTER SETTING kv.tenant_rate_limiter.read_cost_per_mebibyte = 0", + "SET CLUSTER SETTING kv.tenant_rate_limiter.write_cost_per_megabyte = 0", + "SET CLUSTER SETTING kv.tenant_rate_limiter.write_request_cost = 0", + // Disable trace redaction (this ought to be configurable per-tenant, but is not possible yet in v23.1). + "SET CLUSTER SETTING server.secondary_tenants.redact_trace.enabled = false", + // Enable zone config changes in secondary tenants (this ought to be configurable per-tenant, but is not possible yet in v23.1). + "SET CLUSTER SETTING sql.zone_configs.allow_for_secondary_tenant.enabled = true", + // Enable multi-region abstractions in secondary tenants. + "SET CLUSTER SETTING sql.multi_region.allow_abstractions_for_secondary_tenants.enabled = true", + // Disable range coalescing (as long as the problems related + // to range coalescing have not been solved yet). + "SET CLUSTER SETTING spanconfig.storage_coalesce_adjacent.enabled = false", + "SET CLUSTER SETTING spanconfig.tenant_coalesce_adjacent.enabled = false", + // Make the operator double-check tenant deletions. + "SET CLUSTER SETTING sql.drop_tenant.enabled = false", + }, + nil, /* txnSQL */ + ), + makeTask("create tenant template", + nil, /* nonTxnSQL */ + []string{ + // Create a main secondary tenant template. + "CREATE TENANT template", + "ALTER TENANT template GRANT CAPABILITY can_admin_relocate_range, can_admin_unsplit, can_view_node_info, can_view_tsdb_metrics, exempt_from_rate_limiting", + // Disable span config limits and splitter. + // TODO(knz): Move this to in-tenant config task. + "ALTER TENANT template SET CLUSTER SETTING spanconfig.tenant_limit = 1000000", + // "ALTER TENANT template SET CLUSTER SETTING spanconfig.tenant_split.enabled = false", + // Disable RU accounting. + // TODO(knz): Move this to in-tenant config task. + "SELECT crdb_internal.update_tenant_resource_limits('template', 10000000000, 0, 10000000000, now(), 0)", + // Enable admin scatter/split in tenant SQL. + // TODO(knz): Move this to in-tenant config task. + "ALTER TENANT template SET CLUSTER SETTING sql.scatter.allow_for_secondary_tenant.enabled = true", + "ALTER TENANT template SET CLUSTER SETTING sql.split_at.allow_for_secondary_tenant.enabled = true", + }, + ), + // Finally. + makeTask("use the application tenant template by default in CREATE TENANT", + /* nonTxnSQL */ []string{ + "SET CLUSTER SETTING sql.create_tenant.default_template = 'template'", + }, + nil, /* txnSQL */ + ), +} + +func makeTask(description string, nonTxnSQL, txnSQL []string) autoconfigpb.Task { + return autoconfigpb.Task{ + Description: description, + MinVersion: clusterversion.ByKey(clusterversion.BinaryVersionKey), + Payload: &autoconfigpb.Task_SimpleSQL{ + SimpleSQL: &autoconfigpb.SimpleSQL{ + NonTransactionalStatements: nonTxnSQL, + TransactionalStatements: txnSQL, + }, + }, + } +} + +func init() { + // Give all tasks monotonically increasing IDs. + for _, p := range staticProfiles { + taskID := autoconfigpb.TaskID(1) + for i := range p.tasks { + p.tasks[i].TaskID = taskID + taskID++ + } + } +} + +func TestingGetProfiles() map[string][]autoconfigpb.Task { + v := make(map[string][]autoconfigpb.Task, len(staticProfiles)) + for n, p := range staticProfiles { + v[n] = p.tasks + } + return v +} diff --git a/pkg/configprofiles/profiles_test.go b/pkg/configprofiles/profiles_test.go new file mode 100644 index 000000000000..e865532f675a --- /dev/null +++ b/pkg/configprofiles/profiles_test.go @@ -0,0 +1,89 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package configprofiles_test + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + // To enable the CCL-only functions. + "github.com/cockroachdb/cockroach/pkg/ccl" + "github.com/cockroachdb/cockroach/pkg/configprofiles" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/autoconfigpb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// TestProfilesValidSQL checks that the SQL of each static +// configuration profile can be applied successfully on a new cluster. +func TestProfilesValidSQL(t *testing.T) { + defer leaktest.AfterTest(t)() + defer ccl.TestingEnableEnterprise()() + + for profileName, tasks := range configprofiles.TestingGetProfiles() { + t.Run(profileName, func(t *testing.T) { + defer log.Scope(t).Close(t) + + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + DisableDefaultTestTenant: true, + }) + defer s.Stopper().Stop(context.Background()) + db := sqlutils.MakeSQLRunner(sqlDB) + + for _, task := range tasks { + t.Run(fmt.Sprintf("%d/%s", task.TaskID, task.Description), func(t *testing.T) { + switch payload := task.GetPayload().(type) { + case *autoconfigpb.Task_SimpleSQL: + for _, stmt := range payload.SimpleSQL.NonTransactionalStatements { + t.Logf("non-txn statement: %s", stmt) + db.Exec(t, stmt) + // Try it a second time too -- it should work since + // non-txn statements are supposed to be idempotent + // (because jobs can be retried). + db.Exec(t, stmt) + } + db.Exec(t, "BEGIN TRANSACTION") + for _, stmt := range payload.SimpleSQL.TransactionalStatements { + t.Logf("txn statement: %s", stmt) + db.Exec(t, stmt) + } + db.Exec(t, "COMMIT TRANSACTION") + + default: + t.Fatalf("unsupported payload type %T", payload) + } + }) + } + }) + } +} + +// TestMonotonicTaskIDs checks that the task IDs of each profile +// are increasing monotonically. +func TestMonotonicTaskIDs(t *testing.T) { + defer leaktest.AfterTest(t)() + + for profileName, tasks := range configprofiles.TestingGetProfiles() { + t.Run(profileName, func(t *testing.T) { + prevTaskID := autoconfigpb.TaskID(0) + for taskNum, task := range tasks { + if task.TaskID <= prevTaskID { + t.Fatalf("%d: task ID %d not greater than previous task ID %d", taskNum, task.TaskID, prevTaskID) + } + prevTaskID = task.TaskID + } + }) + } +} diff --git a/pkg/configprofiles/provider.go b/pkg/configprofiles/provider.go new file mode 100644 index 000000000000..c3cac0ffc6be --- /dev/null +++ b/pkg/configprofiles/provider.go @@ -0,0 +1,86 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package configprofiles + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/autoconfigpb" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +// profileTaskProvider is an implementation of acprovider.Provider +// which reports task to execute from a static configuration profile. +type profileTaskProvider struct { + syncutil.Mutex + + envID autoconfigpb.EnvironmentID + tasks []autoconfigpb.Task + nextTaskIdx int +} + +var _ acprovider.Provider = (*profileTaskProvider)(nil) + +// EnvUpdate is part of the acprovider.Provider interface. +func (p *profileTaskProvider) EnvUpdate() <-chan struct{} { + ch := make(chan struct{}, 1) + ch <- struct{}{} + return ch +} + +// ActiveEnvironments is part of the acprovider.Provider interface. +func (p *profileTaskProvider) ActiveEnvironments() []autoconfigpb.EnvironmentID { + p.Lock() + defer p.Unlock() + if p.nextTaskIdx >= len(p.tasks) { + return nil + } + return []autoconfigpb.EnvironmentID{p.envID} +} + +// Peek is part of the acprovider.Provider interface. +func (p *profileTaskProvider) Peek( + ctx context.Context, envID autoconfigpb.EnvironmentID, +) (autoconfigpb.Task, error) { + p.Lock() + defer p.Unlock() + if p.envID != envID { + return autoconfigpb.Task{}, acprovider.ErrNoMoreTasks + } + if p.nextTaskIdx >= len(p.tasks) { + return autoconfigpb.Task{}, acprovider.ErrNoMoreTasks + } + task := p.tasks[p.nextTaskIdx] + return task, nil +} + +// Pop is part of the acprovider.Provider interface. +func (p *profileTaskProvider) Pop( + ctx context.Context, envID autoconfigpb.EnvironmentID, completedTaskID autoconfigpb.TaskID, +) error { + p.Lock() + defer p.Unlock() + if p.envID != envID { + // Not this environment. Nothing to do. + return nil + } + for p.nextTaskIdx < len(p.tasks) { + if completedTaskID >= p.tasks[p.nextTaskIdx].TaskID { + log.Infof(ctx, "sliding task %d out of queue", p.tasks[p.nextTaskIdx].TaskID) + p.nextTaskIdx++ + continue + } + break + } + return nil +} diff --git a/pkg/configprofiles/setter.go b/pkg/configprofiles/setter.go new file mode 100644 index 000000000000..dc699936cc58 --- /dev/null +++ b/pkg/configprofiles/setter.go @@ -0,0 +1,125 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package configprofiles + +import ( + "fmt" + "math" + "sort" + "strings" + "text/tabwriter" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/autoconfigpb" + "github.com/cockroachdb/errors" + "github.com/spf13/pflag" +) + +// profileSetter is an implementation of pflag.Value +// which is able to define a configuration provider +// from a command-line flag (or env var). +type profileSetter struct { + profileName string + taskProvider *acprovider.Provider +} + +// NewProfileSetter initializes a profile setter able to assign the +// given Provider reference. +func NewProfileSetter(provider *acprovider.Provider) pflag.Value { + return &profileSetter{ + profileName: defaultProfileName, + taskProvider: provider, + } +} + +// String implements the pflag.Value interface. +func (ps *profileSetter) String() string { + return ps.profileName +} + +// Type implements the pflag.Value interface. +func (ps *profileSetter) Type() string { return "" } + +// Set implements the pflag.Value interface. +func (ps *profileSetter) Set(v string) error { + p, ok := staticProfiles[v] + if !ok { + return errors.Newf("unknown profile: %q\n%s", v, profileHelp) + } + + // Add a pseudo-task with the maximum possible task ID, to ensure no + // more task will ever be run by this profile. This makes it + // possible to add new tasks to an existing config profile to apply + // to new clusters, without impacting clusters created in previous + // versions with the same profile name. + tasks := append(p.tasks, endProfileTask) + + *ps.taskProvider = &profileTaskProvider{ + // The envID defines the idempotency key for the application of + // configuration profiles on new and existing clusters. + // + // This string (and corresponding UUID) key must never change for + // the static config profiles, so that the profiles never get + // applied more than once after a cluster is initialized. + envID: "staticprofile:c623b0e0-633d-4014-9eaf-d13d0a15d2a1", + tasks: tasks, + } + return nil +} + +// endProfileTask is an empty task which terminates the sequence of +// tasks in a static profile. It uses MaxUint64 as task ID to ensure +// that no more tasks will ever execute on a given cluster after the +// profile has been applied once. +var endProfileTask = autoconfigpb.Task{ + TaskID: autoconfigpb.TaskID(math.MaxUint64), + Description: "end of configuration profile", + MinVersion: clusterversion.ByKey(clusterversion.BinaryVersionKey), + Payload: &autoconfigpb.Task_SimpleSQL{ + SimpleSQL: &autoconfigpb.SimpleSQL{}, + }, +} + +// profileHelp is a help text that documents available profiles. +// It also checks that aliases point to valid profiles. +var profileHelp = func() string { + allNames := make([]string, 0, len(staticProfiles)+len(aliases)) + for name := range staticProfiles { + allNames = append(allNames, name) + } + for name, a := range aliases { + if _, ok := staticProfiles[name]; ok { + panic(errors.AssertionFailedf("alias name duplicates a profile name: %q", name)) + } + if _, ok := staticProfiles[a.aliasTarget]; !ok { + panic(errors.AssertionFailedf("alias %q refers to non-existent profile %q", name, a.aliasTarget)) + } + allNames = append(allNames, name) + } + sort.Strings(allNames) + + var buf strings.Builder + w := tabwriter.NewWriter(&buf, 2 /* minwidth */, 1 /* tabwidth */, 2 /* padding */, ' ', 0) + fmt.Fprintln(w, "Available profiles:") + for _, name := range allNames { + if a, ok := aliases[name]; ok { + fmt.Fprintf(w, "%s\t%s (alias for %q)\n", name, a.description, a.aliasTarget) + continue + } + p := staticProfiles[name] + fmt.Fprintf(w, "%s\t%s\n", name, p.description) + } + if err := w.Flush(); err != nil { + panic(err) + } + return buf.String() +}() diff --git a/pkg/server/config.go b/pkg/server/config.go index c1fb0dfc78ca..c3b03cbff3ed 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider" "github.com/cockroachdb/cockroach/pkg/server/status" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -257,6 +258,10 @@ type BaseConfig struct { // These events are meant for the Observability Service, but they might pass // through an OpenTelemetry Collector. ObsServiceAddr string + + // AutoConfigProvider provides auto-configuration tasks to apply on + // the cluster during server initialization. + AutoConfigProvider acprovider.Provider } // MakeBaseConfig returns a BaseConfig with default values. @@ -296,6 +301,7 @@ func (cfg *BaseConfig) SetDefaults( cfg.Stores = base.StoreSpecList{ Specs: []base.StoreSpec{storeSpec}, } + cfg.AutoConfigProvider = acprovider.NoTaskProvider{} // We use the tag "n" here for both KV nodes and SQL instances, // using the knowledge that the value part of a SQL instance ID // container will prefix the value with the string "sql", resulting diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 4d46f360e3d9..4928f014fdc7 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -51,7 +51,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/clientsecopts" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/autoconfig" - "github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider" "github.com/cockroachdb/cockroach/pkg/server/diagnostics" "github.com/cockroachdb/cockroach/pkg/server/pgurl" "github.com/cockroachdb/cockroach/pkg/server/serverpb" @@ -1026,7 +1025,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { TenantCapabilitiesReader: cfg.tenantCapabilitiesReader, // TODO(knz): We will replace this provider by an actual provider // in a later commit. - AutoConfigProvider: acprovider.NoTaskProvider{}, + AutoConfigProvider: cfg.AutoConfigProvider, } if sqlSchemaChangerTestingKnobs := cfg.TestingKnobs.SQLSchemaChanger; sqlSchemaChangerTestingKnobs != nil { diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index e84ec554bc41..ca632d0db3f3 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -254,6 +254,9 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { if params.SnapshotSendLimit != 0 { cfg.SnapshotSendLimit = params.SnapshotSendLimit } + if params.AutoConfigProvider != nil { + cfg.AutoConfigProvider = params.AutoConfigProvider + } // Ensure we have the correct number of engines. Add in-memory ones where // needed. There must be at least one store/engine.